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

[01/46] hive git commit: HIVE-20045 : Update hidden config list

Repository: hive
Updated Branches:
  refs/heads/master-txnstats d7bbc20d2 -> 4db8b1c66


HIVE-20045 : Update hidden config list


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

Branch: refs/heads/master-txnstats
Commit: b122aea4ec8775c158fff975ffa472be7bfc0711
Parents: 455b71e
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Sun Jul 1 14:15:48 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Sun Jul 1 14:15:48 2018 -0700

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b122aea4/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 9e9e987..35aee20 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -4351,6 +4351,7 @@ public class HiveConf extends Configuration {
         "Comma separated list of configuration options which are immutable at runtime"),
     HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list",
         METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname
+        + "," + DRUID_METADATA_DB_PASSWORD.varname
         // Adding the S3 credentials from Hadoop config to be hidden
         + ",fs.s3.awsAccessKeyId"
         + ",fs.s3.awsSecretAccessKey"


[38/46] hive git commit: HIVE-19326 : stats auto gather: incorrect aggregation during UNION queries (may lead to incorrect results) (Zoltan Haindrich via Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-19326 : stats auto gather: incorrect aggregation during UNION queries (may lead to incorrect results) (Zoltan Haindrich via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-txnstats
Commit: 2ca70b91917b8ce668a6fd277a32ab4f2c9a68c0
Parents: 4dbf991
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Fri May 11 11:22:00 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Tue Jul 3 10:17:06 2018 -0700

----------------------------------------------------------------------
 .../hive/accumulo/serde/AccumuloSerDe.java      |   6 +-
 .../test/resources/testconfiguration.properties |   1 +
 .../apache/hadoop/hive/ql/DriverContext.java    |  29 ++-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  15 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |  11 +-
 .../hive/ql/exec/SerializationUtilities.java    |  22 --
 .../hadoop/hive/ql/exec/TableScanOperator.java  |   1 +
 .../hive/ql/optimizer/GenMapRedUtils.java       |   4 +-
 .../hadoop/hive/ql/parse/GenTezUtils.java       |   2 +-
 .../hive/ql/stats/StatsCollectionContext.java   |  12 +-
 .../hive/ql/stats/fs/FSStatsPublisher.java      |  15 +-
 .../queries/clientpositive/autoColumnStats_9.q  |   3 +
 .../queries/clientpositive/union_fast_stats.q   |   6 +
 .../queries/clientpositive/union_rowcounts.q    |  51 ++++
 .../test/queries/clientpositive/union_stats.q   |  44 ++--
 .../clientpositive/autoColumnStats_9.q.out      |  13 +-
 .../clientpositive/llap/multiMapJoin1.q.out     | 150 ++++++------
 .../clientpositive/llap/union_fast_stats.q.out  |  58 ++---
 .../clientpositive/llap/union_rowcounts.q.out   | 180 ++++++++++++++
 .../clientpositive/llap/union_stats.q.out       | 145 ++++++++++-
 .../results/clientpositive/union_stats.q.out    | 243 +++++++++++++++++++
 .../apache/hadoop/hive/serde2/Deserializer.java |   2 +
 22 files changed, 816 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloSerDe.java
----------------------------------------------------------------------
diff --git a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloSerDe.java b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloSerDe.java
index fcd819b..6fa48dd 100644
--- a/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloSerDe.java
+++ b/accumulo-handler/src/java/org/apache/hadoop/hive/accumulo/serde/AccumuloSerDe.java
@@ -54,6 +54,7 @@ public class AccumuloSerDe extends AbstractSerDe {
 
   private static final Logger log = LoggerFactory.getLogger(AccumuloSerDe.class);
 
+  @Override
   public void initialize(Configuration conf, Properties properties) throws SerDeException {
     accumuloSerDeParameters = new AccumuloSerDeParameters(conf, properties, getClass().getName());
 
@@ -109,6 +110,7 @@ public class AccumuloSerDe extends AbstractSerDe {
     return cachedRow;
   }
 
+  @Override
   public Class<? extends Writable> getSerializedClass() {
     return Mutation.class;
   }
@@ -135,12 +137,14 @@ public class AccumuloSerDe extends AbstractSerDe {
     return cachedRow;
   }
 
+  @Override
   public ObjectInspector getObjectInspector() throws SerDeException {
     return cachedObjectInspector;
   }
 
+  @Override
   public SerDeStats getSerDeStats() {
-    throw new UnsupportedOperationException("SerdeStats not supported.");
+    return null;
   }
 
   public AccumuloSerDeParameters getParams() {

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 385b71e..d02c0fe 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -728,6 +728,7 @@ minillaplocal.query.files=\
   udaf_collect_set_2.q,\
   udaf_all_keyword.q,\
   union_fast_stats.q,\
+  union_rowcounts.q,\
   union_remove_26.q,\
   union_top_level.q,\
   update_access_time_non_current_db.q, \

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java b/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
index bda1079..2dd83fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/DriverContext.java
@@ -18,28 +18,27 @@
 
 package org.apache.hadoop.hive.ql;
 
-import org.apache.hadoop.hive.ql.exec.StatsTask;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.NodeUtils;
 import org.apache.hadoop.hive.ql.exec.NodeUtils.Function;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.StatsTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskRunner;
 import org.apache.hadoop.hive.ql.exec.mr.MapRedTask;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.MapWork;
 import org.apache.hadoop.hive.ql.plan.ReduceWork;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Iterator;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -223,7 +222,11 @@ public class DriverContext {
       }
     });
     for (String statKey : statKeys) {
-      statsTasks.get(statKey).getWork().setSourceTask(mapredTask);
+      if (statsTasks.containsKey(statKey)) {
+        statsTasks.get(statKey).getWork().setSourceTask(mapredTask);
+      } else {
+        LOG.debug("There is no correspoing statTask for: " + statKey);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 949a9e8..267d602 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -90,11 +90,9 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import org.apache.hive.common.util.HiveStringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
 /**
  * File Sink operator implementation.
  **/
@@ -321,7 +319,9 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
           // affects some less obscure scenario.
           try {
             FileSystem fpfs = finalPath.getFileSystem(hconf);
-            if (fpfs.exists(finalPath)) throw new RuntimeException(finalPath + " already exists");
+            if (fpfs.exists(finalPath)) {
+              throw new RuntimeException(finalPath + " already exists");
+            }
           } catch (IOException e) {
             throw new RuntimeException(e);
           }
@@ -354,7 +354,9 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     }
 
     public Path buildTaskOutputTempPath() {
-      if (taskOutputTempPathRoot == null) return null;
+      if (taskOutputTempPathRoot == null) {
+        return null;
+      }
       assert subdirForTxn == null;
       String pathStr = taskOutputTempPathRoot.toString();
       if (subdirBeforeTxn != null) {
@@ -457,7 +459,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     // 'Parent'
     boolean isLinked = conf.isLinkedFileSink();
     if (!isLinked) {
-      // Simple case - no union. 
+      // Simple case - no union.
       specPath = conf.getDirName();
       unionPath = null;
     } else {
@@ -1521,7 +1523,8 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         }
       }
     }
-    sContext.setIndexForTezUnion(this.getIndexForTezUnion());
+    sContext.setContextSuffix(getOperatorId());
+
     if (!statsPublisher.closeConnection(sContext)) {
       LOG.error("Failed to close stats");
       // The original exception is lost.

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/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 acadb43..38316bf 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
@@ -86,7 +86,6 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   private transient boolean rootInitializeCalled = false;
   protected transient long numRows = 0;
   protected transient long runTimeNumRows = 0;
-  protected int indexForTezUnion = -1;
   private transient Configuration hconf;
   protected final transient Collection<Future<?>> asyncInitOperations = new HashSet<>();
   private String marker;
@@ -1557,8 +1556,8 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   private void publishRunTimeStats() throws HiveException {
     StatsPublisher statsPublisher = new FSStatsPublisher();
     StatsCollectionContext sContext = new StatsCollectionContext(hconf);
-    sContext.setIndexForTezUnion(indexForTezUnion);
     sContext.setStatsTmpDir(conf.getRuntimeStatsTmpDir());
+    sContext.setContextSuffix(getOperatorId());
 
     if (!statsPublisher.connect(sContext)) {
       LOG.error("StatsPublishing error: cannot connect to database");
@@ -1580,14 +1579,6 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
     }
   }
 
-  public int getIndexForTezUnion() {
-    return indexForTezUnion;
-  }
-
-  public void setIndexForTezUnion(int indexForTezUnion) {
-    this.indexForTezUnion = indexForTezUnion;
-  }
-
   /**
    * Decides whether two operators are logically the same.
    * This can be used to merge same operators and avoid repeated computation.

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
index ed1566f..e03429b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
@@ -668,28 +668,6 @@ public class SerializationUtilities {
     return result;
   }
 
-  public static List<Operator<?>> cloneOperatorTree(List<Operator<?>> roots, int indexForTezUnion) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
-    CompilationOpContext ctx = roots.isEmpty() ? null : roots.get(0).getCompilationOpContext();
-    serializePlan(roots, baos, true);
-    @SuppressWarnings("unchecked")
-    List<Operator<?>> result =
-        deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
-            roots.getClass(), true);
-    // Restore the context.
-    LinkedList<Operator<?>> newOps = new LinkedList<>(result);
-    while (!newOps.isEmpty()) {
-      Operator<?> newOp = newOps.poll();
-      newOp.setIndexForTezUnion(indexForTezUnion);
-      newOp.setCompilationOpContext(ctx);
-      List<Operator<?>> children = newOp.getChildOperators();
-      if (children != null) {
-        newOps.addAll(children);
-      }
-    }
-    return result;
-  }
-
   /**
    * Clones using the powers of XML. Do not use unless necessary.
    * @param plan The plan.

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
index 0799181..5780bd4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
@@ -349,6 +349,7 @@ public class TableScanOperator extends Operator<TableScanDesc> implements
     StatsPublisher statsPublisher = Utilities.getStatsPublisher(jc);
     StatsCollectionContext sc = new StatsCollectionContext(jc);
     sc.setStatsTmpDir(conf.getTmpStatsDir());
+    sc.setContextSuffix(getOperatorId());
     if (!statsPublisher.connect(sc)) {
       // just return, stats gathering should not block the main query.
       if (LOG.isInfoEnabled()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index 605bb09..fa92385 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -1907,12 +1907,12 @@ public final class GenMapRedUtils {
         mvTasks, fsOp.getConf().getFinalDirName(), fsOp.getConf().isMmTable());
 
     // TODO: wtf?!! why is this in this method? This has nothing to do with anything.
-    if (mvTask != null && isInsertTable && hconf.getBoolVar(ConfVars.HIVESTATSAUTOGATHER)
+    if (isInsertTable && hconf.getBoolVar(ConfVars.HIVESTATSAUTOGATHER)
         && !fsOp.getConf().isMaterialization()) {
       // mark the MapredWork and FileSinkOperator for gathering stats
       fsOp.getConf().setGatherStats(true);
       fsOp.getConf().setStatsReliable(hconf.getBoolVar(ConfVars.HIVE_STATS_RELIABLE));
-      if (!mvTask.hasFollowingStatsTask()) {
+      if (mvTask != null && !mvTask.hasFollowingStatsTask()) {
         GenMapRedUtils.addStatsTask(fsOp, mvTask, currTask, hconf);
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/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 741833b..bb0de94 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
@@ -219,7 +219,7 @@ public class GenTezUtils {
     roots.addAll(context.eventOperatorSet);
 
     // need to clone the plan.
-    List<Operator<?>> newRoots = SerializationUtilities.cloneOperatorTree(roots, indexForTezUnion);
+    List<Operator<?>> newRoots = SerializationUtilities.cloneOperatorTree(roots);
 
     // we're cloning the operator plan but we're retaining the original work. That means
     // that root operators have to be replaced with the cloned ops. The replacement map

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsCollectionContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsCollectionContext.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsCollectionContext.java
index 5c3328c..e5ed621 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsCollectionContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/StatsCollectionContext.java
@@ -29,13 +29,13 @@ import org.apache.hadoop.hive.ql.exec.Task;
 
 
 @InterfaceAudience.Public
-@InterfaceStability.Stable
+@InterfaceStability.Unstable
 public class StatsCollectionContext {
 
   private final Configuration hiveConf;
   private Task task;
   private List<String> statsTmpDirs;
-  private int indexForTezUnion;
+  private String contextSuffix;
 
   public List<String> getStatsTmpDirs() {
     return statsTmpDirs;
@@ -67,11 +67,11 @@ public class StatsCollectionContext {
     this.task = task;
   }
 
-  public int getIndexForTezUnion() {
-    return indexForTezUnion;
+  public void setContextSuffix(String suffix) {
+    this.contextSuffix = suffix;
   }
 
-  public void setIndexForTezUnion(int indexForTezUnion) {
-    this.indexForTezUnion = indexForTezUnion;
+  public String getContextSuffix() {
+    return contextSuffix;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/java/org/apache/hadoop/hive/ql/stats/fs/FSStatsPublisher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/stats/fs/FSStatsPublisher.java b/ql/src/java/org/apache/hadoop/hive/ql/stats/fs/FSStatsPublisher.java
index 902b37f..67ab51d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/stats/fs/FSStatsPublisher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/fs/FSStatsPublisher.java
@@ -95,16 +95,17 @@ public class FSStatsPublisher implements StatsPublisher {
   public boolean closeConnection(StatsCollectionContext context) {
     List<String> statsDirs = context.getStatsTmpDirs();
     assert statsDirs.size() == 1 : "Found multiple stats dirs: " + statsDirs;
+    if (context.getContextSuffix() == null) {
+      throw new RuntimeException("ContextSuffix must be set before publishing!");
+    }
+
     Path statsDir = new Path(statsDirs.get(0));
     try {
-      Path statsFile = null;
-      if (context.getIndexForTezUnion() != -1) {
-        statsFile = new Path(statsDir, StatsSetupConst.STATS_FILE_PREFIX
-            + conf.getInt("mapred.task.partition", 0) + "_" + context.getIndexForTezUnion());
-      } else {
-        statsFile = new Path(statsDir, StatsSetupConst.STATS_FILE_PREFIX
-            + conf.getInt("mapred.task.partition", 0));
+      String suffix = Integer.toString(conf.getInt("mapred.task.partition", 0));
+      if (context.getContextSuffix() != null) {
+        suffix += "_" + context.getContextSuffix();
       }
+      Path statsFile = new Path(statsDir, StatsSetupConst.STATS_FILE_PREFIX + suffix);
       Utilities.FILE_OP_LOGGER.trace("About to create stats file for this task : {}", statsFile);
       Output output = new Output(statsFile.getFileSystem(conf).create(statsFile,true));
       LOG.debug("Created file : " + statsFile);

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/queries/clientpositive/autoColumnStats_9.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/autoColumnStats_9.q b/ql/src/test/queries/clientpositive/autoColumnStats_9.q
index 2b9eb82..63c94e6 100644
--- a/ql/src/test/queries/clientpositive/autoColumnStats_9.q
+++ b/ql/src/test/queries/clientpositive/autoColumnStats_9.q
@@ -17,6 +17,9 @@ INSERT OVERWRITE TABLE dest_j1_n23 SELECT src1.key, src2.value;
 FROM src src1 JOIN src src2 ON (src1.key = src2.key)
 INSERT OVERWRITE TABLE dest_j1_n23 SELECT src1.key, src2.value;
 
+
+select 'cnt, check desc',count(*) from dest_j1_n23 group by key*key >= 0;
+
 desc formatted dest_j1_n23;
 
 desc formatted dest_j1_n23 key;

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/queries/clientpositive/union_fast_stats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/union_fast_stats.q b/ql/src/test/queries/clientpositive/union_fast_stats.q
index 221fbc1..6359482 100644
--- a/ql/src/test/queries/clientpositive/union_fast_stats.q
+++ b/ql/src/test/queries/clientpositive/union_fast_stats.q
@@ -60,8 +60,14 @@ create table small_alltypesorc_a_n2 stored as orc as select * from
 
 desc formatted small_alltypesorc_a_n2;
 
+set hive.optimize.metadataonly=true;
+select 15,count(*) from small_alltypesorc_a_n2;
+
 ANALYZE TABLE small_alltypesorc_a_n2 COMPUTE STATISTICS;
 
+-- select assert_true(15=count(*)) from small_alltypesorc_a_n2;
+select 15,count(*) from small_alltypesorc_a_n2;
+
 desc formatted small_alltypesorc_a_n2;
 
 insert into table small_alltypesorc_a_n2 select * from small_alltypesorc1a_n2;

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/queries/clientpositive/union_rowcounts.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/union_rowcounts.q b/ql/src/test/queries/clientpositive/union_rowcounts.q
new file mode 100644
index 0000000..2dc5c06
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/union_rowcounts.q
@@ -0,0 +1,51 @@
+create table t1 (a int) ;
+insert into t1 values (1);
+
+create table t2a  as 
+        select * from t1
+        union all
+        select * from t1
+;
+
+select 2,count(*) from t2a;
+
+create table t2b  as select * from
+(
+        select * from (select * from t1) sq1
+        union all
+        select * from (select * from t1) sq2
+) tt
+;
+
+
+select 2,count(*) from t2b;
+
+drop table if exists t1;
+drop table if exists t2a;
+drop table if exists t2b;
+
+set hive.merge.tezfiles=true;
+
+create table t1 (a int) stored as orc;
+insert into t1 values (1);
+
+analyze table t1 compute statistics for columns;
+
+create table t2a stored as orc as
+	select * from t1
+	union all
+	select * from t1
+;
+
+select 2,count(*) from t2a;
+
+create table t2b stored as orc as select * from
+(
+	select * from (select * from t1) sq1
+	union all
+	select * from (select * from t1) sq2
+) tt
+;
+
+
+select 2,count(*) from t2b;

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/queries/clientpositive/union_stats.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/union_stats.q b/ql/src/test/queries/clientpositive/union_stats.q
index 80856ed..f36e15c 100644
--- a/ql/src/test/queries/clientpositive/union_stats.q
+++ b/ql/src/test/queries/clientpositive/union_stats.q
@@ -1,30 +1,38 @@
---! qt_n4:dataset_n4:src
-explain extended create table t_n4 as select_n4 * from src union all select_n4 * from src;
+--! qt:dataset:src
+explain extended create table t as select * from src union all select * from src;
 
-create table t_n4 as select_n4 * from src union all select_n4 * from src;
+create table t as select * from src union all select * from src;
 
-select_n4 count_n4(1) from t_n4;
+select count(1) from t;
 
-desc formatted t_n4;
+desc formatted t;
 
-create table tt_n4 as select_n4 * from t_n4 union all select_n4 * from src;
+create table tt as select * from t union all select * from src;
 
-desc formatted tt_n4;
+desc formatted tt;
 
-drop table tt_n4;
+drop table tt;
 
-create table tt_n4 as select_n4 * from src union all select_n4 * from t_n4;
+create table tt as select * from src union all select * from t;
 
-desc formatted tt_n4;
+desc formatted tt;
 
-create table t1_n26 like src;
-create table t2_n17 like src;
+create table t1 like src;
+create table t2 like src;
+create table t3 like src;
 
-from (select_n4 * from src union all select_n4 * from src)s
-insert_n4 overwrite table t1_n26 select_n4 *
-insert_n4 overwrite table t2_n17 select_n4 *;
+set hive.explain.user=true;
+explain from (select * from src union all select * from src)s
+insert overwrite table t1 select *
+insert overwrite table t2 select *
+insert overwrite table t3 select *;
 
-desc formatted t1_n26;
-desc formatted t2_n17;
+from (select * from src union all select * from src)s
+insert overwrite table t1 select *
+insert overwrite table t2 select *
+insert overwrite table t3 select *;
 
-select_n4 count_n4(1) from t1_n26;
+desc formatted t1;
+desc formatted t2;
+
+select count(1) from t1;

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/autoColumnStats_9.q.out b/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
index 8a94108..1c3db7c 100644
--- a/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
+++ b/ql/src/test/results/clientpositive/autoColumnStats_9.q.out
@@ -209,6 +209,15 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: default@dest_j1_n23
 POSTHOOK: Lineage: dest_j1_n23.key EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: dest_j1_n23.value SIMPLE [(src)src2.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select 'cnt, check desc',count(*) from dest_j1_n23 group by key*key >= 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest_j1_n23
+#### A masked pattern was here ####
+POSTHOOK: query: select 'cnt, check desc',count(*) from dest_j1_n23 group by key*key >= 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest_j1_n23
+#### A masked pattern was here ####
+cnt, check desc	1028
 PREHOOK: query: desc formatted dest_j1_n23
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@dest_j1_n23
@@ -229,8 +238,8 @@ Table Parameters:
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
 	bucketing_version   	2                   
 	numFiles            	137                 
-	numRows             	855                 
-	rawDataSize         	9143                
+	numRows             	1028                
+	rawDataSize         	10968               
 	totalSize           	11996               
 #### A masked pattern was here ####
 	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/results/clientpositive/llap/multiMapJoin1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/multiMapJoin1.q.out b/ql/src/test/results/clientpositive/llap/multiMapJoin1.q.out
index ed1745f..261e153 100644
--- a/ql/src/test/results/clientpositive/llap/multiMapJoin1.q.out
+++ b/ql/src/test/results/clientpositive/llap/multiMapJoin1.q.out
@@ -193,14 +193,14 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key is not null and value is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 175168 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 1748368 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 450 Data size: 157651 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4500 Data size: 1573531 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 450 Data size: 157651 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4500 Data size: 1573531 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -210,11 +210,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 495 Data size: 173416 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4950 Data size: 1730884 Basic stats: COMPLETE Column stats: NONE
                         Select Operator
                           expressions: _col1 (type: string)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 495 Data size: 173416 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 4950 Data size: 1730884 Basic stats: COMPLETE Column stats: NONE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -223,7 +223,7 @@ STAGE PLANS:
                               1 _col0 (type: string)
                             input vertices:
                               1 Map 4
-                            Statistics: Num rows: 544 Data size: 190757 Basic stats: COMPLETE Column stats: NONE
+                            Statistics: Num rows: 5445 Data size: 1903972 Basic stats: COMPLETE Column stats: NONE
                             Group By Operator
                               aggregations: count()
                               mode: hash
@@ -359,14 +359,14 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key is not null and value is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 175168 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 1748368 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 450 Data size: 157651 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4500 Data size: 1573531 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 450 Data size: 157651 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4500 Data size: 1573531 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -376,11 +376,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 495 Data size: 173416 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4950 Data size: 1730884 Basic stats: COMPLETE Column stats: NONE
                         Select Operator
                           expressions: _col1 (type: string)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 495 Data size: 173416 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 4950 Data size: 1730884 Basic stats: COMPLETE Column stats: NONE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -389,7 +389,7 @@ STAGE PLANS:
                               1 _col0 (type: string)
                             input vertices:
                               1 Map 4
-                            Statistics: Num rows: 544 Data size: 190757 Basic stats: COMPLETE Column stats: NONE
+                            Statistics: Num rows: 5445 Data size: 1903972 Basic stats: COMPLETE Column stats: NONE
                             Group By Operator
                               aggregations: count()
                               mode: hash
@@ -527,14 +527,14 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key is not null and value is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 175168 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 1748368 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 450 Data size: 157651 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4500 Data size: 1573531 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key (type: string), value (type: string)
                       outputColumnNames: _col0, _col1
-                      Statistics: Num rows: 450 Data size: 157651 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4500 Data size: 1573531 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -544,11 +544,11 @@ STAGE PLANS:
                         outputColumnNames: _col1
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 495 Data size: 173416 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4950 Data size: 1730884 Basic stats: COMPLETE Column stats: NONE
                         Select Operator
                           expressions: _col1 (type: string)
                           outputColumnNames: _col0
-                          Statistics: Num rows: 495 Data size: 173416 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 4950 Data size: 1730884 Basic stats: COMPLETE Column stats: NONE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -558,18 +558,18 @@ STAGE PLANS:
                             outputColumnNames: _col1
                             input vertices:
                               1 Map 4
-                            Statistics: Num rows: 544 Data size: 190757 Basic stats: COMPLETE Column stats: NONE
+                            Statistics: Num rows: 5445 Data size: 1903972 Basic stats: COMPLETE Column stats: NONE
                             Group By Operator
                               aggregations: count()
                               keys: _col1 (type: string)
                               mode: hash
                               outputColumnNames: _col0, _col1
-                              Statistics: Num rows: 544 Data size: 190757 Basic stats: COMPLETE Column stats: NONE
+                              Statistics: Num rows: 5445 Data size: 1903972 Basic stats: COMPLETE Column stats: NONE
                               Reduce Output Operator
                                 key expressions: _col0 (type: string)
                                 sort order: +
                                 Map-reduce partition columns: _col0 (type: string)
-                                Statistics: Num rows: 544 Data size: 190757 Basic stats: COMPLETE Column stats: NONE
+                                Statistics: Num rows: 5445 Data size: 1903972 Basic stats: COMPLETE Column stats: NONE
                                 value expressions: _col1 (type: bigint)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -622,14 +622,14 @@ STAGE PLANS:
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 272 Data size: 95378 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 2722 Data size: 951811 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col1 (type: bigint)
                   outputColumnNames: _col0
-                  Statistics: Num rows: 272 Data size: 95378 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 2722 Data size: 951811 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    Statistics: Num rows: 272 Data size: 95378 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 2722 Data size: 951811 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -835,14 +835,14 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key1 is not null and value is not null and key2 is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 262752 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 2622552 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key1 is not null and key2 is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key1 (type: string), key2 (type: string), value (type: string)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -852,11 +852,11 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2, _col3
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                         Select Operator
                           expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1, _col2, _col3
-                          Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -866,11 +866,11 @@ STAGE PLANS:
                             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                             input vertices:
                               1 Map 4
-                            Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                            Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                             Select Operator
                               expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col5 (type: string), _col3 (type: string), _col4 (type: string)
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                              Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                              Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                               Map Join Operator
                                 condition map:
                                      Inner Join 0 to 1
@@ -880,11 +880,11 @@ STAGE PLANS:
                                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                                 input vertices:
                                   1 Map 5
-                                Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                                Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                                 Select Operator
                                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col4 (type: string), _col5 (type: string)
                                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                                  Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                                  Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                                   Map Join Operator
                                     condition map:
                                          Inner Join 0 to 1
@@ -894,11 +894,11 @@ STAGE PLANS:
                                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                                     input vertices:
                                       1 Map 6
-                                    Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                                    Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                                     Select Operator
                                       expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int), hash(_col3) (type: int), hash(_col4) (type: int), hash(_col7) (type: int), hash(_col5) (type: int), hash(_col6) (type: int)
                                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                                      Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                                      Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                                       Group By Operator
                                         aggregations: sum(_col0), sum(_col1), sum(_col2), sum(_col3), sum(_col4), sum(_col5), sum(_col6), sum(_col7)
                                         mode: hash
@@ -1175,14 +1175,14 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key1 is not null and value is not null and key2 is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 262752 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 2622552 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key1 is not null and key2 is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key1 (type: string), key2 (type: string), value (type: string)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                       Map Join Operator
                         condition map:
                              Inner Join 0 to 1
@@ -1192,11 +1192,11 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2, _col3
                         input vertices:
                           1 Map 3
-                        Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                         Select Operator
                           expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col2 (type: string)
                           outputColumnNames: _col0, _col1, _col2, _col3
-                          Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                          Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                           Map Join Operator
                             condition map:
                                  Inner Join 0 to 1
@@ -1206,11 +1206,11 @@ STAGE PLANS:
                             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                             input vertices:
                               1 Map 4
-                            Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                            Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                             Select Operator
                               expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col5 (type: string), _col3 (type: string), _col4 (type: string)
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                              Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                              Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                               Map Join Operator
                                 condition map:
                                      Inner Join 0 to 1
@@ -1220,11 +1220,11 @@ STAGE PLANS:
                                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                                 input vertices:
                                   1 Map 5
-                                Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                                Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                                 Select Operator
                                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col4 (type: string), _col5 (type: string)
                                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                                  Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                                  Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                                   Map Join Operator
                                     condition map:
                                          Inner Join 0 to 1
@@ -1234,11 +1234,11 @@ STAGE PLANS:
                                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                                     input vertices:
                                       1 Map 6
-                                    Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                                    Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                                     Select Operator
                                       expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int), hash(_col3) (type: int), hash(_col4) (type: int), hash(_col7) (type: int), hash(_col5) (type: int), hash(_col6) (type: int)
                                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                                      Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                                      Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                                       Group By Operator
                                         aggregations: sum(_col0), sum(_col1), sum(_col2), sum(_col3), sum(_col4), sum(_col5), sum(_col6), sum(_col7)
                                         mode: hash
@@ -1518,19 +1518,19 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key1 is not null and value is not null and key2 is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 262752 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 2622552 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key1 is not null and key2 is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key1 (type: string), key2 (type: string), value (type: string)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: string), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -1625,16 +1625,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col3 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col3 (type: string)
-                    Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -1646,16 +1646,16 @@ STAGE PLANS:
                   0 _col3 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col5 (type: string), _col3 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
-                    Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -1667,16 +1667,16 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col4 (type: string), _col5 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
-                    Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string)
         Reducer 5 
             Execution mode: llap
@@ -1688,11 +1688,11 @@ STAGE PLANS:
                   0 _col2 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int), hash(_col3) (type: int), hash(_col4) (type: int), hash(_col7) (type: int), hash(_col5) (type: int), hash(_col6) (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                  Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2), sum(_col3), sum(_col4), sum(_col5), sum(_col6), sum(_col7)
                     mode: hash
@@ -1889,19 +1889,19 @@ STAGE PLANS:
                 TableScan
                   alias: bigtbl
                   filterExpr: (key1 is not null and value is not null and key2 is not null) (type: boolean)
-                  Statistics: Num rows: 500 Data size: 262752 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5000 Data size: 2622552 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (key1 is not null and key2 is not null and value is not null) (type: boolean)
-                    Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                     Select Operator
                       expressions: key1 (type: string), key2 (type: string), value (type: string)
                       outputColumnNames: _col0, _col1, _col2
-                      Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                      Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
-                        Statistics: Num rows: 425 Data size: 223339 Basic stats: COMPLETE Column stats: NONE
+                        Statistics: Num rows: 4250 Data size: 2229169 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: string), _col2 (type: string)
             Execution mode: vectorized, llap
             LLAP IO: no inputs
@@ -1996,16 +1996,16 @@ STAGE PLANS:
                   0 _col0 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
-                Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col2 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
-                  Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col3 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col3 (type: string)
-                    Statistics: Num rows: 467 Data size: 245672 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 4675 Data size: 2452085 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
             Execution mode: llap
@@ -2017,16 +2017,16 @@ STAGE PLANS:
                   0 _col3 (type: string)
                   1 _col1 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col5 (type: string), _col3 (type: string), _col4 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                  Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
-                    Statistics: Num rows: 513 Data size: 270239 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 5142 Data size: 2697293 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
         Reducer 4 
             Execution mode: llap
@@ -2038,16 +2038,16 @@ STAGE PLANS:
                   0 _col1 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col6 (type: string), _col4 (type: string), _col5 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
-                  Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
-                    Statistics: Num rows: 564 Data size: 297262 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 5656 Data size: 2967022 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string)
         Reducer 5 
             Execution mode: llap
@@ -2059,11 +2059,11 @@ STAGE PLANS:
                   0 _col2 (type: string)
                   1 _col0 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: hash(_col0) (type: int), hash(_col1) (type: int), hash(_col2) (type: int), hash(_col3) (type: int), hash(_col4) (type: int), hash(_col7) (type: int), hash(_col5) (type: int), hash(_col6) (type: int)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
-                  Statistics: Num rows: 620 Data size: 326988 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 6221 Data size: 3263724 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: sum(_col0), sum(_col1), sum(_col2), sum(_col3), sum(_col4), sum(_col5), sum(_col6), sum(_col7)
                     mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out b/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
index cdb9193..40f469b 100644
--- a/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/union_fast_stats.q.out
@@ -175,14 +175,12 @@ Retention:          	0
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
+	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	bucketing_version   	2                   
 	numFiles            	3                   
-<<<<<<< HEAD
-=======
-	numRows             	5                   
-	rawDataSize         	1300                
->>>>>>> asf/master
-	totalSize           	4033                
+	numRows             	15                  
+	rawDataSize         	3315                
+	totalSize           	4152                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -235,7 +233,7 @@ Table Parameters:
 	numFiles            	3                   
 	numRows             	15                  
 	rawDataSize         	3483                
-	totalSize           	4033                
+	totalSize           	4152                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -299,8 +297,8 @@ Table Parameters:
 	bucketing_version   	2                   
 	numFiles            	4                   
 	numRows             	20                  
-	rawDataSize         	4552                
-	totalSize           	5406                
+	rawDataSize         	4468                
+	totalSize           	5569                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -513,13 +511,9 @@ Table Parameters:
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	bucketing_version   	2                   
 	numFiles            	1                   
-	numRows             	5                   
-	rawDataSize         	1069                
-<<<<<<< HEAD
-	totalSize           	3243                
-=======
-	totalSize           	3247                
->>>>>>> asf/master
+	numRows             	15                  
+	rawDataSize         	3315                
+	totalSize           	3318                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -532,6 +526,15 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
+PREHOOK: query: select 15,count(*) from small_alltypesorc_a_n2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@small_alltypesorc_a_n2
+#### A masked pattern was here ####
+POSTHOOK: query: select 15,count(*) from small_alltypesorc_a_n2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@small_alltypesorc_a_n2
+#### A masked pattern was here ####
+15	15
 PREHOOK: query: ANALYZE TABLE small_alltypesorc_a_n2 COMPUTE STATISTICS
 PREHOOK: type: QUERY
 PREHOOK: Input: default@small_alltypesorc_a_n2
@@ -540,6 +543,15 @@ POSTHOOK: query: ANALYZE TABLE small_alltypesorc_a_n2 COMPUTE STATISTICS
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@small_alltypesorc_a_n2
 POSTHOOK: Output: default@small_alltypesorc_a_n2
+PREHOOK: query: select 15,count(*) from small_alltypesorc_a_n2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@small_alltypesorc_a_n2
+#### A masked pattern was here ####
+POSTHOOK: query: select 15,count(*) from small_alltypesorc_a_n2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@small_alltypesorc_a_n2
+#### A masked pattern was here ####
+15	15
 PREHOOK: query: desc formatted small_alltypesorc_a_n2
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@small_alltypesorc_a_n2
@@ -572,11 +584,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	15                  
 	rawDataSize         	3320                
-<<<<<<< HEAD
-	totalSize           	3243                
-=======
-	totalSize           	3247                
->>>>>>> asf/master
+	totalSize           	3318                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -640,12 +648,8 @@ Table Parameters:
 	bucketing_version   	2                   
 	numFiles            	2                   
 	numRows             	20                  
-	rawDataSize         	4389                
-<<<<<<< HEAD
-	totalSize           	4616                
-=======
-	totalSize           	4620                
->>>>>>> asf/master
+	rawDataSize         	4305                
+	totalSize           	4735                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/results/clientpositive/llap/union_rowcounts.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/union_rowcounts.q.out b/ql/src/test/results/clientpositive/llap/union_rowcounts.q.out
new file mode 100644
index 0000000..8b06d6e
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/union_rowcounts.q.out
@@ -0,0 +1,180 @@
+PREHOOK: query: create table t1 (a int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1 (a int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: insert into t1 values (1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t1
+POSTHOOK: query: insert into t1 values (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.a SCRIPT []
+PREHOOK: query: create table t2a  as 
+        select * from t1
+        union all
+        select * from t1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2a
+POSTHOOK: query: create table t2a  as 
+        select * from t1
+        union all
+        select * from t1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2a
+POSTHOOK: Lineage: t2a.a EXPRESSION [(t1)t1.FieldSchema(name:a, type:int, comment:null), ]
+PREHOOK: query: select 2,count(*) from t2a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2a
+#### A masked pattern was here ####
+POSTHOOK: query: select 2,count(*) from t2a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2a
+#### A masked pattern was here ####
+2	2
+PREHOOK: query: create table t2b  as select * from
+(
+        select * from (select * from t1) sq1
+        union all
+        select * from (select * from t1) sq2
+) tt
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2b
+POSTHOOK: query: create table t2b  as select * from
+(
+        select * from (select * from t1) sq1
+        union all
+        select * from (select * from t1) sq2
+) tt
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2b
+POSTHOOK: Lineage: t2b.a EXPRESSION [(t1)t1.FieldSchema(name:a, type:int, comment:null), ]
+PREHOOK: query: select 2,count(*) from t2b
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2b
+#### A masked pattern was here ####
+POSTHOOK: query: select 2,count(*) from t2b
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2b
+#### A masked pattern was here ####
+2	2
+PREHOOK: query: drop table if exists t1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+POSTHOOK: query: drop table if exists t1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: default@t1
+PREHOOK: query: drop table if exists t2a
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t2a
+PREHOOK: Output: default@t2a
+POSTHOOK: query: drop table if exists t2a
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t2a
+POSTHOOK: Output: default@t2a
+PREHOOK: query: drop table if exists t2b
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t2b
+PREHOOK: Output: default@t2b
+POSTHOOK: query: drop table if exists t2b
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t2b
+POSTHOOK: Output: default@t2b
+PREHOOK: query: create table t1 (a int) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1 (a int) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: insert into t1 values (1)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@t1
+POSTHOOK: query: insert into t1 values (1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.a SCRIPT []
+PREHOOK: query: analyze table t1 compute statistics for columns
+PREHOOK: type: ANALYZE_TABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table t1 compute statistics for columns
+POSTHOOK: type: ANALYZE_TABLE
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: default@t1
+#### A masked pattern was here ####
+PREHOOK: query: create table t2a stored as orc as
+	select * from t1
+	union all
+	select * from t1
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2a
+POSTHOOK: query: create table t2a stored as orc as
+	select * from t1
+	union all
+	select * from t1
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2a
+POSTHOOK: Lineage: t2a.a EXPRESSION [(t1)t1.FieldSchema(name:a, type:int, comment:null), ]
+PREHOOK: query: select 2,count(*) from t2a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2a
+#### A masked pattern was here ####
+POSTHOOK: query: select 2,count(*) from t2a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2a
+#### A masked pattern was here ####
+2	2
+PREHOOK: query: create table t2b stored as orc as select * from
+(
+	select * from (select * from t1) sq1
+	union all
+	select * from (select * from t1) sq2
+) tt
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2b
+POSTHOOK: query: create table t2b stored as orc as select * from
+(
+	select * from (select * from t1) sq1
+	union all
+	select * from (select * from t1) sq2
+) tt
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2b
+POSTHOOK: Lineage: t2b.a EXPRESSION [(t1)t1.FieldSchema(name:a, type:int, comment:null), ]
+PREHOOK: query: select 2,count(*) from t2b
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2b
+#### A masked pattern was here ####
+POSTHOOK: query: select 2,count(*) from t2b
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2b
+#### A masked pattern was here ####
+2	2

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/results/clientpositive/llap/union_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/union_stats.q.out b/ql/src/test/results/clientpositive/llap/union_stats.q.out
index 5a088f4..cea4847 100644
--- a/ql/src/test/results/clientpositive/llap/union_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/union_stats.q.out
@@ -236,7 +236,7 @@ POSTHOOK: query: select count(1) from t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@t
 #### A masked pattern was here ####
-500
+1000
 PREHOOK: query: desc formatted t
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@t
@@ -257,8 +257,8 @@ Table Parameters:
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	bucketing_version   	2                   
 	numFiles            	2                   
-	numRows             	500                 
-	rawDataSize         	5312                
+	numRows             	1000                
+	rawDataSize         	10624               
 	totalSize           	11624               
 #### A masked pattern was here ####
 	 	 
@@ -306,8 +306,8 @@ Table Parameters:
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	bucketing_version   	2                   
 	numFiles            	2                   
-	numRows             	1000                
-	rawDataSize         	10624               
+	numRows             	1500                
+	rawDataSize         	15936               
 	totalSize           	17436               
 #### A masked pattern was here ####
 	 	 
@@ -363,8 +363,8 @@ Table Parameters:
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	bucketing_version   	2                   
 	numFiles            	2                   
-	numRows             	1000                
-	rawDataSize         	10624               
+	numRows             	1500                
+	rawDataSize         	15936               
 	totalSize           	17436               
 #### A masked pattern was here ####
 	 	 
@@ -394,24 +394,155 @@ POSTHOOK: query: create table t2 like src
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t2
+PREHOOK: query: create table t3 like src
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t3
+POSTHOOK: query: create table t3 like src
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t3
+PREHOOK: query: explain from (select * from src union all select * from src)s
+insert overwrite table t1 select *
+insert overwrite table t2 select *
+insert overwrite table t3 select *
+PREHOOK: type: QUERY
+POSTHOOK: query: explain from (select * from src union all select * from src)s
+insert overwrite table t1 select *
+insert overwrite table t2 select *
+insert overwrite table t3 select *
+POSTHOOK: type: QUERY
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Union 2 (CONTAINS)
+Map 6 <- Union 2 (CONTAINS)
+Reducer 3 <- Union 2 (CUSTOM_SIMPLE_EDGE)
+Reducer 4 <- Union 2 (CUSTOM_SIMPLE_EDGE)
+Reducer 5 <- Union 2 (CUSTOM_SIMPLE_EDGE)
+
+Stage-5
+  Stats Work{}
+    Stage-0
+      Move Operator
+        table:{"name:":"default.t1"}
+        Stage-4
+          Dependency Collection{}
+            Stage-3
+              Reducer 3 llap
+              File Output Operator [FS_6]
+                Group By Operator [GBY_4] (rows=1 width=880)
+                  Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0)","compute_stats(VALUE._col1)"]
+                <-Union 2 [CUSTOM_SIMPLE_EDGE]
+                  <-Map 1 [CONTAINS] llap
+                    File Output Operator [FS_7]
+                      table:{"name:":"default.t1"}
+                      Select Operator [SEL_1] (rows=500 width=178)
+                        Output:["_col0","_col1"]
+                        TableScan [TS_0] (rows=500 width=178)
+                          Output:["key","value"]
+                    Reduce Output Operator [RS_3]
+                      Group By Operator [GBY_2] (rows=1 width=880)
+                        Output:["_col0","_col1"],aggregations:["compute_stats(key, 'hll')","compute_stats(value, 'hll')"]
+                        Select Operator [SEL_1] (rows=1000 width=178)
+                          Output:["key","value"]
+                           Please refer to the previous Select Operator [SEL_1]
+                    File Output Operator [FS_9]
+                      table:{"name:":"default.t2"}
+                       Please refer to the previous Select Operator [SEL_1]
+                    Reduce Output Operator [RS_3]
+                      Group By Operator [GBY_2] (rows=1 width=880)
+                        Output:["_col0","_col1"],aggregations:["compute_stats(key, 'hll')","compute_stats(value, 'hll')"]
+                        Select Operator [SEL_1] (rows=1000 width=178)
+                          Output:["key","value"]
+                           Please refer to the previous Select Operator [SEL_1]
+                    File Output Operator [FS_11]
+                      table:{"name:":"default.t3"}
+                       Please refer to the previous Select Operator [SEL_1]
+                    Reduce Output Operator [RS_3]
+                      Group By Operator [GBY_2] (rows=1 width=880)
+                        Output:["_col0","_col1"],aggregations:["compute_stats(key, 'hll')","compute_stats(value, 'hll')"]
+                        Select Operator [SEL_1] (rows=1000 width=178)
+                          Output:["key","value"]
+                           Please refer to the previous Select Operator [SEL_1]
+                  <-Map 6 [CONTAINS] llap
+                    File Output Operator [FS_7]
+                      table:{"name:":"default.t1"}
+                      Select Operator [SEL_3] (rows=500 width=178)
+                        Output:["_col0","_col1"]
+                        TableScan [TS_2] (rows=500 width=178)
+                          Output:["key","value"]
+                    Reduce Output Operator [RS_3]
+                      Group By Operator [GBY_2] (rows=1 width=880)
+                        Output:["_col0","_col1"],aggregations:["compute_stats(key, 'hll')","compute_stats(value, 'hll')"]
+                        Select Operator [SEL_1] (rows=1000 width=178)
+                          Output:["key","value"]
+                           Please refer to the previous Select Operator [SEL_3]
+                    File Output Operator [FS_9]
+                      table:{"name:":"default.t2"}
+                       Please refer to the previous Select Operator [SEL_3]
+                    Reduce Output Operator [RS_3]
+                      Group By Operator [GBY_2] (rows=1 width=880)
+                        Output:["_col0","_col1"],aggregations:["compute_stats(key, 'hll')","compute_stats(value, 'hll')"]
+                        Select Operator [SEL_1] (rows=1000 width=178)
+                          Output:["key","value"]
+                           Please refer to the previous Select Operator [SEL_3]
+                    File Output Operator [FS_11]
+                      table:{"name:":"default.t3"}
+                       Please refer to the previous Select Operator [SEL_3]
+                    Reduce Output Operator [RS_3]
+                      Group By Operator [GBY_2] (rows=1 width=880)
+                        Output:["_col0","_col1"],aggregations:["compute_stats(key, 'hll')","compute_stats(value, 'hll')"]
+                        Select Operator [SEL_1] (rows=1000 width=178)
+                          Output:["key","value"]
+                           Please refer to the previous Select Operator [SEL_3]
+              Reducer 4 llap
+              File Output Operator [FS_6]
+                Group By Operator [GBY_4] (rows=1 width=880)
+                  Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0)","compute_stats(VALUE._col1)"]
+                <- Please refer to the previous Union 2 [CUSTOM_SIMPLE_EDGE]
+              Reducer 5 llap
+              File Output Operator [FS_6]
+                Group By Operator [GBY_4] (rows=1 width=880)
+                  Output:["_col0","_col1"],aggregations:["compute_stats(VALUE._col0)","compute_stats(VALUE._col1)"]
+                <- Please refer to the previous Union 2 [CUSTOM_SIMPLE_EDGE]
+Stage-6
+  Stats Work{}
+    Stage-1
+      Move Operator
+        table:{"name:":"default.t2"}
+         Please refer to the previous Stage-4
+Stage-7
+  Stats Work{}
+    Stage-2
+      Move Operator
+        table:{"name:":"default.t3"}
+         Please refer to the previous Stage-4
+
 PREHOOK: query: from (select * from src union all select * from src)s
 insert overwrite table t1 select *
 insert overwrite table t2 select *
+insert overwrite table t3 select *
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@t1
 PREHOOK: Output: default@t2
+PREHOOK: Output: default@t3
 POSTHOOK: query: from (select * from src union all select * from src)s
 insert overwrite table t1 select *
 insert overwrite table t2 select *
+insert overwrite table t3 select *
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@t1
 POSTHOOK: Output: default@t2
+POSTHOOK: Output: default@t3
 POSTHOOK: Lineage: t1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: t1.value EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 POSTHOOK: Lineage: t2.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: t2.value EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: t3.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: t3.value EXPRESSION [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 PREHOOK: query: desc formatted t1
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@t1


[18/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
index 2d87a2f..53c4d24 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
@@ -2231,10 +2231,8 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
   }
 
   @Override
-  public void replCommitTxn(long srcTxnId, String replPolicy)
+  public void replCommitTxn(CommitTxnRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, TException {
-    CommitTxnRequest rqst = new CommitTxnRequest(srcTxnId);
-    rqst.setReplPolicy(replPolicy);
     client.commit_txn(rqst);
   }
 
@@ -2475,6 +2473,12 @@ public class HiveMetaStoreClientPreCatalog implements IMetaStoreClient, AutoClos
     return client.fire_listener_event(rqst);
   }
 
+  @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
+  @Override
+  public void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException {
+    client.add_write_notification_log(rqst);
+  }
+
   /**
    * Creates a synchronized wrapper for any {@link IMetaStoreClient}.
    * This may be used by multi-threaded applications until we have


[16/46] hive git commit: HIVE-17840: HiveMetaStore eats exception if transactionalListeners.notifyEvent fail (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Daniel Dai)

Posted by se...@apache.org.
HIVE-17840: HiveMetaStore eats exception if transactionalListeners.notifyEvent fail (Sankar Hariappan, reviewed by Mahesh Kumar Behera, Daniel Dai)


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

Branch: refs/heads/master-txnstats
Commit: f01269123652d977e93dba656a847f800eeb2c61
Parents: 1fc1221
Author: Sankar Hariappan <sa...@apache.org>
Authored: Tue Jul 3 10:08:57 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Tue Jul 3 10:08:57 2018 +0530

----------------------------------------------------------------------
 .../hive/hcatalog/listener/DummyRawStoreFailEvent.java       | 2 +-
 .../java/org/apache/hadoop/hive/metastore/ObjectStore.java   | 7 ++++---
 .../main/java/org/apache/hadoop/hive/metastore/RawStore.java | 3 ++-
 .../org/apache/hadoop/hive/metastore/cache/CachedStore.java  | 2 +-
 .../hadoop/hive/metastore/DummyRawStoreControlledCommit.java | 2 +-
 .../hadoop/hive/metastore/DummyRawStoreForJdoConnection.java | 2 +-
 .../org/apache/hadoop/hive/metastore/TestObjectStore.java    | 8 ++++----
 7 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index 3c334fa..abf67a8 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -866,7 +866,7 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
-  public void addNotificationEvent(NotificationEvent event) {
+  public void addNotificationEvent(NotificationEvent event) throws MetaException {
     objectStore.addNotificationEvent(event);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 0d2da7a..7490243 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -9697,7 +9697,7 @@ public class ObjectStore implements RawStore, Configurable {
   }
 
   @Override
-  public void addNotificationEvent(NotificationEvent entry) {
+  public void addNotificationEvent(NotificationEvent entry) throws MetaException {
     boolean commited = false;
     Query query = null;
     try {
@@ -9721,8 +9721,9 @@ public class ObjectStore implements RawStore, Configurable {
       }
       pm.makePersistent(translateThriftToDb(entry));
       commited = commitTransaction();
-    } catch (Exception e) {
-      LOG.error("couldnot get lock for update", e);
+    } catch (MetaException e) {
+      LOG.error("Couldn't get lock for update", e);
+      throw e;
     } finally {
       rollbackAndCleanup(commited, query);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index c8905c8..d019941 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -1195,8 +1195,9 @@ public interface RawStore extends Configurable {
   /**
    * Add a notification entry.  This should only be called from inside the metastore
    * @param event the notification to add
+   * @throws MetaException error accessing RDBMS
    */
-  void addNotificationEvent(NotificationEvent event);
+  void addNotificationEvent(NotificationEvent event) throws MetaException;
 
   /**
    * Remove older notification events.

http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index 1da9798..a526019 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -2026,7 +2026,7 @@ public class CachedStore implements RawStore, Configurable {
   }
 
   @Override
-  public void addNotificationEvent(NotificationEvent event) {
+  public void addNotificationEvent(NotificationEvent event) throws MetaException {
     rawStore.addNotificationEvent(event);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 8e195d0..c482d28 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -818,7 +818,7 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
-  public void addNotificationEvent(NotificationEvent event) {
+  public void addNotificationEvent(NotificationEvent event) throws MetaException {
     objectStore.addNotificationEvent(event);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index 85eb6d5..d253005 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -823,7 +823,7 @@ public class DummyRawStoreForJdoConnection implements RawStore {
   }
 
   @Override
-  public void addNotificationEvent(NotificationEvent event) {
+  public void addNotificationEvent(NotificationEvent event) throws MetaException {
 
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f0126912/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
index 5d5bc76..833e2bd 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/TestObjectStore.java
@@ -765,7 +765,7 @@ public class TestObjectStore {
    */
   // TODO MS-SPLIT uncomment once we move EventMessage over
   @Test
-  public void testNotificationOps() throws InterruptedException {
+  public void testNotificationOps() throws InterruptedException, MetaException {
     final int NO_EVENT_ID = 0;
     final int FIRST_EVENT_ID = 1;
     final int SECOND_EVENT_ID = 2;
@@ -817,7 +817,7 @@ public class TestObjectStore {
           + " https://db.apache.org/derby/docs/10.10/devguide/cdevconcepts842385.html"
   )
   @Test
-  public void testConcurrentAddNotifications() throws ExecutionException, InterruptedException {
+  public void testConcurrentAddNotifications() throws ExecutionException, InterruptedException, MetaException {
 
     final int NUM_THREADS = 10;
     CyclicBarrier cyclicBarrier = new CyclicBarrier(NUM_THREADS,
@@ -866,10 +866,10 @@ public class TestObjectStore {
 
             try {
               cyclicBarrier.await();
-            } catch (InterruptedException | BrokenBarrierException e) {
+              store.addNotificationEvent(dbEvent);
+            } catch (InterruptedException | BrokenBarrierException | MetaException e) {
               throw new RuntimeException(e);
             }
-            store.addNotificationEvent(dbEvent);
             System.out.println("FINISH NOTIFICATION");
           });
     }


[12/46] hive git commit: HIVE-19711 Refactor Hive Schema Tool (Miklos Gergely via Alan Gates).

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java
new file mode 100644
index 0000000..8ca8343
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolCommandLine.java
@@ -0,0 +1,286 @@
+/*
+ * 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.beeline.schematool;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
+
+import com.google.common.collect.ImmutableSet;
+
+import java.util.Set;
+
+class HiveSchemaToolCommandLine {
+  private final Options cmdLineOptions = createOptions();
+
+  @SuppressWarnings("static-access")
+  private Options createOptions() {
+    Option help = new Option("help", "print this message");
+    Option infoOpt = new Option("info", "Show config and schema details");
+    Option upgradeOpt = new Option("upgradeSchema", "Schema upgrade");
+    Option upgradeFromOpt = OptionBuilder.withArgName("upgradeFrom").hasArg()
+        .withDescription("Schema upgrade from a version")
+        .create("upgradeSchemaFrom");
+    Option initOpt = new Option("initSchema", "Schema initialization");
+    Option initToOpt = OptionBuilder.withArgName("initTo").hasArg()
+        .withDescription("Schema initialization to a version")
+        .create("initSchemaTo");
+    Option validateOpt = new Option("validate", "Validate the database");
+    Option createCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Create a catalog, requires --catalogLocation parameter as well")
+        .create("createCatalog");
+    Option alterCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Alter a catalog, requires --catalogLocation and/or --catalogDescription parameter as well")
+        .create("alterCatalog");
+    Option moveDatabase = OptionBuilder
+        .hasArg()
+        .withDescription("Move a database between catalogs.  Argument is the database name. " +
+            "Requires --fromCatalog and --toCatalog parameters as well")
+        .create("moveDatabase");
+    Option moveTable = OptionBuilder
+        .hasArg()
+        .withDescription("Move a table to a different database.  Argument is the table name. " +
+            "Requires --fromCatalog, --toCatalog, --fromDatabase, and --toDatabase " +
+            " parameters as well.")
+        .create("moveTable");
+
+    OptionGroup optGroup = new OptionGroup();
+    optGroup
+      .addOption(help)
+      .addOption(infoOpt)
+      .addOption(upgradeOpt)
+      .addOption(upgradeFromOpt)
+      .addOption(initOpt)
+      .addOption(initToOpt)
+      .addOption(validateOpt)
+      .addOption(createCatalog)
+      .addOption(alterCatalog)
+      .addOption(moveDatabase)
+      .addOption(moveTable);
+    optGroup.setRequired(true);
+
+    Option userNameOpt = OptionBuilder.withArgName("user")
+        .hasArgs()
+        .withDescription("Override config file user name")
+        .create("userName");
+    Option passwdOpt = OptionBuilder.withArgName("password")
+        .hasArgs()
+        .withDescription("Override config file password")
+        .create("passWord");
+    Option dbTypeOpt = OptionBuilder.withArgName("databaseType")
+        .hasArgs().withDescription("Metastore database type").isRequired()
+        .create("dbType");
+    Option metaDbTypeOpt = OptionBuilder.withArgName("metaDatabaseType")
+        .hasArgs().withDescription("Used only if upgrading the system catalog for hive")
+        .create("metaDbType");
+    Option urlOpt = OptionBuilder.withArgName("url")
+        .hasArgs().withDescription("connection url to the database")
+        .create("url");
+    Option driverOpt = OptionBuilder.withArgName("driver")
+        .hasArgs().withDescription("driver name for connection")
+        .create("driver");
+    Option dbOpts = OptionBuilder.withArgName("databaseOpts")
+        .hasArgs().withDescription("Backend DB specific options")
+        .create("dbOpts");
+    Option dryRunOpt = new Option("dryRun", "list SQL scripts (no execute)");
+    Option verboseOpt = new Option("verbose", "only print SQL statements");
+    Option serversOpt = OptionBuilder.withArgName("serverList")
+        .hasArgs().withDescription("a comma-separated list of servers used in location validation in the format of " +
+            "scheme://authority (e.g. hdfs://localhost:8000)")
+        .create("servers");
+    Option catalogLocation = OptionBuilder
+        .hasArg()
+        .withDescription("Location of new catalog, required when adding a catalog")
+        .create("catalogLocation");
+    Option catalogDescription = OptionBuilder
+        .hasArg()
+        .withDescription("Description of new catalog")
+        .create("catalogDescription");
+    Option ifNotExists = OptionBuilder
+        .withDescription("If passed then it is not an error to create an existing catalog")
+        .create("ifNotExists");
+    Option fromCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Catalog a moving database or table is coming from.  This is " +
+            "required if you are moving a database or table.")
+        .create("fromCatalog");
+    Option toCatalog = OptionBuilder
+        .hasArg()
+        .withDescription("Catalog a moving database or table is going to.  This is " +
+            "required if you are moving a database or table.")
+        .create("toCatalog");
+    Option fromDatabase = OptionBuilder
+        .hasArg()
+        .withDescription("Database a moving table is coming from.  This is " +
+            "required if you are moving a table.")
+        .create("fromDatabase");
+    Option toDatabase = OptionBuilder
+        .hasArg()
+        .withDescription("Database a moving table is going to.  This is " +
+            "required if you are moving a table.")
+        .create("toDatabase");
+
+    Options options = new Options();
+    options.addOption(help);
+    options.addOptionGroup(optGroup);
+    options.addOption(dbTypeOpt);
+    options.addOption(metaDbTypeOpt);
+    options.addOption(userNameOpt);
+    options.addOption(passwdOpt);
+    options.addOption(urlOpt);
+    options.addOption(driverOpt);
+    options.addOption(dbOpts);
+    options.addOption(dryRunOpt);
+    options.addOption(verboseOpt);
+    options.addOption(serversOpt);
+    options.addOption(catalogLocation);
+    options.addOption(catalogDescription);
+    options.addOption(ifNotExists);
+    options.addOption(fromCatalog);
+    options.addOption(toCatalog);
+    options.addOption(fromDatabase);
+    options.addOption(toDatabase);
+
+    return options;
+  }
+
+  private final CommandLine cl;
+  private final String dbType;
+  private final String metaDbType;
+
+  HiveSchemaToolCommandLine(String[] args) throws ParseException {
+    cl = getCommandLine(args);
+    if (cl.hasOption("help")) {
+      printAndExit(null);
+    }
+
+    dbType = cl.getOptionValue("dbType");
+    metaDbType = cl.getOptionValue("metaDbType");
+
+    validate();
+  }
+
+  private CommandLine getCommandLine(String[] args)  throws ParseException {
+    try {
+      CommandLineParser parser = new GnuParser();
+      return parser.parse(cmdLineOptions, args);
+    } catch (ParseException e) {
+      printAndExit("HiveSchemaTool:Parsing failed. Reason: " + e.getLocalizedMessage());
+      return null;
+    }
+  }
+
+  private static final Set<String> VALID_DB_TYPES = ImmutableSet.of(HiveSchemaHelper.DB_DERBY,
+      HiveSchemaHelper.DB_HIVE, HiveSchemaHelper.DB_MSSQL, HiveSchemaHelper.DB_MYSQL,
+      HiveSchemaHelper.DB_POSTGRACE, HiveSchemaHelper.DB_ORACLE);
+
+  private static final Set<String> VALID_META_DB_TYPES = ImmutableSet.of(HiveSchemaHelper.DB_DERBY,
+      HiveSchemaHelper.DB_MSSQL, HiveSchemaHelper.DB_MYSQL, HiveSchemaHelper.DB_POSTGRACE,
+      HiveSchemaHelper.DB_ORACLE);
+
+  private void validate() throws ParseException {
+    if (!VALID_DB_TYPES.contains(dbType)) {
+      printAndExit("Unsupported dbType " + dbType);
+    }
+
+    if (metaDbType != null) {
+      if (!dbType.equals(HiveSchemaHelper.DB_HIVE)) {
+        printAndExit("metaDbType may only be set if dbType is hive");
+      }
+      if (!VALID_META_DB_TYPES.contains(metaDbType)) {
+        printAndExit("Unsupported metaDbType " + metaDbType);
+      }
+    } else if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
+      System.err.println();
+      printAndExit("metaDbType must be set if dbType is hive");
+    }
+
+    if ((cl.hasOption("createCatalog")) && !cl.hasOption("catalogLocation")) {
+      System.err.println();
+      printAndExit("catalogLocation must be set for createCatalog");
+    }
+
+    if (!cl.hasOption("createCatalog") && !cl.hasOption("alterCatalog") &&
+        (cl.hasOption("catalogLocation") || cl.hasOption("catalogDescription"))) {
+      printAndExit("catalogLocation and catalogDescription may be set only for createCatalog and alterCatalog");
+    }
+
+    if (!cl.hasOption("createCatalog") && cl.hasOption("ifNotExists")) {
+      printAndExit("ifNotExists may be set only for createCatalog");
+    }
+
+    if (cl.hasOption("moveDatabase") &&
+        (!cl.hasOption("fromCatalog") || !cl.hasOption("toCatalog"))) {
+      printAndExit("fromCatalog and toCatalog must be set for moveDatabase");
+    }
+
+    if (cl.hasOption("moveTable") &&
+        (!cl.hasOption("fromCatalog") || !cl.hasOption("toCatalog") ||
+         !cl.hasOption("fromDatabase") || !cl.hasOption("toDatabase"))) {
+      printAndExit("fromCatalog, toCatalog, fromDatabase and toDatabase must be set for moveTable");
+    }
+
+    if ((!cl.hasOption("moveDatabase") && !cl.hasOption("moveTable")) &&
+        (cl.hasOption("fromCatalog") || cl.hasOption("toCatalog"))) {
+      printAndExit("fromCatalog and toCatalog may be set only for moveDatabase and moveTable");
+    }
+
+    if (!cl.hasOption("moveTable") &&
+        (cl.hasOption("fromDatabase") || cl.hasOption("toDatabase"))) {
+      printAndExit("fromDatabase and toDatabase may be set only for moveTable");
+    }
+  }
+
+  private void printAndExit(String reason) throws ParseException {
+    if (reason != null) {
+      System.err.println(reason);
+    }
+    HelpFormatter formatter = new HelpFormatter();
+    formatter.printHelp("schemaTool", cmdLineOptions);
+    if (reason != null) {
+      throw new ParseException(reason);
+    } else {
+      System.exit(0);
+    }
+  }
+
+  String getDbType() {
+    return dbType;
+  }
+
+  String getMetaDbType() {
+    return metaDbType;
+  }
+
+  boolean hasOption(String opt) {
+    return cl.hasOption(opt);
+  }
+
+  String getOptionValue(String opt) {
+    return cl.getOptionValue(opt);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java
new file mode 100644
index 0000000..3172c2f
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTask.java
@@ -0,0 +1,32 @@
+/*
+ * 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.beeline.schematool;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+
+abstract class HiveSchemaToolTask {
+  protected HiveSchemaTool schemaTool;
+
+  void setHiveSchemaTool(HiveSchemaTool schemaTool) {
+    this.schemaTool = schemaTool;
+  }
+
+  abstract void setCommandLineArguments(HiveSchemaToolCommandLine cl);
+
+  abstract void execute() throws HiveMetaException;
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.java
new file mode 100644
index 0000000..802fe3a
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskAlterCatalog.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.hive.beeline.schematool;
+
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Print Hive version and schema version.
+ */
+class HiveSchemaToolTaskAlterCatalog extends HiveSchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskAlterCatalog.class.getName());
+
+  private String catName;
+  private String location;
+  private String description;
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    catName = normalizeIdentifier(cl.getOptionValue("alterCatalog"));
+    location = cl.getOptionValue("catalogLocation");
+    description = cl.getOptionValue("catalogDescription");
+  }
+
+  private static final String UPDATE_CATALOG_STMT =
+      "update <q>CTLGS<q> " +
+      "   set <q>LOCATION_URI<q> = %s, " +
+      "       <qa>DESC<qa> = %s " +
+      " where <q>NAME<q> = '%s'";
+
+  @Override
+  void execute() throws HiveMetaException {
+    if (location == null && description == null) {
+      throw new HiveMetaException("Asked to update catalog " + catName + " but not given any changes to update");
+    }
+    System.out.println("Updating catalog " + catName);
+
+    Connection conn = schemaTool.getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        Object updateLocation = location == null ? schemaTool.quote("<q>LOCATION_URI<q>") : "'" + location + "'";
+        Object updateDescription = description == null ? schemaTool.quote("<qa>DESC<qa>") : "'" + description + "'";
+        String update = String.format(schemaTool.quote(UPDATE_CATALOG_STMT), updateLocation, updateDescription,
+            catName);
+        LOG.debug("Going to run " + update);
+        int count = stmt.executeUpdate(update);
+        if (count != 1) {
+          throw new HiveMetaException("Failed to find catalog " + catName + " to update");
+        }
+        conn.commit();
+        success = true;
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to update catalog", e);
+    } finally {
+      try {
+        if (!success) {
+          conn.rollback();
+        }
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.", e);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java
new file mode 100644
index 0000000..810dfea
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskCreateCatalog.java
@@ -0,0 +1,132 @@
+/*
+ * 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.beeline.schematool;
+
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Print Hive version and schema version.
+ */
+class HiveSchemaToolTaskCreateCatalog extends HiveSchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskCreateCatalog.class.getName());
+
+  private String catName;
+  private String location;
+  private String description;
+  private boolean ifNotExists;
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    catName = normalizeIdentifier(cl.getOptionValue("createCatalog"));
+    location = cl.getOptionValue("catalogLocation");
+    description = cl.getOptionValue("catalogDescription");
+    ifNotExists = cl.hasOption("ifNotExists");
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    System.out.println("Create catalog " + catName + " at location " + location);
+
+    Connection conn = schemaTool.getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        // If they set ifNotExists check for existence first, and bail if it exists.  This is
+        // more reliable then attempting to parse the error message from the SQLException.
+        if (ifNotExists && catalogExists(stmt)) {
+          return;
+        }
+
+        int catNum = getNextCatalogId(stmt);
+        addCatalog(conn, stmt, catNum);
+        success = true;
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to add catalog", e);
+    } finally {
+      try {
+        if (!success) {
+          conn.rollback();
+        }
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.", e);
+      }
+    }
+  }
+
+  private static final String CATALOG_EXISTS_QUERY =
+      "select <q>NAME<q> " +
+      "  from <q>CTLGS<q> " +
+      " where <q>NAME<q> = '%s'";
+
+  private boolean catalogExists(Statement stmt) throws SQLException {
+    String query = String.format(schemaTool.quote(CATALOG_EXISTS_QUERY), catName);
+    LOG.debug("Going to run " + query);
+    try (ResultSet rs = stmt.executeQuery(query)) {
+      if (rs.next()) {
+        System.out.println("Catalog " + catName + " already exists");
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  private static final String NEXT_CATALOG_ID_QUERY =
+      "select max(<q>CTLG_ID<q>) " +
+      "  from <q>CTLGS<q>";
+
+  private int getNextCatalogId(Statement stmt) throws SQLException, HiveMetaException {
+    String query = schemaTool.quote(NEXT_CATALOG_ID_QUERY);
+    LOG.debug("Going to run " + query);
+    try (ResultSet rs = stmt.executeQuery(query)) {
+      if (!rs.next()) {
+        throw new HiveMetaException("No catalogs found, have you upgraded the database?");
+      }
+      int nextId = rs.getInt(1) + 1;
+      // We need to stay out of the way of any sequences used by the underlying database.
+      // Otherwise the next time the client tries to add a catalog we'll get an error.
+      // There should never be billions of catalogs, so we'll shift our sequence number up
+      // there to avoid clashes.
+      int floor = 1 << 30;
+      return Math.max(nextId, floor);
+    }
+  }
+
+  private static final String ADD_CATALOG_STMT =
+      "insert into <q>CTLGS<q> (<q>CTLG_ID<q>, <q>NAME<q>, <qa>DESC<qa>, <q>LOCATION_URI<q>) " +
+      "     values (%d, '%s', '%s', '%s')";
+
+  private void addCatalog(Connection conn, Statement stmt, int catNum) throws SQLException {
+    String update = String.format(schemaTool.quote(ADD_CATALOG_STMT), catNum, catName, description, location);
+    LOG.debug("Going to run " + update);
+    stmt.execute(update);
+    conn.commit();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java
new file mode 100644
index 0000000..b70ea87
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInfo.java
@@ -0,0 +1,43 @@
+/*
+ * 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.beeline.schematool;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
+
+/**
+ * Print Hive version and schema version.
+ */
+class HiveSchemaToolTaskInfo extends HiveSchemaToolTask {
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    // do nothing
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    String hiveVersion = schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion();
+    MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(true);
+    String dbVersion = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
+
+    System.out.println("Hive distribution version:\t " + hiveVersion);
+    System.out.println("Metastore schema version:\t " + dbVersion);
+
+    schemaTool.assertCompatibleVersion(hiveVersion, dbVersion);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java
new file mode 100644
index 0000000..40fd1e7
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskInit.java
@@ -0,0 +1,73 @@
+/*
+ * 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.beeline.schematool;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+
+/**
+ * Initialize the metastore schema.
+ */
+class HiveSchemaToolTaskInit extends HiveSchemaToolTask {
+  private boolean validate = true;
+  private String toVersion;
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    if (cl.hasOption("initSchemaTo")) {
+      this.toVersion = cl.getOptionValue("initSchemaTo");
+      this.validate = false;
+    }
+  }
+
+  private void ensureToVersion() throws HiveMetaException {
+    if (toVersion != null) {
+      return;
+    }
+
+    // If null then current hive version is used
+    toVersion = schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion();
+    System.out.println("Initializing the schema to: " + toVersion);
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    ensureToVersion();
+
+    schemaTool.testConnectionToMetastore();
+    System.out.println("Starting metastore schema initialization to " + toVersion);
+
+    String initScriptDir = schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir();
+    String initScriptFile = schemaTool.getMetaStoreSchemaInfo().generateInitFileName(toVersion);
+
+    try {
+      System.out.println("Initialization script " + initScriptFile);
+      if (!schemaTool.isDryRun()) {
+        schemaTool.runBeeLine(initScriptDir, initScriptFile);
+        System.out.println("Initialization script completed");
+      }
+    } catch (IOException e) {
+      throw new HiveMetaException("Schema initialization FAILED! Metastore state would be inconsistent!", e);
+    }
+
+    if (validate) {
+      schemaTool.verifySchemaVersion();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java
new file mode 100644
index 0000000..ee7c0f8
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveDatabase.java
@@ -0,0 +1,96 @@
+/*
+ * 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.beeline.schematool;
+
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Print Hive version and schema version.
+ */
+class HiveSchemaToolTaskMoveDatabase extends HiveSchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskMoveDatabase.class.getName());
+
+  private String fromCatName;
+  private String toCatName;
+  private String dbName;
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    fromCatName = normalizeIdentifier(cl.getOptionValue("fromCatalog"));
+    toCatName = normalizeIdentifier(cl.getOptionValue("toCatalog"));
+    dbName = normalizeIdentifier(cl.getOptionValue("moveDatabase"));
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    System.out.println(String.format("Moving database %s from catalog %s to catalog %s",
+        dbName, fromCatName, toCatName));
+    Connection conn = schemaTool.getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        updateCatalogNameInTable(stmt, "DBS", "CTLG_NAME", "NAME", fromCatName, toCatName, dbName, false);
+        updateCatalogNameInTable(stmt, "TAB_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        updateCatalogNameInTable(stmt, "PART_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        updateCatalogNameInTable(stmt, "PARTITION_EVENTS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        updateCatalogNameInTable(stmt, "NOTIFICATION_LOG", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
+        conn.commit();
+        success = true;
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to move database", e);
+    } finally {
+      try {
+        if (!success) {
+          conn.rollback();
+        }
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.");
+      }
+    }
+  }
+
+  private static final String UPDATE_CATALOG_NAME_STMT =
+      "update <q>%s<q> " +
+      "   set <q>%s<q> = '%s' " +
+      " where <q>%s<q> = '%s' " +
+      "   and <q>%s<q> = '%s'";
+
+  private void updateCatalogNameInTable(Statement stmt, String tableName, String catColName, String dbColName,
+      String fromCatName, String toCatName, String dbName, boolean zeroUpdatesOk)
+      throws HiveMetaException, SQLException {
+    String update = String.format(schemaTool.quote(UPDATE_CATALOG_NAME_STMT), tableName, catColName, toCatName,
+        catColName, fromCatName, dbColName, dbName);
+    LOG.debug("Going to run " + update);
+    int numUpdated = stmt.executeUpdate(update);
+    if (numUpdated != 1 && !(zeroUpdatesOk && numUpdated == 0)) {
+      throw new HiveMetaException("Failed to properly update the " + tableName +
+          " table.  Expected to update 1 row but instead updated " + numUpdated);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java
new file mode 100644
index 0000000..fcefef8
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskMoveTable.java
@@ -0,0 +1,142 @@
+/*
+ * 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.beeline.schematool;
+
+import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Print Hive version and schema version.
+ */
+class HiveSchemaToolTaskMoveTable extends HiveSchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskMoveTable.class.getName());
+
+  private String fromCat;
+  private String toCat;
+  private String fromDb;
+  private String toDb;
+  private String tableName;
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    fromCat = normalizeIdentifier(cl.getOptionValue("fromCatalog"));
+    toCat = normalizeIdentifier(cl.getOptionValue("toCatalog"));
+    fromDb = normalizeIdentifier(cl.getOptionValue("fromDatabase"));
+    toDb = normalizeIdentifier(cl.getOptionValue("toDatabase"));
+    tableName = normalizeIdentifier(cl.getOptionValue("moveTable"));
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    Connection conn = schemaTool.getConnectionToMetastore(true);
+    boolean success = false;
+    try {
+      conn.setAutoCommit(false);
+      try (Statement stmt = conn.createStatement()) {
+        updateTableId(stmt);
+        updateDbNameForTable(stmt, "TAB_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        updateDbNameForTable(stmt, "PART_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        updateDbNameForTable(stmt, "PARTITION_EVENTS", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        updateDbNameForTable(stmt, "NOTIFICATION_LOG", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
+        conn.commit();
+        success = true;
+      }
+    } catch (SQLException se) {
+      throw new HiveMetaException("Failed to move table", se);
+    } finally {
+      try {
+        if (!success) {
+          conn.rollback();
+        }
+      } catch (SQLException e) {
+        // Not really much we can do here.
+        LOG.error("Failed to rollback, everything will probably go bad from here.");
+      }
+
+    }
+  }
+
+  private static final String UPDATE_TABLE_ID_STMT =
+      "update <q>TBLS<q> " +
+      "   set <q>DB_ID<q> = %d " +
+      " where <q>DB_ID<q> = %d " +
+      "   and <q>TBL_NAME<q> = '%s'";
+
+  private void updateTableId(Statement stmt) throws SQLException, HiveMetaException {
+    // Find the old database id
+    long oldDbId = getDbId(stmt, fromDb, fromCat);
+
+    // Find the new database id
+    long newDbId = getDbId(stmt, toDb, toCat);
+
+    String update = String.format(schemaTool.quote(UPDATE_TABLE_ID_STMT), newDbId, oldDbId, tableName);
+    LOG.debug("Going to run " + update);
+    int numUpdated = stmt.executeUpdate(update);
+    if (numUpdated != 1) {
+      throw new HiveMetaException(
+          "Failed to properly update TBLS table.  Expected to update " +
+              "1 row but instead updated " + numUpdated);
+    }
+  }
+
+  private static final String DB_ID_QUERY =
+      "select <q>DB_ID<q> " +
+      "  from <q>DBS<q> " +
+      " where <q>NAME<q> = '%s' " +
+      "   and <q>CTLG_NAME<q> = '%s'";
+
+  private long getDbId(Statement stmt, String db, String catalog) throws SQLException, HiveMetaException {
+    String query = String.format(schemaTool.quote(DB_ID_QUERY), db, catalog);
+    LOG.debug("Going to run " + query);
+    try (ResultSet rs = stmt.executeQuery(query)) {
+      if (!rs.next()) {
+        throw new HiveMetaException("Unable to find database " + fromDb);
+      }
+      return rs.getLong(1);
+    }
+  }
+
+  private static final String UPDATE_DB_NAME_STMT =
+      "update <q>%s<q> " +
+      "   set <q>CAT_NAME<q> = '%s', " +
+      "       <q>DB_NAME<q> = '%s' " +
+      " where <q>CAT_NAME<q> = '%s' " +
+      "   and <q>DB_NAME<q> = '%s' " +
+      "   and <q>%s<q> = '%s'";
+
+  private void updateDbNameForTable(Statement stmt, String tableName, String tableColumnName, String fromCat,
+      String toCat, String fromDb, String toDb, String hiveTblName) throws HiveMetaException, SQLException {
+    String update = String.format(schemaTool.quote(UPDATE_DB_NAME_STMT), tableName, toCat, toDb, fromCat, fromDb,
+        tableColumnName, hiveTblName);
+
+    LOG.debug("Going to run " + update);
+    int numUpdated = stmt.executeUpdate(update);
+    if (numUpdated > 1 || numUpdated < 0) {
+      throw new HiveMetaException("Failed to properly update the " + tableName +
+          " table.  Expected to update 1 row but instead updated " + numUpdated);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java
new file mode 100644
index 0000000..fa4742f
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskUpgrade.java
@@ -0,0 +1,116 @@
+/*
+ * 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.beeline.schematool;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
+
+/**
+ * Perform metastore schema upgrade.
+ */
+class HiveSchemaToolTaskUpgrade extends HiveSchemaToolTask {
+  private String fromVersion;
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    if (cl.hasOption("upgradeSchemaFrom")) {
+      this.fromVersion = cl.getOptionValue("upgradeSchemaFrom");
+    }
+  }
+
+  private void ensureFromVersion() throws HiveMetaException {
+    if (fromVersion != null) {
+      return;
+    }
+
+    // If null, then read from the metastore
+    MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(false);
+    fromVersion = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
+    if (fromVersion == null || fromVersion.isEmpty()) {
+      throw new HiveMetaException("Schema version not stored in the metastore. " +
+          "Metastore schema is too old or corrupt. Try specifying the version manually");
+    }
+    System.out.println("Upgrading from the version " + fromVersion);
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    ensureFromVersion();
+
+    if (schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion().equals(fromVersion)) {
+      System.out.println("No schema upgrade required from version " + fromVersion);
+      return;
+    }
+
+    // Find the list of scripts to execute for this upgrade
+    List<String> upgradeScripts = schemaTool.getMetaStoreSchemaInfo().getUpgradeScripts(fromVersion);
+    schemaTool.testConnectionToMetastore();
+    System.out.println("Starting upgrade metastore schema from version " + fromVersion + " to " +
+        schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion());
+    String scriptDir = schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir();
+    try {
+      for (String scriptFile : upgradeScripts) {
+        System.out.println("Upgrade script " + scriptFile);
+        if (!schemaTool.isDryRun()) {
+          runPreUpgrade(scriptDir, scriptFile);
+          schemaTool.runBeeLine(scriptDir, scriptFile);
+          System.out.println("Completed " + scriptFile);
+        }
+      }
+    } catch (IOException e) {
+      throw new HiveMetaException("Upgrade FAILED! Metastore state would be inconsistent !!", e);
+    }
+
+    // Revalidated the new version after upgrade
+    schemaTool.verifySchemaVersion();
+  }
+
+  /**
+   *  Run pre-upgrade scripts corresponding to a given upgrade script,
+   *  if any exist. The errors from pre-upgrade are ignored.
+   *  Pre-upgrade scripts typically contain setup statements which
+   *  may fail on some database versions and failure is ignorable.
+   *
+   *  @param scriptDir upgrade script directory name
+   *  @param scriptFile upgrade script file name
+   */
+  private void runPreUpgrade(String scriptDir, String scriptFile) {
+    for (int i = 0;; i++) {
+      String preUpgradeScript = schemaTool.getMetaStoreSchemaInfo().getPreUpgradeScriptName(i, scriptFile);
+      File preUpgradeScriptFile = new File(scriptDir, preUpgradeScript);
+      if (!preUpgradeScriptFile.isFile()) {
+        break;
+      }
+
+      try {
+        schemaTool.runBeeLine(scriptDir, preUpgradeScript);
+        System.out.println("Completed " + preUpgradeScript);
+      } catch (Exception e) {
+        // Ignore the pre-upgrade script errors
+        System.err.println("Warning in pre-upgrade script " + preUpgradeScript + ": " + e.getMessage());
+        if (schemaTool.isVerbose()) {
+          e.printStackTrace();
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java
new file mode 100644
index 0000000..c4f6d19
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaToolTaskValidate.java
@@ -0,0 +1,631 @@
+/*
+ * 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.beeline.schematool;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.net.URI;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Print Hive version and schema version.
+ */
+class HiveSchemaToolTaskValidate extends HiveSchemaToolTask {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaToolTaskValidate.class.getName());
+
+  @Override
+  void setCommandLineArguments(HiveSchemaToolCommandLine cl) {
+    // do nothing
+  }
+
+  @Override
+  void execute() throws HiveMetaException {
+    System.out.println("Starting metastore validation\n");
+    Connection conn = schemaTool.getConnectionToMetastore(false);
+    boolean success = true;
+    try {
+      success &= validateSchemaVersions();
+      success &= validateSequences(conn);
+      success &= validateSchemaTables(conn);
+      success &= validateLocations(conn, schemaTool.getValidationServers());
+      success &= validateColumnNullValues(conn);
+    } finally {
+      if (conn != null) {
+        try {
+          conn.close();
+        } catch (SQLException e) {
+          throw new HiveMetaException("Failed to close metastore connection", e);
+        }
+      }
+    }
+
+    System.out.print("Done with metastore validation: ");
+    if (!success) {
+      System.out.println("[FAIL]");
+      throw new HiveMetaException("Validation failed");
+    } else {
+      System.out.println("[SUCCESS]");
+    }
+  }
+
+  boolean validateSchemaVersions() throws HiveMetaException {
+    System.out.println("Validating schema version");
+    try {
+      String hiveSchemaVersion = schemaTool.getMetaStoreSchemaInfo().getHiveSchemaVersion();
+      MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(false);
+      String newSchemaVersion = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
+      schemaTool.assertCompatibleVersion(hiveSchemaVersion, newSchemaVersion);
+    } catch (HiveMetaException hme) {
+      if (hme.getMessage().contains("Metastore schema version is not compatible") ||
+          hme.getMessage().contains("Multiple versions were found in metastore") ||
+          hme.getMessage().contains("Could not find version info in metastore VERSION table")) {
+        System.err.println(hme.getMessage());
+        System.out.println("[FAIL]\n");
+        return false;
+      } else {
+        throw hme;
+      }
+    }
+    System.out.println("[SUCCESS]\n");
+    return true;
+  }
+
+  private static final String QUERY_SEQ =
+      "  select t.<q>NEXT_VAL<q>" +
+      "    from <q>SEQUENCE_TABLE<q> t " +
+      "   where t.<q>SEQUENCE_NAME<q> = ? " +
+      "order by t.<q>SEQUENCE_NAME<q>";
+
+  private static final String QUERY_MAX_ID =
+      "select max(<q>%s<q>)" +
+      "  from <q>%s<q>";
+
+  @VisibleForTesting
+  boolean validateSequences(Connection conn) throws HiveMetaException {
+    Map<String, Pair<String, String>> seqNameToTable =
+        new ImmutableMap.Builder<String, Pair<String, String>>()
+        .put("MDatabase", Pair.of("DBS", "DB_ID"))
+        .put("MRole", Pair.of("ROLES", "ROLE_ID"))
+        .put("MGlobalPrivilege", Pair.of("GLOBAL_PRIVS", "USER_GRANT_ID"))
+        .put("MTable", Pair.of("TBLS","TBL_ID"))
+        .put("MStorageDescriptor", Pair.of("SDS", "SD_ID"))
+        .put("MSerDeInfo", Pair.of("SERDES", "SERDE_ID"))
+        .put("MColumnDescriptor", Pair.of("CDS", "CD_ID"))
+        .put("MTablePrivilege", Pair.of("TBL_PRIVS", "TBL_GRANT_ID"))
+        .put("MTableColumnStatistics", Pair.of("TAB_COL_STATS", "CS_ID"))
+        .put("MPartition", Pair.of("PARTITIONS", "PART_ID"))
+        .put("MPartitionColumnStatistics", Pair.of("PART_COL_STATS", "CS_ID"))
+        .put("MFunction", Pair.of("FUNCS", "FUNC_ID"))
+        .put("MIndex", Pair.of("IDXS", "INDEX_ID"))
+        .put("MStringList", Pair.of("SKEWED_STRING_LIST", "STRING_LIST_ID"))
+        .build();
+
+    System.out.println("Validating sequence number for SEQUENCE_TABLE");
+
+    boolean isValid = true;
+    try {
+      Statement stmt = conn.createStatement();
+      for (Map.Entry<String, Pair<String, String>> e : seqNameToTable.entrySet()) {
+        String tableName = e.getValue().getLeft();
+        String tableKey = e.getValue().getRight();
+        String fullSequenceName = "org.apache.hadoop.hive.metastore.model." + e.getKey();
+        String seqQuery = schemaTool.quote(QUERY_SEQ);
+        String maxIdQuery = String.format(schemaTool.quote(QUERY_MAX_ID), tableKey, tableName);
+
+        ResultSet res = stmt.executeQuery(maxIdQuery);
+        if (res.next()) {
+          long maxId = res.getLong(1);
+          if (maxId > 0) {
+            PreparedStatement stmtSeq = conn.prepareStatement(seqQuery);
+            stmtSeq.setString(1, fullSequenceName);
+            ResultSet resSeq = stmtSeq.executeQuery();
+            if (!resSeq.next()) {
+              isValid = false;
+              System.err.println("Missing SEQUENCE_NAME " + e.getKey() + " from SEQUENCE_TABLE");
+            } else if (resSeq.getLong(1) < maxId) {
+              isValid = false;
+              System.err.println("NEXT_VAL for " + e.getKey() + " in SEQUENCE_TABLE < max(" + tableKey +
+                  ") in " + tableName);
+            }
+          }
+        }
+      }
+
+      System.out.println(isValid ? "[SUCCESS]\n" :"[FAIL]\n");
+      return isValid;
+    } catch (SQLException e) {
+        throw new HiveMetaException("Failed to validate sequence number for SEQUENCE_TABLE", e);
+    }
+  }
+
+  @VisibleForTesting
+  boolean validateSchemaTables(Connection conn) throws HiveMetaException {
+    System.out.println("Validating metastore schema tables");
+    String version = null;
+    try {
+      MetaStoreConnectionInfo connectionInfo = schemaTool.getConnectionInfo(false);
+      version = schemaTool.getMetaStoreSchemaInfo().getMetaStoreSchemaVersion(connectionInfo);
+    } catch (HiveMetaException he) {
+      System.err.println("Failed to determine schema version from Hive Metastore DB. " + he.getMessage());
+      System.out.println("Failed in schema table validation.");
+      LOG.debug("Failed to determine schema version from Hive Metastore DB," + he.getMessage(), he);
+      return false;
+    }
+
+    Connection hmsConn = schemaTool.getConnectionToMetastore(false);
+
+    LOG.debug("Validating tables in the schema for version " + version);
+    List<String> dbTables = new ArrayList<String>();
+    ResultSet rs = null;
+    try {
+      String schema = null;
+      try {
+        schema = hmsConn.getSchema();
+      } catch (SQLFeatureNotSupportedException e) {
+        LOG.debug("schema is not supported");
+      }
+
+      DatabaseMetaData metadata = conn.getMetaData();
+      rs = metadata.getTables(null, schema, "%", new String[] {"TABLE"});
+
+      while (rs.next()) {
+        String table = rs.getString("TABLE_NAME");
+        dbTables.add(table.toLowerCase());
+        LOG.debug("Found table " + table + " in HMS dbstore");
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to retrieve schema tables from Hive Metastore DB," +
+          e.getMessage(), e);
+    } finally {
+      if (rs != null) {
+        try {
+          rs.close();
+        } catch (SQLException e) {
+          throw new HiveMetaException("Failed to close resultset", e);
+        }
+      }
+    }
+
+    // parse the schema file to determine the tables that are expected to exist
+    // we are using oracle schema because it is simpler to parse, no quotes or backticks etc
+    List<String> schemaTables = new ArrayList<String>();
+    List<String> subScripts   = new ArrayList<String>();
+
+    String baseDir    = new File(schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir()).getParent();
+    String schemaFile = new File(schemaTool.getMetaStoreSchemaInfo().getMetaStoreScriptDir(),
+        schemaTool.getMetaStoreSchemaInfo().generateInitFileName(version)).getPath();
+    try {
+      LOG.debug("Parsing schema script " + schemaFile);
+      subScripts.addAll(findCreateTable(schemaFile, schemaTables));
+      while (subScripts.size() > 0) {
+        schemaFile = baseDir + "/" + schemaTool.getDbType() + "/" + subScripts.remove(0);
+        LOG.debug("Parsing subscript " + schemaFile);
+        subScripts.addAll(findCreateTable(schemaFile, schemaTables));
+      }
+    } catch (Exception e) {
+      System.err.println("Exception in parsing schema file. Cause:" + e.getMessage());
+      System.out.println("Failed in schema table validation.");
+      return false;
+    }
+
+    LOG.debug("Schema tables:[ " + Arrays.toString(schemaTables.toArray()) + " ]");
+    LOG.debug("DB tables:[ " + Arrays.toString(dbTables.toArray()) + " ]");
+
+    // now diff the lists
+    schemaTables.removeAll(dbTables);
+    if (schemaTables.size() > 0) {
+      Collections.sort(schemaTables);
+      System.err.println("Table(s) [ " + Arrays.toString(schemaTables.toArray()) + " ] " +
+          "are missing from the metastore database schema.");
+      System.out.println("[FAIL]\n");
+      return false;
+    } else {
+      System.out.println("[SUCCESS]\n");
+      return true;
+    }
+  }
+
+  @VisibleForTesting
+  List<String> findCreateTable(String path, List<String> tableList) throws Exception {
+    if (!(new File(path)).exists()) {
+      throw new Exception(path + " does not exist. Potentially incorrect version in the metastore VERSION table");
+    }
+
+    List<String> subs = new ArrayList<String>();
+    NestedScriptParser sp = HiveSchemaHelper.getDbCommandParser(schemaTool.getDbType(), false);
+    Pattern regexp = Pattern.compile("CREATE TABLE(\\s+IF NOT EXISTS)?\\s+(\\S+).*");
+
+    try (BufferedReader reader = new BufferedReader(new FileReader(path));) {
+      String line = null;
+      while ((line = reader.readLine()) != null) {
+        if (sp.isNestedScript(line)) {
+          String subScript = sp.getScriptName(line);
+          LOG.debug("Schema subscript " + subScript + " found");
+          subs.add(subScript);
+          continue;
+        }
+        line = line.replaceAll("( )+", " "); //suppress multi-spaces
+        line = line.replaceAll("\\(", " ");
+        line = line.replaceAll("IF NOT EXISTS ", "");
+        line = line.replaceAll("`", "");
+        line = line.replaceAll("'", "");
+        line = line.replaceAll("\"", "");
+        Matcher matcher = regexp.matcher(line);
+
+        if (matcher.find()) {
+          String table = matcher.group(2);
+          if (schemaTool.getDbType().equals("derby")) {
+            table = table.replaceAll("APP\\.", "");
+          }
+          tableList.add(table.toLowerCase());
+          LOG.debug("Found table " + table + " in the schema");
+        }
+      }
+    } catch (IOException ex){
+      throw new Exception(ex.getMessage());
+    }
+
+    return subs;
+  }
+
+  @VisibleForTesting
+  boolean validateLocations(Connection conn, URI[] defaultServers) throws HiveMetaException {
+    System.out.println("Validating DFS locations");
+    boolean rtn = true;
+    rtn &= checkMetaStoreDBLocation(conn, defaultServers);
+    rtn &= checkMetaStoreTableLocation(conn, defaultServers);
+    rtn &= checkMetaStorePartitionLocation(conn, defaultServers);
+    rtn &= checkMetaStoreSkewedColumnsLocation(conn, defaultServers);
+    System.out.println(rtn ? "[SUCCESS]\n" : "[FAIL]\n");
+    return rtn;
+  }
+
+  private static final String QUERY_DB_LOCATION =
+      "  select dbt.<q>DB_ID<q>, " +
+      "         dbt.<q>NAME<q>, " +
+      "         dbt.<q>DB_LOCATION_URI<q> " +
+      "    from <q>DBS<q> dbt " +
+      "order by dbt.<q>DB_ID<q> ";
+
+  private boolean checkMetaStoreDBLocation(Connection conn, URI[] defaultServers) throws HiveMetaException {
+    String dbLocQuery = schemaTool.quote(QUERY_DB_LOCATION);
+
+    int numOfInvalid = 0;
+    try (Statement stmt = conn.createStatement();
+         ResultSet res = stmt.executeQuery(dbLocQuery)) {
+      while (res.next()) {
+        String locValue = res.getString(3);
+        String dbName = getNameOrID(res, 2, 1);
+        if (!checkLocation("Database " + dbName, locValue, defaultServers)) {
+          numOfInvalid++;
+        }
+      }
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to get DB Location Info.", e);
+    }
+    return numOfInvalid == 0;
+  }
+
+  private static final String TAB_ID_RANGE_QUERY =
+      "select max(<q>TBL_ID<q>), " +
+      "       min(<q>TBL_ID<q>) " +
+      "  from <q>TBLS<q> ";
+
+  private static final String TAB_LOC_QUERY =
+      "    select tbl.<q>TBL_ID<q>, " +
+      "           tbl.<q>TBL_NAME<q>, " +
+      "           sd.<q>LOCATION<q>, " +
+      "           dbt.<q>DB_ID<q>, " +
+      "           dbt.<q>NAME<q> " +
+      "      from <q>TBLS<q> tbl " +
+      "inner join <q>SDS<q> sd on sd.<q>SD_ID<q> = tbl.<q>SD_ID<q> " +
+      "inner join <q>DBS<q> dbt on tbl.<q>DB_ID<q> = dbt.<q>DB_ID<q> " +
+      "     where tbl.<q>TBL_TYPE<q> != '%s' " +
+      "       and tbl.<q>TBL_ID<q> >= ? " +
+      "       and tbl.<q>TBL_ID<q> <= ? " +
+      "  order by tbl.<q>TBL_ID<q> ";
+
+  private static final int TAB_LOC_CHECK_SIZE = 2000;
+
+  private boolean checkMetaStoreTableLocation(Connection conn, URI[] defaultServers)
+      throws HiveMetaException {
+    String tabIDRangeQuery = schemaTool.quote(TAB_ID_RANGE_QUERY);
+    String tabLocQuery = String.format(schemaTool.quote(TAB_LOC_QUERY), TableType.VIRTUAL_VIEW);
+
+    try {
+      long maxID = 0, minID = 0;
+      try (Statement stmt = conn.createStatement();
+           ResultSet res = stmt.executeQuery(tabIDRangeQuery)) {
+        if (res.next()) {
+          maxID = res.getLong(1);
+          minID = res.getLong(2);
+        }
+      }
+
+      int numOfInvalid = 0;
+      try (PreparedStatement pStmt = conn.prepareStatement(tabLocQuery)) {
+        while (minID <= maxID) {
+          pStmt.setLong(1, minID);
+          pStmt.setLong(2, minID + TAB_LOC_CHECK_SIZE);
+          try (ResultSet res = pStmt.executeQuery()) {
+            while (res.next()) {
+              String locValue = res.getString(3);
+              String entity = "Database " + getNameOrID(res, 5, 4) + ", Table "  + getNameOrID(res, 2, 1);
+              if (!checkLocation(entity, locValue, defaultServers)) {
+                numOfInvalid++;
+              }
+            }
+          }
+          minID += TAB_LOC_CHECK_SIZE + 1;
+        }
+      }
+
+      return numOfInvalid == 0;
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to get Table Location Info.", e);
+    }
+  }
+
+  private static final String QUERY_PART_ID_RANGE =
+      "select max(<q>PART_ID<q>)," +
+      "       min(<q>PART_ID<q>)" +
+      "  from <q>PARTITIONS<q> ";
+
+  private static final String QUERY_PART_LOC =
+      "    select pt.<q>PART_ID<q>, " +
+      "           pt.<q>PART_NAME<q>, " +
+      "           sd.<q>LOCATION<q>, " +
+      "           tbl.<q>TBL_ID<q>, " +
+      "           tbl.<q>TBL_NAME<q>, " +
+      "           dbt.<q>DB_ID<q>, " +
+      "           dbt.<q>NAME<q> " +
+      "      from <q>PARTITIONS<q> pt " +
+      "inner join <q>SDS<q> sd on sd.<q>SD_ID<q> = pt.<q>SD_ID<q> " +
+      "inner join <q>TBLS<q> tbl on tbl.<q>TBL_ID<q> = pt.<q>TBL_ID<q> " +
+      "inner join <q>DBS<q> dbt on dbt.<q>DB_ID<q> = tbl.<q>DB_ID<q> " +
+      "     where pt.<q>PART_ID<q> >= ? " +
+      "       and pt.<q>PART_ID<q> <= ? " +
+      "  order by tbl.<q>TBL_ID<q> ";
+
+  private static final int PART_LOC_CHECK_SIZE = 2000;
+
+  private boolean checkMetaStorePartitionLocation(Connection conn, URI[] defaultServers)
+      throws HiveMetaException {
+    String queryPartIDRange = schemaTool.quote(QUERY_PART_ID_RANGE);
+    String queryPartLoc = schemaTool.quote(QUERY_PART_LOC);
+
+    try {
+      long maxID = 0, minID = 0;
+      try (Statement stmt = conn.createStatement();
+           ResultSet res = stmt.executeQuery(queryPartIDRange)) {
+        if (res.next()) {
+          maxID = res.getLong(1);
+          minID = res.getLong(2);
+        }
+      }
+
+      int numOfInvalid = 0;
+      try (PreparedStatement pStmt = conn.prepareStatement(queryPartLoc)) {
+        while (minID <= maxID) {
+          pStmt.setLong(1, minID);
+          pStmt.setLong(2, minID + PART_LOC_CHECK_SIZE);
+          try (ResultSet res = pStmt.executeQuery()) {
+            while (res.next()) {
+              String locValue = res.getString(3);
+              String entity = "Database " + getNameOrID(res, 7, 6) + ", Table "  + getNameOrID(res, 5, 4) +
+                  ", Partition " + getNameOrID(res, 2, 1);
+              if (!checkLocation(entity, locValue, defaultServers)) {
+                numOfInvalid++;
+              }
+            }
+          }
+          minID += PART_LOC_CHECK_SIZE + 1;
+        }
+      }
+
+      return numOfInvalid == 0;
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to get Partition Location Info.", e);
+    }
+  }
+
+  private static final String QUERY_SKEWED_COL_ID_RANGE =
+      "select max(<q>STRING_LIST_ID_KID<q>), " +
+      "       min(<q>STRING_LIST_ID_KID<q>) " +
+      "  from <q>SKEWED_COL_VALUE_LOC_MAP<q> ";
+
+  private static final String QUERY_SKEWED_COL_LOC =
+      "  select t.<q>TBL_NAME<q>, " +
+      "         t.<q>TBL_ID<q>, " +
+      "         sk.<q>STRING_LIST_ID_KID<q>, " +
+      "         sk.<q>LOCATION<q>, " +
+      "         db.<q>NAME<q>, " +
+      "         db.<q>DB_ID<q> " +
+      "    from <q>TBLS<q> t " +
+      "    join <q>SDS<q> s on s.<q>SD_ID<q> = t.<q>SD_ID<q> " +
+      "    join <q>DBS<q> db on db.<q>DB_ID<q> = t.<q>DB_ID<q> " +
+      "    join <q>SKEWED_COL_VALUE_LOC_MAP<q> sk on sk.<q>SD_ID<q> = s.<q>SD_ID<q> " +
+      "   where sk.<q>STRING_LIST_ID_KID<q> >= ? " +
+      "     and sk.<q>STRING_LIST_ID_KID<q> <= ? " +
+      "order by t.<q>TBL_ID<q> ";
+
+  private static final int SKEWED_COL_LOC_CHECK_SIZE = 2000;
+
+  private boolean checkMetaStoreSkewedColumnsLocation(Connection conn, URI[] defaultServers)
+      throws HiveMetaException {
+    String querySkewedColIDRange = schemaTool.quote(QUERY_SKEWED_COL_ID_RANGE);
+    String querySkewedColLoc = schemaTool.quote(QUERY_SKEWED_COL_LOC);
+
+    try {
+      long maxID = 0, minID = 0;
+      try (Statement stmt = conn.createStatement();
+           ResultSet res = stmt.executeQuery(querySkewedColIDRange)) {
+        if (res.next()) {
+          maxID = res.getLong(1);
+          minID = res.getLong(2);
+        }
+      }
+
+      int numOfInvalid = 0;
+      try (PreparedStatement pStmt = conn.prepareStatement(querySkewedColLoc)) {
+        while (minID <= maxID) {
+          pStmt.setLong(1, minID);
+          pStmt.setLong(2, minID + SKEWED_COL_LOC_CHECK_SIZE);
+          try (ResultSet res = pStmt.executeQuery()) {
+            while (res.next()) {
+              String locValue = res.getString(4);
+              String entity = "Database " + getNameOrID(res, 5, 6) + ", Table " + getNameOrID(res, 1, 2) +
+                  ", String list " + res.getString(3);
+              if (!checkLocation(entity, locValue, defaultServers)) {
+                numOfInvalid++;
+              }
+            }
+          }
+          minID += SKEWED_COL_LOC_CHECK_SIZE + 1;
+        }
+      }
+
+      return numOfInvalid == 0;
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to get skewed columns location info.", e);
+    }
+  }
+
+  /**
+   * Check if the location is valid for the given entity.
+   * @param entity          the entity to represent a database, partition or table
+   * @param entityLocation  the location
+   * @param defaultServers  a list of the servers that the location needs to match.
+   *                        The location host needs to match one of the given servers.
+   *                        If empty, then no check against such list.
+   * @return true if the location is valid
+   */
+  private boolean checkLocation(String entity, String entityLocation, URI[] defaultServers) {
+    boolean isValid = true;
+
+    if (entityLocation == null) {
+      System.err.println(entity + ", Error: empty location");
+      isValid = false;
+    } else {
+      try {
+        URI currentUri = new Path(entityLocation).toUri();
+        String scheme = currentUri.getScheme();
+        String path   = currentUri.getPath();
+        if (StringUtils.isEmpty(scheme)) {
+          System.err.println(entity + ", Location: "+ entityLocation + ", Error: missing location scheme.");
+          isValid = false;
+        } else if (StringUtils.isEmpty(path)) {
+          System.err.println(entity + ", Location: "+ entityLocation + ", Error: missing location path.");
+          isValid = false;
+        } else if (ArrayUtils.isNotEmpty(defaultServers) && currentUri.getAuthority() != null) {
+          String authority = currentUri.getAuthority();
+          boolean matchServer = false;
+          for(URI server : defaultServers) {
+            if (StringUtils.equalsIgnoreCase(server.getScheme(), scheme) &&
+                StringUtils.equalsIgnoreCase(server.getAuthority(), authority)) {
+              matchServer = true;
+              break;
+            }
+          }
+          if (!matchServer) {
+            System.err.println(entity + ", Location: " + entityLocation + ", Error: mismatched server.");
+            isValid = false;
+          }
+        }
+
+        // if there is no path element other than "/", report it but not fail
+        if (isValid && StringUtils.containsOnly(path, "/")) {
+          System.err.println(entity + ", Location: "+ entityLocation + ", Warn: location set to root, " +
+              "not a recommended config.");
+        }
+      } catch (Exception pe) {
+        System.err.println(entity + ", Error: invalid location - " + pe.getMessage());
+        isValid =false;
+      }
+    }
+
+    return isValid;
+  }
+
+  private String getNameOrID(ResultSet res, int nameInx, int idInx) throws SQLException {
+    String itemName = res.getString(nameInx);
+    return  (itemName == null || itemName.isEmpty()) ? "ID: " + res.getString(idInx) : "Name: " + itemName;
+  }
+
+  private static final String QUERY_COLUMN_NULL_VALUES =
+      "  select t.*" +
+      "    from <q>TBLS<q> t" +
+      "   where t.<q>SD_ID<q> IS NULL" +
+      "     and (t.<q>TBL_TYPE<q> = '" + TableType.EXTERNAL_TABLE + "' or" +
+      "          t.<q>TBL_TYPE<q> = '" + TableType.MANAGED_TABLE + "') " +
+      "order by t.<q>TBL_ID<q> ";
+
+  @VisibleForTesting
+  boolean validateColumnNullValues(Connection conn) throws HiveMetaException {
+    System.out.println("Validating columns for incorrect NULL values.");
+
+    boolean isValid = true;
+    String queryColumnNullValues = schemaTool.quote(QUERY_COLUMN_NULL_VALUES);
+
+    try (Statement stmt = conn.createStatement();
+         ResultSet res = stmt.executeQuery(queryColumnNullValues)) {
+      while (res.next()) {
+         long tableId = res.getLong("TBL_ID");
+         String tableName = res.getString("TBL_NAME");
+         String tableType = res.getString("TBL_TYPE");
+         isValid = false;
+         System.err.println("SD_ID in TBLS should not be NULL for Table Name=" + tableName + ", Table ID=" + tableId + ", Table Type=" + tableType);
+      }
+
+      System.out.println(isValid ? "[SUCCESS]\n" : "[FAIL]\n");
+      return isValid;
+    } catch(SQLException e) {
+        throw new HiveMetaException("Failed to validate columns for incorrect NULL values", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java b/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
deleted file mode 100644
index 3d5f086..0000000
--- a/beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java
+++ /dev/null
@@ -1,92 +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.hive.beeline;
-
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.powermock.core.classloader.annotations.PowerMockIgnore;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Matchers.same;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.mockStatic;
-import static org.powermock.api.mockito.PowerMockito.verifyStatic;
-
-@RunWith(PowerMockRunner.class)
-@PowerMockIgnore("javax.management.*")
-@PrepareForTest({ HiveSchemaHelper.class, HiveSchemaTool.CommandBuilder.class })
-public class TestHiveSchemaTool {
-
-  String scriptFile = System.getProperty("java.io.tmpdir") + File.separator + "someScript.sql";
-  @Mock
-  private HiveConf hiveConf;
-  private HiveSchemaTool.CommandBuilder builder;
-  private String pasword = "reallySimplePassword";
-
-  @Before
-  public void setup() throws IOException {
-    mockStatic(HiveSchemaHelper.class);
-    when(HiveSchemaHelper
-        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECT_URL_KEY), same(hiveConf)))
-        .thenReturn("someURL");
-    when(HiveSchemaHelper
-        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTION_DRIVER), same(hiveConf)))
-        .thenReturn("someDriver");
-
-    File file = new File(scriptFile);
-    if (!file.exists()) {
-      file.createNewFile();
-    }
-    builder = new HiveSchemaTool.CommandBuilder(hiveConf, null, null, "testUser", pasword, scriptFile);
-  }
-
-  @After
-  public void globalAssert() throws IOException {
-    verifyStatic();
-    HiveSchemaHelper.getValidConfVar(eq(MetastoreConf.ConfVars.CONNECT_URL_KEY), same(hiveConf));
-    HiveSchemaHelper
-        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTION_DRIVER), same(hiveConf));
-
-    new File(scriptFile).delete();
-  }
-
-  @Test
-  public void shouldReturnStrippedPassword() throws IOException {
-    assertFalse(builder.buildToLog().contains(pasword));
-  }
-
-  @Test
-  public void shouldReturnActualPassword() throws IOException {
-    String[] strings = builder.buildToRun();
-    assertTrue(Arrays.asList(strings).contains(pasword));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java b/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
new file mode 100644
index 0000000..8b477bd
--- /dev/null
+++ b/beeline/src/test/org/apache/hive/beeline/schematool/TestHiveSchemaTool.java
@@ -0,0 +1,92 @@
+/*
+ * 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.beeline.schematool;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.same;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.mockStatic;
+import static org.powermock.api.mockito.PowerMockito.verifyStatic;
+
+@RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
+@PrepareForTest({ HiveSchemaHelper.class, HiveSchemaTool.CommandBuilder.class })
+public class TestHiveSchemaTool {
+
+  String scriptFile = System.getProperty("java.io.tmpdir") + File.separator + "someScript.sql";
+  @Mock
+  private HiveConf hiveConf;
+  private HiveSchemaTool.CommandBuilder builder;
+  private String pasword = "reallySimplePassword";
+
+  @Before
+  public void setup() throws IOException {
+    mockStatic(HiveSchemaHelper.class);
+    when(HiveSchemaHelper
+        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECT_URL_KEY), same(hiveConf)))
+        .thenReturn("someURL");
+    when(HiveSchemaHelper
+        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTION_DRIVER), same(hiveConf)))
+        .thenReturn("someDriver");
+
+    File file = new File(scriptFile);
+    if (!file.exists()) {
+      file.createNewFile();
+    }
+    builder = new HiveSchemaTool.CommandBuilder(hiveConf, null, null, "testUser", pasword, scriptFile);
+  }
+
+  @After
+  public void globalAssert() throws IOException {
+    verifyStatic();
+    HiveSchemaHelper.getValidConfVar(eq(MetastoreConf.ConfVars.CONNECT_URL_KEY), same(hiveConf));
+    HiveSchemaHelper
+        .getValidConfVar(eq(MetastoreConf.ConfVars.CONNECTION_DRIVER), same(hiveConf));
+
+    new File(scriptFile).delete();
+  }
+
+  @Test
+  public void shouldReturnStrippedPassword() throws IOException {
+    assertFalse(builder.buildToLog().contains(pasword));
+  }
+
+  @Test
+  public void shouldReturnActualPassword() throws IOException {
+    String[] strings = builder.buildToRun();
+    assertTrue(Arrays.asList(strings).contains(pasword));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/bin/ext/schemaTool.sh
----------------------------------------------------------------------
diff --git a/bin/ext/schemaTool.sh b/bin/ext/schemaTool.sh
index 94c56ef..2bed678 100644
--- a/bin/ext/schemaTool.sh
+++ b/bin/ext/schemaTool.sh
@@ -18,12 +18,12 @@ export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
 
 schemaTool() {
   HIVE_OPTS=''
-  CLASS=org.apache.hive.beeline.HiveSchemaTool
+  CLASS=org.apache.hive.beeline.schematool.HiveSchemaTool
   execHiveCmd $CLASS "$@"
 }
 
 schemaTool_help () {
   HIVE_OPTS=''
-  CLASS=org.apache.hive.beeline.HiveSchemaTool
+  CLASS=org.apache.hive.beeline.schematool.HiveSchemaTool
   execHiveCmd $CLASS "--help"
 }


[26/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index 672ebf9..ec129ef 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -372,6 +372,8 @@ import org.slf4j.LoggerFactory;
 
     public void flushCache() throws org.apache.thrift.TException;
 
+    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst) throws org.apache.thrift.TException;
+
     public CmRecycleResponse cm_recycle(CmRecycleRequest request) throws MetaException, org.apache.thrift.TException;
 
     public GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req) throws org.apache.thrift.TException;
@@ -790,6 +792,8 @@ import org.slf4j.LoggerFactory;
 
     public void flushCache(org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void add_write_notification_log(WriteNotificationLogRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void cm_recycle(CmRecycleRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void get_file_metadata_by_expr(GetFileMetadataByExprRequest req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -5644,6 +5648,29 @@ import org.slf4j.LoggerFactory;
       return;
     }
 
+    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst) throws org.apache.thrift.TException
+    {
+      send_add_write_notification_log(rqst);
+      return recv_add_write_notification_log();
+    }
+
+    public void send_add_write_notification_log(WriteNotificationLogRequest rqst) throws org.apache.thrift.TException
+    {
+      add_write_notification_log_args args = new add_write_notification_log_args();
+      args.setRqst(rqst);
+      sendBase("add_write_notification_log", args);
+    }
+
+    public WriteNotificationLogResponse recv_add_write_notification_log() throws org.apache.thrift.TException
+    {
+      add_write_notification_log_result result = new add_write_notification_log_result();
+      receiveBase(result, "add_write_notification_log");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "add_write_notification_log failed: unknown result");
+    }
+
     public CmRecycleResponse cm_recycle(CmRecycleRequest request) throws MetaException, org.apache.thrift.TException
     {
       send_cm_recycle(request);
@@ -12619,6 +12646,38 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void add_write_notification_log(WriteNotificationLogRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      add_write_notification_log_call method_call = new add_write_notification_log_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private WriteNotificationLogRequest rqst;
+      public add_write_notification_log_call(WriteNotificationLogRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.rqst = rqst;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("add_write_notification_log", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        add_write_notification_log_args args = new add_write_notification_log_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public WriteNotificationLogResponse getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_add_write_notification_log();
+      }
+    }
+
     public void cm_recycle(CmRecycleRequest request, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       cm_recycle_call method_call = new cm_recycle_call(request, resultHandler, this, ___protocolFactory, ___transport);
@@ -14150,6 +14209,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_notification_events_count", new get_notification_events_count());
       processMap.put("fire_listener_event", new fire_listener_event());
       processMap.put("flushCache", new flushCache());
+      processMap.put("add_write_notification_log", new add_write_notification_log());
       processMap.put("cm_recycle", new cm_recycle());
       processMap.put("get_file_metadata_by_expr", new get_file_metadata_by_expr());
       processMap.put("get_file_metadata", new get_file_metadata());
@@ -18426,6 +18486,26 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log<I extends Iface> extends org.apache.thrift.ProcessFunction<I, add_write_notification_log_args> {
+      public add_write_notification_log() {
+        super("add_write_notification_log");
+      }
+
+      public add_write_notification_log_args getEmptyArgsInstance() {
+        return new add_write_notification_log_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public add_write_notification_log_result getResult(I iface, add_write_notification_log_args args) throws org.apache.thrift.TException {
+        add_write_notification_log_result result = new add_write_notification_log_result();
+        result.success = iface.add_write_notification_log(args.rqst);
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cm_recycle<I extends Iface> extends org.apache.thrift.ProcessFunction<I, cm_recycle_args> {
       public cm_recycle() {
         super("cm_recycle");
@@ -19683,6 +19763,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("get_notification_events_count", new get_notification_events_count());
       processMap.put("fire_listener_event", new fire_listener_event());
       processMap.put("flushCache", new flushCache());
+      processMap.put("add_write_notification_log", new add_write_notification_log());
       processMap.put("cm_recycle", new cm_recycle());
       processMap.put("get_file_metadata_by_expr", new get_file_metadata_by_expr());
       processMap.put("get_file_metadata", new get_file_metadata());
@@ -29810,6 +29891,57 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class add_write_notification_log<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, add_write_notification_log_args, WriteNotificationLogResponse> {
+      public add_write_notification_log() {
+        super("add_write_notification_log");
+      }
+
+      public add_write_notification_log_args getEmptyArgsInstance() {
+        return new add_write_notification_log_args();
+      }
+
+      public AsyncMethodCallback<WriteNotificationLogResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<WriteNotificationLogResponse>() { 
+          public void onComplete(WriteNotificationLogResponse o) {
+            add_write_notification_log_result result = new add_write_notification_log_result();
+            result.success = o;
+            try {
+              fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
+              return;
+            } catch (Exception e) {
+              LOGGER.error("Exception writing to internal frame buffer", e);
+            }
+            fb.close();
+          }
+          public void onError(Exception e) {
+            byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
+            org.apache.thrift.TBase msg;
+            add_write_notification_log_result result = new add_write_notification_log_result();
+            {
+              msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
+              msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage());
+            }
+            try {
+              fcall.sendResponse(fb,msg,msgType,seqid);
+              return;
+            } catch (Exception ex) {
+              LOGGER.error("Exception writing to internal frame buffer", ex);
+            }
+            fb.close();
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public void start(I iface, add_write_notification_log_args args, org.apache.thrift.async.AsyncMethodCallback<WriteNotificationLogResponse> resultHandler) throws TException {
+        iface.add_write_notification_log(args.rqst,resultHandler);
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class cm_recycle<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, cm_recycle_args, CmRecycleResponse> {
       public cm_recycle() {
         super("cm_recycle");
@@ -42252,13 +42384,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list928.size);
-                  String _elem929;
-                  for (int _i930 = 0; _i930 < _list928.size; ++_i930)
+                  org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list952.size);
+                  String _elem953;
+                  for (int _i954 = 0; _i954 < _list952.size; ++_i954)
                   {
-                    _elem929 = iprot.readString();
-                    struct.success.add(_elem929);
+                    _elem953 = iprot.readString();
+                    struct.success.add(_elem953);
                   }
                   iprot.readListEnd();
                 }
@@ -42293,9 +42425,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter931 : struct.success)
+            for (String _iter955 : struct.success)
             {
-              oprot.writeString(_iter931);
+              oprot.writeString(_iter955);
             }
             oprot.writeListEnd();
           }
@@ -42334,9 +42466,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter932 : struct.success)
+            for (String _iter956 : struct.success)
             {
-              oprot.writeString(_iter932);
+              oprot.writeString(_iter956);
             }
           }
         }
@@ -42351,13 +42483,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list933.size);
-            String _elem934;
-            for (int _i935 = 0; _i935 < _list933.size; ++_i935)
+            org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list957.size);
+            String _elem958;
+            for (int _i959 = 0; _i959 < _list957.size; ++_i959)
             {
-              _elem934 = iprot.readString();
-              struct.success.add(_elem934);
+              _elem958 = iprot.readString();
+              struct.success.add(_elem958);
             }
           }
           struct.setSuccessIsSet(true);
@@ -43011,13 +43143,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list936.size);
-                  String _elem937;
-                  for (int _i938 = 0; _i938 < _list936.size; ++_i938)
+                  org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list960.size);
+                  String _elem961;
+                  for (int _i962 = 0; _i962 < _list960.size; ++_i962)
                   {
-                    _elem937 = iprot.readString();
-                    struct.success.add(_elem937);
+                    _elem961 = iprot.readString();
+                    struct.success.add(_elem961);
                   }
                   iprot.readListEnd();
                 }
@@ -43052,9 +43184,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter939 : struct.success)
+            for (String _iter963 : struct.success)
             {
-              oprot.writeString(_iter939);
+              oprot.writeString(_iter963);
             }
             oprot.writeListEnd();
           }
@@ -43093,9 +43225,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter940 : struct.success)
+            for (String _iter964 : struct.success)
             {
-              oprot.writeString(_iter940);
+              oprot.writeString(_iter964);
             }
           }
         }
@@ -43110,13 +43242,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list941.size);
-            String _elem942;
-            for (int _i943 = 0; _i943 < _list941.size; ++_i943)
+            org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list965.size);
+            String _elem966;
+            for (int _i967 = 0; _i967 < _list965.size; ++_i967)
             {
-              _elem942 = iprot.readString();
-              struct.success.add(_elem942);
+              _elem966 = iprot.readString();
+              struct.success.add(_elem966);
             }
           }
           struct.setSuccessIsSet(true);
@@ -47723,16 +47855,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map944 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Type>(2*_map944.size);
-                  String _key945;
-                  Type _val946;
-                  for (int _i947 = 0; _i947 < _map944.size; ++_i947)
+                  org.apache.thrift.protocol.TMap _map968 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Type>(2*_map968.size);
+                  String _key969;
+                  Type _val970;
+                  for (int _i971 = 0; _i971 < _map968.size; ++_i971)
                   {
-                    _key945 = iprot.readString();
-                    _val946 = new Type();
-                    _val946.read(iprot);
-                    struct.success.put(_key945, _val946);
+                    _key969 = iprot.readString();
+                    _val970 = new Type();
+                    _val970.read(iprot);
+                    struct.success.put(_key969, _val970);
                   }
                   iprot.readMapEnd();
                 }
@@ -47767,10 +47899,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Type> _iter948 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter972 : struct.success.entrySet())
             {
-              oprot.writeString(_iter948.getKey());
-              _iter948.getValue().write(oprot);
+              oprot.writeString(_iter972.getKey());
+              _iter972.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -47809,10 +47941,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Type> _iter949 : struct.success.entrySet())
+            for (Map.Entry<String, Type> _iter973 : struct.success.entrySet())
             {
-              oprot.writeString(_iter949.getKey());
-              _iter949.getValue().write(oprot);
+              oprot.writeString(_iter973.getKey());
+              _iter973.getValue().write(oprot);
             }
           }
         }
@@ -47827,16 +47959,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map950 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Type>(2*_map950.size);
-            String _key951;
-            Type _val952;
-            for (int _i953 = 0; _i953 < _map950.size; ++_i953)
+            org.apache.thrift.protocol.TMap _map974 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Type>(2*_map974.size);
+            String _key975;
+            Type _val976;
+            for (int _i977 = 0; _i977 < _map974.size; ++_i977)
             {
-              _key951 = iprot.readString();
-              _val952 = new Type();
-              _val952.read(iprot);
-              struct.success.put(_key951, _val952);
+              _key975 = iprot.readString();
+              _val976 = new Type();
+              _val976.read(iprot);
+              struct.success.put(_key975, _val976);
             }
           }
           struct.setSuccessIsSet(true);
@@ -48871,14 +49003,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list954 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list954.size);
-                  FieldSchema _elem955;
-                  for (int _i956 = 0; _i956 < _list954.size; ++_i956)
+                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list978.size);
+                  FieldSchema _elem979;
+                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
                   {
-                    _elem955 = new FieldSchema();
-                    _elem955.read(iprot);
-                    struct.success.add(_elem955);
+                    _elem979 = new FieldSchema();
+                    _elem979.read(iprot);
+                    struct.success.add(_elem979);
                   }
                   iprot.readListEnd();
                 }
@@ -48931,9 +49063,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter957 : struct.success)
+            for (FieldSchema _iter981 : struct.success)
             {
-              _iter957.write(oprot);
+              _iter981.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -48988,9 +49120,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter958 : struct.success)
+            for (FieldSchema _iter982 : struct.success)
             {
-              _iter958.write(oprot);
+              _iter982.write(oprot);
             }
           }
         }
@@ -49011,14 +49143,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list959 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list959.size);
-            FieldSchema _elem960;
-            for (int _i961 = 0; _i961 < _list959.size; ++_i961)
+            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list983.size);
+            FieldSchema _elem984;
+            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
             {
-              _elem960 = new FieldSchema();
-              _elem960.read(iprot);
-              struct.success.add(_elem960);
+              _elem984 = new FieldSchema();
+              _elem984.read(iprot);
+              struct.success.add(_elem984);
             }
           }
           struct.setSuccessIsSet(true);
@@ -50172,14 +50304,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list962 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list962.size);
-                  FieldSchema _elem963;
-                  for (int _i964 = 0; _i964 < _list962.size; ++_i964)
+                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list986.size);
+                  FieldSchema _elem987;
+                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
                   {
-                    _elem963 = new FieldSchema();
-                    _elem963.read(iprot);
-                    struct.success.add(_elem963);
+                    _elem987 = new FieldSchema();
+                    _elem987.read(iprot);
+                    struct.success.add(_elem987);
                   }
                   iprot.readListEnd();
                 }
@@ -50232,9 +50364,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter965 : struct.success)
+            for (FieldSchema _iter989 : struct.success)
             {
-              _iter965.write(oprot);
+              _iter989.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -50289,9 +50421,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter966 : struct.success)
+            for (FieldSchema _iter990 : struct.success)
             {
-              _iter966.write(oprot);
+              _iter990.write(oprot);
             }
           }
         }
@@ -50312,14 +50444,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list967 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list967.size);
-            FieldSchema _elem968;
-            for (int _i969 = 0; _i969 < _list967.size; ++_i969)
+            org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list991.size);
+            FieldSchema _elem992;
+            for (int _i993 = 0; _i993 < _list991.size; ++_i993)
             {
-              _elem968 = new FieldSchema();
-              _elem968.read(iprot);
-              struct.success.add(_elem968);
+              _elem992 = new FieldSchema();
+              _elem992.read(iprot);
+              struct.success.add(_elem992);
             }
           }
           struct.setSuccessIsSet(true);
@@ -51364,14 +51496,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list970 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list970.size);
-                  FieldSchema _elem971;
-                  for (int _i972 = 0; _i972 < _list970.size; ++_i972)
+                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list994.size);
+                  FieldSchema _elem995;
+                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
                   {
-                    _elem971 = new FieldSchema();
-                    _elem971.read(iprot);
-                    struct.success.add(_elem971);
+                    _elem995 = new FieldSchema();
+                    _elem995.read(iprot);
+                    struct.success.add(_elem995);
                   }
                   iprot.readListEnd();
                 }
@@ -51424,9 +51556,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter973 : struct.success)
+            for (FieldSchema _iter997 : struct.success)
             {
-              _iter973.write(oprot);
+              _iter997.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -51481,9 +51613,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter974 : struct.success)
+            for (FieldSchema _iter998 : struct.success)
             {
-              _iter974.write(oprot);
+              _iter998.write(oprot);
             }
           }
         }
@@ -51504,14 +51636,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list975 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list975.size);
-            FieldSchema _elem976;
-            for (int _i977 = 0; _i977 < _list975.size; ++_i977)
+            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list999.size);
+            FieldSchema _elem1000;
+            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
             {
-              _elem976 = new FieldSchema();
-              _elem976.read(iprot);
-              struct.success.add(_elem976);
+              _elem1000 = new FieldSchema();
+              _elem1000.read(iprot);
+              struct.success.add(_elem1000);
             }
           }
           struct.setSuccessIsSet(true);
@@ -52665,14 +52797,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
-                  struct.success = new ArrayList<FieldSchema>(_list978.size);
-                  FieldSchema _elem979;
-                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
+                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
+                  struct.success = new ArrayList<FieldSchema>(_list1002.size);
+                  FieldSchema _elem1003;
+                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
                   {
-                    _elem979 = new FieldSchema();
-                    _elem979.read(iprot);
-                    struct.success.add(_elem979);
+                    _elem1003 = new FieldSchema();
+                    _elem1003.read(iprot);
+                    struct.success.add(_elem1003);
                   }
                   iprot.readListEnd();
                 }
@@ -52725,9 +52857,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (FieldSchema _iter981 : struct.success)
+            for (FieldSchema _iter1005 : struct.success)
             {
-              _iter981.write(oprot);
+              _iter1005.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -52782,9 +52914,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (FieldSchema _iter982 : struct.success)
+            for (FieldSchema _iter1006 : struct.success)
             {
-              _iter982.write(oprot);
+              _iter1006.write(oprot);
             }
           }
         }
@@ -52805,14 +52937,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<FieldSchema>(_list983.size);
-            FieldSchema _elem984;
-            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
+            org.apache.thrift.protocol.TList _list1007 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<FieldSchema>(_list1007.size);
+            FieldSchema _elem1008;
+            for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009)
             {
-              _elem984 = new FieldSchema();
-              _elem984.read(iprot);
-              struct.success.add(_elem984);
+              _elem1008 = new FieldSchema();
+              _elem1008.read(iprot);
+              struct.success.add(_elem1008);
             }
           }
           struct.setSuccessIsSet(true);
@@ -55941,14 +56073,14 @@ import org.slf4j.LoggerFactory;
             case 2: // PRIMARY_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
-                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list986.size);
-                  SQLPrimaryKey _elem987;
-                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
+                  org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin();
+                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1010.size);
+                  SQLPrimaryKey _elem1011;
+                  for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012)
                   {
-                    _elem987 = new SQLPrimaryKey();
-                    _elem987.read(iprot);
-                    struct.primaryKeys.add(_elem987);
+                    _elem1011 = new SQLPrimaryKey();
+                    _elem1011.read(iprot);
+                    struct.primaryKeys.add(_elem1011);
                   }
                   iprot.readListEnd();
                 }
@@ -55960,14 +56092,14 @@ import org.slf4j.LoggerFactory;
             case 3: // FOREIGN_KEYS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list989 = iprot.readListBegin();
-                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list989.size);
-                  SQLForeignKey _elem990;
-                  for (int _i991 = 0; _i991 < _list989.size; ++_i991)
+                  org.apache.thrift.protocol.TList _list1013 = iprot.readListBegin();
+                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1013.size);
+                  SQLForeignKey _elem1014;
+                  for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
                   {
-                    _elem990 = new SQLForeignKey();
-                    _elem990.read(iprot);
-                    struct.foreignKeys.add(_elem990);
+                    _elem1014 = new SQLForeignKey();
+                    _elem1014.read(iprot);
+                    struct.foreignKeys.add(_elem1014);
                   }
                   iprot.readListEnd();
                 }
@@ -55979,14 +56111,14 @@ import org.slf4j.LoggerFactory;
             case 4: // UNIQUE_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list992 = iprot.readListBegin();
-                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list992.size);
-                  SQLUniqueConstraint _elem993;
-                  for (int _i994 = 0; _i994 < _list992.size; ++_i994)
+                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
+                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1016.size);
+                  SQLUniqueConstraint _elem1017;
+                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
                   {
-                    _elem993 = new SQLUniqueConstraint();
-                    _elem993.read(iprot);
-                    struct.uniqueConstraints.add(_elem993);
+                    _elem1017 = new SQLUniqueConstraint();
+                    _elem1017.read(iprot);
+                    struct.uniqueConstraints.add(_elem1017);
                   }
                   iprot.readListEnd();
                 }
@@ -55998,14 +56130,14 @@ import org.slf4j.LoggerFactory;
             case 5: // NOT_NULL_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list995 = iprot.readListBegin();
-                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list995.size);
-                  SQLNotNullConstraint _elem996;
-                  for (int _i997 = 0; _i997 < _list995.size; ++_i997)
+                  org.apache.thrift.protocol.TList _list1019 = iprot.readListBegin();
+                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1019.size);
+                  SQLNotNullConstraint _elem1020;
+                  for (int _i1021 = 0; _i1021 < _list1019.size; ++_i1021)
                   {
-                    _elem996 = new SQLNotNullConstraint();
-                    _elem996.read(iprot);
-                    struct.notNullConstraints.add(_elem996);
+                    _elem1020 = new SQLNotNullConstraint();
+                    _elem1020.read(iprot);
+                    struct.notNullConstraints.add(_elem1020);
                   }
                   iprot.readListEnd();
                 }
@@ -56017,14 +56149,14 @@ import org.slf4j.LoggerFactory;
             case 6: // DEFAULT_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list998 = iprot.readListBegin();
-                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list998.size);
-                  SQLDefaultConstraint _elem999;
-                  for (int _i1000 = 0; _i1000 < _list998.size; ++_i1000)
+                  org.apache.thrift.protocol.TList _list1022 = iprot.readListBegin();
+                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1022.size);
+                  SQLDefaultConstraint _elem1023;
+                  for (int _i1024 = 0; _i1024 < _list1022.size; ++_i1024)
                   {
-                    _elem999 = new SQLDefaultConstraint();
-                    _elem999.read(iprot);
-                    struct.defaultConstraints.add(_elem999);
+                    _elem1023 = new SQLDefaultConstraint();
+                    _elem1023.read(iprot);
+                    struct.defaultConstraints.add(_elem1023);
                   }
                   iprot.readListEnd();
                 }
@@ -56036,14 +56168,14 @@ import org.slf4j.LoggerFactory;
             case 7: // CHECK_CONSTRAINTS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1001 = iprot.readListBegin();
-                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1001.size);
-                  SQLCheckConstraint _elem1002;
-                  for (int _i1003 = 0; _i1003 < _list1001.size; ++_i1003)
+                  org.apache.thrift.protocol.TList _list1025 = iprot.readListBegin();
+                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1025.size);
+                  SQLCheckConstraint _elem1026;
+                  for (int _i1027 = 0; _i1027 < _list1025.size; ++_i1027)
                   {
-                    _elem1002 = new SQLCheckConstraint();
-                    _elem1002.read(iprot);
-                    struct.checkConstraints.add(_elem1002);
+                    _elem1026 = new SQLCheckConstraint();
+                    _elem1026.read(iprot);
+                    struct.checkConstraints.add(_elem1026);
                   }
                   iprot.readListEnd();
                 }
@@ -56074,9 +56206,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-            for (SQLPrimaryKey _iter1004 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1028 : struct.primaryKeys)
             {
-              _iter1004.write(oprot);
+              _iter1028.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56086,9 +56218,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-            for (SQLForeignKey _iter1005 : struct.foreignKeys)
+            for (SQLForeignKey _iter1029 : struct.foreignKeys)
             {
-              _iter1005.write(oprot);
+              _iter1029.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56098,9 +56230,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-            for (SQLUniqueConstraint _iter1006 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1030 : struct.uniqueConstraints)
             {
-              _iter1006.write(oprot);
+              _iter1030.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56110,9 +56242,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-            for (SQLNotNullConstraint _iter1007 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1031 : struct.notNullConstraints)
             {
-              _iter1007.write(oprot);
+              _iter1031.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56122,9 +56254,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-            for (SQLDefaultConstraint _iter1008 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1032 : struct.defaultConstraints)
             {
-              _iter1008.write(oprot);
+              _iter1032.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56134,9 +56266,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-            for (SQLCheckConstraint _iter1009 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1033 : struct.checkConstraints)
             {
-              _iter1009.write(oprot);
+              _iter1033.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -56188,54 +56320,54 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPrimaryKeys()) {
           {
             oprot.writeI32(struct.primaryKeys.size());
-            for (SQLPrimaryKey _iter1010 : struct.primaryKeys)
+            for (SQLPrimaryKey _iter1034 : struct.primaryKeys)
             {
-              _iter1010.write(oprot);
+              _iter1034.write(oprot);
             }
           }
         }
         if (struct.isSetForeignKeys()) {
           {
             oprot.writeI32(struct.foreignKeys.size());
-            for (SQLForeignKey _iter1011 : struct.foreignKeys)
+            for (SQLForeignKey _iter1035 : struct.foreignKeys)
             {
-              _iter1011.write(oprot);
+              _iter1035.write(oprot);
             }
           }
         }
         if (struct.isSetUniqueConstraints()) {
           {
             oprot.writeI32(struct.uniqueConstraints.size());
-            for (SQLUniqueConstraint _iter1012 : struct.uniqueConstraints)
+            for (SQLUniqueConstraint _iter1036 : struct.uniqueConstraints)
             {
-              _iter1012.write(oprot);
+              _iter1036.write(oprot);
             }
           }
         }
         if (struct.isSetNotNullConstraints()) {
           {
             oprot.writeI32(struct.notNullConstraints.size());
-            for (SQLNotNullConstraint _iter1013 : struct.notNullConstraints)
+            for (SQLNotNullConstraint _iter1037 : struct.notNullConstraints)
             {
-              _iter1013.write(oprot);
+              _iter1037.write(oprot);
             }
           }
         }
         if (struct.isSetDefaultConstraints()) {
           {
             oprot.writeI32(struct.defaultConstraints.size());
-            for (SQLDefaultConstraint _iter1014 : struct.defaultConstraints)
+            for (SQLDefaultConstraint _iter1038 : struct.defaultConstraints)
             {
-              _iter1014.write(oprot);
+              _iter1038.write(oprot);
             }
           }
         }
         if (struct.isSetCheckConstraints()) {
           {
             oprot.writeI32(struct.checkConstraints.size());
-            for (SQLCheckConstraint _iter1015 : struct.checkConstraints)
+            for (SQLCheckConstraint _iter1039 : struct.checkConstraints)
             {
-              _iter1015.write(oprot);
+              _iter1039.write(oprot);
             }
           }
         }
@@ -56252,84 +56384,84 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1016 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1016.size);
-            SQLPrimaryKey _elem1017;
-            for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
+            org.apache.thrift.protocol.TList _list1040 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1040.size);
+            SQLPrimaryKey _elem1041;
+            for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
             {
-              _elem1017 = new SQLPrimaryKey();
-              _elem1017.read(iprot);
-              struct.primaryKeys.add(_elem1017);
+              _elem1041 = new SQLPrimaryKey();
+              _elem1041.read(iprot);
+              struct.primaryKeys.add(_elem1041);
             }
           }
           struct.setPrimaryKeysIsSet(true);
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1019 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1019.size);
-            SQLForeignKey _elem1020;
-            for (int _i1021 = 0; _i1021 < _list1019.size; ++_i1021)
+            org.apache.thrift.protocol.TList _list1043 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1043.size);
+            SQLForeignKey _elem1044;
+            for (int _i1045 = 0; _i1045 < _list1043.size; ++_i1045)
             {
-              _elem1020 = new SQLForeignKey();
-              _elem1020.read(iprot);
-              struct.foreignKeys.add(_elem1020);
+              _elem1044 = new SQLForeignKey();
+              _elem1044.read(iprot);
+              struct.foreignKeys.add(_elem1044);
             }
           }
           struct.setForeignKeysIsSet(true);
         }
         if (incoming.get(3)) {
           {
-            org.apache.thrift.protocol.TList _list1022 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1022.size);
-            SQLUniqueConstraint _elem1023;
-            for (int _i1024 = 0; _i1024 < _list1022.size; ++_i1024)
+            org.apache.thrift.protocol.TList _list1046 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1046.size);
+            SQLUniqueConstraint _elem1047;
+            for (int _i1048 = 0; _i1048 < _list1046.size; ++_i1048)
             {
-              _elem1023 = new SQLUniqueConstraint();
-              _elem1023.read(iprot);
-              struct.uniqueConstraints.add(_elem1023);
+              _elem1047 = new SQLUniqueConstraint();
+              _elem1047.read(iprot);
+              struct.uniqueConstraints.add(_elem1047);
             }
           }
           struct.setUniqueConstraintsIsSet(true);
         }
         if (incoming.get(4)) {
           {
-            org.apache.thrift.protocol.TList _list1025 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1025.size);
-            SQLNotNullConstraint _elem1026;
-            for (int _i1027 = 0; _i1027 < _list1025.size; ++_i1027)
+            org.apache.thrift.protocol.TList _list1049 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1049.size);
+            SQLNotNullConstraint _elem1050;
+            for (int _i1051 = 0; _i1051 < _list1049.size; ++_i1051)
             {
-              _elem1026 = new SQLNotNullConstraint();
-              _elem1026.read(iprot);
-              struct.notNullConstraints.add(_elem1026);
+              _elem1050 = new SQLNotNullConstraint();
+              _elem1050.read(iprot);
+              struct.notNullConstraints.add(_elem1050);
             }
           }
           struct.setNotNullConstraintsIsSet(true);
         }
         if (incoming.get(5)) {
           {
-            org.apache.thrift.protocol.TList _list1028 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1028.size);
-            SQLDefaultConstraint _elem1029;
-            for (int _i1030 = 0; _i1030 < _list1028.size; ++_i1030)
+            org.apache.thrift.protocol.TList _list1052 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1052.size);
+            SQLDefaultConstraint _elem1053;
+            for (int _i1054 = 0; _i1054 < _list1052.size; ++_i1054)
             {
-              _elem1029 = new SQLDefaultConstraint();
-              _elem1029.read(iprot);
-              struct.defaultConstraints.add(_elem1029);
+              _elem1053 = new SQLDefaultConstraint();
+              _elem1053.read(iprot);
+              struct.defaultConstraints.add(_elem1053);
             }
           }
           struct.setDefaultConstraintsIsSet(true);
         }
         if (incoming.get(6)) {
           {
-            org.apache.thrift.protocol.TList _list1031 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1031.size);
-            SQLCheckConstraint _elem1032;
-            for (int _i1033 = 0; _i1033 < _list1031.size; ++_i1033)
+            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1055.size);
+            SQLCheckConstraint _elem1056;
+            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
             {
-              _elem1032 = new SQLCheckConstraint();
-              _elem1032.read(iprot);
-              struct.checkConstraints.add(_elem1032);
+              _elem1056 = new SQLCheckConstraint();
+              _elem1056.read(iprot);
+              struct.checkConstraints.add(_elem1056);
             }
           }
           struct.setCheckConstraintsIsSet(true);
@@ -65479,13 +65611,13 @@ import org.slf4j.LoggerFactory;
             case 3: // PART_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1034 = iprot.readListBegin();
-                  struct.partNames = new ArrayList<String>(_list1034.size);
-                  String _elem1035;
-                  for (int _i1036 = 0; _i1036 < _list1034.size; ++_i1036)
+                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
+                  struct.partNames = new ArrayList<String>(_list1058.size);
+                  String _elem1059;
+                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
                   {
-                    _elem1035 = iprot.readString();
-                    struct.partNames.add(_elem1035);
+                    _elem1059 = iprot.readString();
+                    struct.partNames.add(_elem1059);
                   }
                   iprot.readListEnd();
                 }
@@ -65521,9 +65653,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter1037 : struct.partNames)
+            for (String _iter1061 : struct.partNames)
             {
-              oprot.writeString(_iter1037);
+              oprot.writeString(_iter1061);
             }
             oprot.writeListEnd();
           }
@@ -65566,9 +65698,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetPartNames()) {
           {
             oprot.writeI32(struct.partNames.size());
-            for (String _iter1038 : struct.partNames)
+            for (String _iter1062 : struct.partNames)
             {
-              oprot.writeString(_iter1038);
+              oprot.writeString(_iter1062);
             }
           }
         }
@@ -65588,13 +65720,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1039 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.partNames = new ArrayList<String>(_list1039.size);
-            String _elem1040;
-            for (int _i1041 = 0; _i1041 < _list1039.size; ++_i1041)
+            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.partNames = new ArrayList<String>(_list1063.size);
+            String _elem1064;
+            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
             {
-              _elem1040 = iprot.readString();
-              struct.partNames.add(_elem1040);
+              _elem1064 = iprot.readString();
+              struct.partNames.add(_elem1064);
             }
           }
           struct.setPartNamesIsSet(true);
@@ -66819,13 +66951,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1042.size);
-                  String _elem1043;
-                  for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
+                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1066.size);
+                  String _elem1067;
+                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
                   {
-                    _elem1043 = iprot.readString();
-                    struct.success.add(_elem1043);
+                    _elem1067 = iprot.readString();
+                    struct.success.add(_elem1067);
                   }
                   iprot.readListEnd();
                 }
@@ -66860,9 +66992,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1045 : struct.success)
+            for (String _iter1069 : struct.success)
             {
-              oprot.writeString(_iter1045);
+              oprot.writeString(_iter1069);
             }
             oprot.writeListEnd();
           }
@@ -66901,9 +67033,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1046 : struct.success)
+            for (String _iter1070 : struct.success)
             {
-              oprot.writeString(_iter1046);
+              oprot.writeString(_iter1070);
             }
           }
         }
@@ -66918,13 +67050,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1047.size);
-            String _elem1048;
-            for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
+            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1071.size);
+            String _elem1072;
+            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
             {
-              _elem1048 = iprot.readString();
-              struct.success.add(_elem1048);
+              _elem1072 = iprot.readString();
+              struct.success.add(_elem1072);
             }
           }
           struct.setSuccessIsSet(true);
@@ -67898,13 +68030,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1050.size);
-                  String _elem1051;
-                  for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
+                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1074.size);
+                  String _elem1075;
+                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
                   {
-                    _elem1051 = iprot.readString();
-                    struct.success.add(_elem1051);
+                    _elem1075 = iprot.readString();
+                    struct.success.add(_elem1075);
                   }
                   iprot.readListEnd();
                 }
@@ -67939,9 +68071,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1053 : struct.success)
+            for (String _iter1077 : struct.success)
             {
-              oprot.writeString(_iter1053);
+              oprot.writeString(_iter1077);
             }
             oprot.writeListEnd();
           }
@@ -67980,9 +68112,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1054 : struct.success)
+            for (String _iter1078 : struct.success)
             {
-              oprot.writeString(_iter1054);
+              oprot.writeString(_iter1078);
             }
           }
         }
@@ -67997,13 +68129,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1055.size);
-            String _elem1056;
-            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
+            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1079.size);
+            String _elem1080;
+            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
             {
-              _elem1056 = iprot.readString();
-              struct.success.add(_elem1056);
+              _elem1080 = iprot.readString();
+              struct.success.add(_elem1080);
             }
           }
           struct.setSuccessIsSet(true);
@@ -68769,13 +68901,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1058.size);
-                  String _elem1059;
-                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
+                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1082.size);
+                  String _elem1083;
+                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
                   {
-                    _elem1059 = iprot.readString();
-                    struct.success.add(_elem1059);
+                    _elem1083 = iprot.readString();
+                    struct.success.add(_elem1083);
                   }
                   iprot.readListEnd();
                 }
@@ -68810,9 +68942,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1061 : struct.success)
+            for (String _iter1085 : struct.success)
             {
-              oprot.writeString(_iter1061);
+              oprot.writeString(_iter1085);
             }
             oprot.writeListEnd();
           }
@@ -68851,9 +68983,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1062 : struct.success)
+            for (String _iter1086 : struct.success)
             {
-              oprot.writeString(_iter1062);
+              oprot.writeString(_iter1086);
             }
           }
         }
@@ -68868,13 +69000,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1063.size);
-            String _elem1064;
-            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
+            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1087.size);
+            String _elem1088;
+            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
             {
-              _elem1064 = iprot.readString();
-              struct.success.add(_elem1064);
+              _elem1088 = iprot.readString();
+              struct.success.add(_elem1088);
             }
           }
           struct.setSuccessIsSet(true);
@@ -69379,13 +69511,13 @@ import org.slf4j.LoggerFactory;
             case 3: // TBL_TYPES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
-                  struct.tbl_types = new ArrayList<String>(_list1066.size);
-                  String _elem1067;
-                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
+                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
+                  struct.tbl_types = new ArrayList<String>(_list1090.size);
+                  String _elem1091;
+                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
                   {
-                    _elem1067 = iprot.readString();
-                    struct.tbl_types.add(_elem1067);
+                    _elem1091 = iprot.readString();
+                    struct.tbl_types.add(_elem1091);
                   }
                   iprot.readListEnd();
                 }
@@ -69421,9 +69553,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-            for (String _iter1069 : struct.tbl_types)
+            for (String _iter1093 : struct.tbl_types)
             {
-              oprot.writeString(_iter1069);
+              oprot.writeString(_iter1093);
             }
             oprot.writeListEnd();
           }
@@ -69466,9 +69598,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_types()) {
           {
             oprot.writeI32(struct.tbl_types.size());
-            for (String _iter1070 : struct.tbl_types)
+            for (String _iter1094 : struct.tbl_types)
             {
-              oprot.writeString(_iter1070);
+              oprot.writeString(_iter1094);
             }
           }
         }
@@ -69488,13 +69620,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(2)) {
           {
-            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_types = new ArrayList<String>(_list1071.size);
-            String _elem1072;
-            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
+            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_types = new ArrayList<String>(_list1095.size);
+            String _elem1096;
+            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
             {
-              _elem1072 = iprot.readString();
-              struct.tbl_types.add(_elem1072);
+              _elem1096 = iprot.readString();
+              struct.tbl_types.add(_elem1096);
             }
           }
           struct.setTbl_typesIsSet(true);
@@ -69900,14 +70032,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                  struct.success = new ArrayList<TableMeta>(_list1074.size);
-                  TableMeta _elem1075;
-                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
+                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
+                  struct.success = new ArrayList<TableMeta>(_list1098.size);
+                  TableMeta _elem1099;
+                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
                   {
-                    _elem1075 = new TableMeta();
-                    _elem1075.read(iprot);
-                    struct.success.add(_elem1075);
+                    _elem1099 = new TableMeta();
+                    _elem1099.read(iprot);
+                    struct.success.add(_elem1099);
                   }
                   iprot.readListEnd();
                 }
@@ -69942,9 +70074,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (TableMeta _iter1077 : struct.success)
+            for (TableMeta _iter1101 : struct.success)
             {
-              _iter1077.write(oprot);
+              _iter1101.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -69983,9 +70115,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (TableMeta _iter1078 : struct.success)
+            for (TableMeta _iter1102 : struct.success)
             {
-              _iter1078.write(oprot);
+              _iter1102.write(oprot);
             }
           }
         }
@@ -70000,14 +70132,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<TableMeta>(_list1079.size);
-            TableMeta _elem1080;
-            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
+            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<TableMeta>(_list1103.size);
+            TableMeta _elem1104;
+            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
             {
-              _elem1080 = new TableMeta();
-              _elem1080.read(iprot);
-              struct.success.add(_elem1080);
+              _elem1104 = new TableMeta();
+              _elem1104.read(iprot);
+              struct.success.add(_elem1104);
             }
           }
           struct.setSuccessIsSet(true);
@@ -70773,13 +70905,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1082.size);
-                  String _elem1083;
-                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
+                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1106.size);
+                  String _elem1107;
+                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
                   {
-                    _elem1083 = iprot.readString();
-                    struct.success.add(_elem1083);
+                    _elem1107 = iprot.readString();
+                    struct.success.add(_elem1107);
                   }
                   iprot.readListEnd();
                 }
@@ -70814,9 +70946,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1085 : struct.success)
+            for (String _iter1109 : struct.success)
             {
-              oprot.writeString(_iter1085);
+              oprot.writeString(_iter1109);
             }
             oprot.writeListEnd();
           }
@@ -70855,9 +70987,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter1086 : struct.success)
+            for (String _iter1110 : struct.success)
             {
-              oprot.writeString(_iter1086);
+              oprot.writeString(_iter1110);
             }
           }
         }
@@ -70872,13 +71004,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1087.size);
-            String _elem1088;
-            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
+            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1111.size);
+            String _elem1112;
+            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
             {
-              _elem1088 = iprot.readString();
-              struct.success.add(_elem1088);
+              _elem1112 = iprot.readString();
+              struct.success.add(_elem1112);
             }
           }
           struct.setSuccessIsSet(true);
@@ -72331,13 +72463,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1090.size);
-                  String _elem1091;
-                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
+                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1114.size);
+                  String _elem1115;
+                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
                   {
-                    _elem1091 = iprot.readString();
-                    struct.tbl_names.add(_elem1091);
+                    _elem1115 = iprot.readString();
+                    struct.tbl_names.add(_elem1115);
                   }
                   iprot.readListEnd();
                 }
@@ -72368,9 +72500,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1093 : struct.tbl_names)
+            for (String _iter1117 : struct.tbl_names)
             {
-              oprot.writeString(_iter1093);
+              oprot.writeString(_iter1117);
             }
             oprot.writeListEnd();
           }
@@ -72407,9 +72539,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1094 : struct.tbl_names)
+            for (String _iter1118 : struct.tbl_names)
             {
-              oprot.writeString(_iter1094);
+              oprot.writeString(_iter1118);
             }
           }
         }
@@ -72425,13 +72557,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1095.size);
-            String _elem1096;
-            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
+            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1119.size);
+            String _elem1120;
+            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
             {
-              _elem1096 = iprot.readString();
-              struct.tbl_names.add(_elem1096);
+              _elem1120 = iprot.readString();
+              struct.tbl_names.add(_elem1120);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -72756,14 +72888,14 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
-                  struct.success = new ArrayList<Table>(_list1098.size);
-                  Table _elem1099;
-                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
+                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
+                  struct.success = new ArrayList<Table>(_list1122.size);
+                  Table _elem1123;
+                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
                   {
-                    _elem1099 = new Table();
-                    _elem1099.read(iprot);
-                    struct.success.add(_elem1099);
+                    _elem1123 = new Table();
+                    _elem1123.read(iprot);
+                    struct.success.add(_elem1123);
                   }
                   iprot.readListEnd();
                 }
@@ -72789,9 +72921,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Table _iter1101 : struct.success)
+            for (Table _iter1125 : struct.success)
             {
-              _iter1101.write(oprot);
+              _iter1125.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -72822,9 +72954,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Table _iter1102 : struct.success)
+            for (Table _iter1126 : struct.success)
             {
-              _iter1102.write(oprot);
+              _iter1126.write(oprot);
             }
           }
         }
@@ -72836,14 +72968,14 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new ArrayList<Table>(_list1103.size);
-            Table _elem1104;
-            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
+            org.apache.thrift.protocol.TList _list1127 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new ArrayList<Table>(_list1127.size);
+            Table _elem1128;
+            for (int _i1129 = 0; _i1129 < _list1127.size; ++_i1129)
             {
-              _elem1104 = new Table();
-              _elem1104.read(iprot);
-              struct.success.add(_elem1104);
+              _elem1128 = new Table();
+              _elem1128.read(iprot);
+              struct.success.add(_elem1128);
             }
           }
           struct.setSuccessIsSet(true);
@@ -75236,13 +75368,13 @@ import org.slf4j.LoggerFactory;
             case 2: // TBL_NAMES
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
-                  struct.tbl_names = new ArrayList<String>(_list1106.size);
-                  String _elem1107;
-                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
+                  org.apache.thrift.protocol.TList _list1130 = iprot.readListBegin();
+                  struct.tbl_names = new ArrayList<String>(_list1130.size);
+                  String _elem1131;
+                  for (int _i1132 = 0; _i1132 < _list1130.size; ++_i1132)
                   {
-                    _elem1107 = iprot.readString();
-                    struct.tbl_names.add(_elem1107);
+                    _elem1131 = iprot.readString();
+                    struct.tbl_names.add(_elem1131);
                   }
                   iprot.readListEnd();
                 }
@@ -75273,9 +75405,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-            for (String _iter1109 : struct.tbl_names)
+            for (String _iter1133 : struct.tbl_names)
             {
-              oprot.writeString(_iter1109);
+              oprot.writeString(_iter1133);
             }
             oprot.writeListEnd();
           }
@@ -75312,9 +75444,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetTbl_names()) {
           {
             oprot.writeI32(struct.tbl_names.size());
-            for (String _iter1110 : struct.tbl_names)
+            for (String _iter1134 : struct.tbl_names)
             {
-              oprot.writeString(_iter1110);
+              oprot.writeString(_iter1134);
             }
           }
         }
@@ -75330,13 +75462,13 @@ import org.slf4j.LoggerFactory;
         }
         if (incoming.get(1)) {
           {
-            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.tbl_names = new ArrayList<String>(_list1111.size);
-            String _elem1112;
-            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
+            org.apache.thrift.protocol.TList _list1135 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.tbl_names = new ArrayList<String>(_list1135.size);
+            String _elem1136;
+            for (int _i1137 = 0; _i1137 < _list1135.size; ++_i1137)
             {
-              _elem1112 = iprot.readString();
-              struct.tbl_names.add(_elem1112);
+              _elem1136 = iprot.readString();
+              struct.tbl_names.add(_elem1136);
             }
           }
           struct.setTbl_namesIsSet(true);
@@ -75909,16 +76041,16 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                 {
-                  org.apache.thrift.protocol.TMap _map1114 = iprot.readMapBegin();
-                  struct.success = new HashMap<String,Materialization>(2*_map1114.size);
-                  String _key1115;
-                  Materialization _val1116;
-                  for (int _i1117 = 0; _i1117 < _map1114.size; ++_i1117)
+                  org.apache.thrift.protocol.TMap _map1138 = iprot.readMapBegin();
+                  struct.success = new HashMap<String,Materialization>(2*_map1138.size);
+                  String _key1139;
+                  Materialization _val1140;
+                  for (int _i1141 = 0; _i1141 < _map1138.size; ++_i1141)
                   {
-                    _key1115 = iprot.readString();
-                    _val1116 = new Materialization();
-                    _val1116.read(iprot);
-                    struct.success.put(_key1115, _val1116);
+                    _key1139 = iprot.readString();
+                    _val1140 = new Materialization();
+                    _val1140.read(iprot);
+                    struct.success.put(_key1139, _val1140);
                   }
                   iprot.readMapEnd();
                 }
@@ -75971,10 +76103,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-            for (Map.Entry<String, Materialization> _iter1118 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1142 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1118.getKey());
-              _iter1118.getValue().write(oprot);
+              oprot.writeString(_iter1142.getKey());
+              _iter1142.getValue().write(oprot);
             }
             oprot.writeMapEnd();
           }
@@ -76029,10 +76161,10 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (Map.Entry<String, Materialization> _iter1119 : struct.success.entrySet())
+            for (Map.Entry<String, Materialization> _iter1143 : struct.success.entrySet())
             {
-              oprot.writeString(_iter1119.getKey());
-              _iter1119.getValue().write(oprot);
+              oprot.writeString(_iter1143.getKey());
+              _iter1143.getValue().write(oprot);
             }
           }
         }
@@ -76053,16 +76185,16 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(4);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TMap _map1120 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-            struct.success = new HashMap<String,Materialization>(2*_map1120.size);
-            String _key1121;
-            Materialization _val1122;
-            for (int _i1123 = 0; _i1123 < _map1120.size; ++_i1123)
+            org.apache.thrift.protocol.TMap _map1144 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+            struct.success = new HashMap<String,Materialization>(2*_map1144.size);
+            String _key1145;
+            Materialization _val1146;
+            for (int _i1147 = 0; _i1147 < _map1144.size; ++_i1147)
             {
-              _key1121 = iprot.readString();
-              _val1122 = new Materialization();
-              _val1122.read(iprot);
-              struct.success.put(_key1121, _val1122);
+              _key1145 = iprot.readString();
+              _val1146 = new Materialization();
+              _val1146.read(iprot);
+              struct.success.put(_key1145, _val1146);
             }
           }
           struct.setSuccessIsSet(true);
@@ -78455,13 +78587,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list1124 = iprot.readListBegin();
-

<TRUNCATED>

[17/46] hive git commit: HIVE-20051 : Skip authorization for temp tables (Zoltan Haindrich via Jason Dere)

Posted by se...@apache.org.
HIVE-20051 : Skip authorization for temp tables (Zoltan Haindrich via Jason Dere)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/master-txnstats
Commit: 285a9b4dc6e8e126864310d0c80426ebc5025cdf
Parents: f012691
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Mon Jul 2 23:39:05 2018 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Mon Jul 2 23:39:05 2018 -0700

----------------------------------------------------------------------
 .../TestHiveAuthorizerCheckInvocation.java      | 94 +++++++++++++++++++-
 .../java/org/apache/hadoop/hive/ql/Driver.java  | 10 ++-
 .../apache/hadoop/hive/ql/metadata/Table.java   |  7 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 13 +--
 4 files changed, 113 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
index e3c83d2..b9ef8b7 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.security.authorization.plugin;
 
+import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -27,6 +28,7 @@ import static org.mockito.Mockito.reset;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.File;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -53,7 +55,6 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import static org.apache.hadoop.hive.metastore.ReplChangeManager.SOURCE_OF_REPLICATION;
 
 /**
  * Test HiveAuthorizer api invocation
@@ -386,6 +387,97 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
+  public void testTempTable() throws Exception {
+
+    String tmpTableDir = getDefaultTmp() + File.separator + "THSAC_testTableTable";
+
+    final String tableName = "testTempTable";
+    { // create temp table
+      reset(mockedAuthorizer);
+      int status = driver.run("create temporary table " + tableName + "(i int) location '" + tmpTableDir + "'")
+          .getResponseCode();
+      assertEquals(0, status);
+
+      List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+      List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+
+      // only the URI should be passed for authorization check
+      assertEquals("input count", 1, inputs.size());
+      assertEquals("input type", HivePrivilegeObjectType.LOCAL_URI, inputs.get(0).getType());
+
+      // only the dbname should be passed authorization check
+      assertEquals("output count", 1, outputs.size());
+      assertEquals("output type", HivePrivilegeObjectType.DATABASE, outputs.get(0).getType());
+
+      status = driver.compile("select * from " + tableName);
+      assertEquals(0, status);
+    }
+    { // select from the temp table
+      reset(mockedAuthorizer);
+      int status = driver.compile("insert into " + tableName + " values(1)");
+      assertEquals(0, status);
+
+      // temp tables should be skipped from authorization
+      List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+      List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+      System.err.println("inputs " + inputs);
+      System.err.println("outputs " + outputs);
+
+      assertEquals("input count", 0, inputs.size());
+      assertEquals("output count", 0, outputs.size());
+    }
+    { // select from the temp table
+      reset(mockedAuthorizer);
+      int status = driver.compile("select * from " + tableName);
+      assertEquals(0, status);
+
+      // temp tables should be skipped from authorization
+      List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+      List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+      System.err.println("inputs " + inputs);
+      System.err.println("outputs " + outputs);
+
+      assertEquals("input count", 0, inputs.size());
+      assertEquals("output count", 0, outputs.size());
+    }
+
+  }
+
+  @Test
+  public void testTempTableImplicit() throws Exception {
+    final String tableName = "testTempTableImplicit";
+    int status = driver.run("create table " + tableName + "(i int)").getResponseCode();
+    assertEquals(0, status);
+
+    reset(mockedAuthorizer);
+    status = driver.compile("insert into " + tableName + " values (1)");
+    assertEquals(0, status);
+
+    List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
+    List<HivePrivilegeObject> outputs = getHivePrivilegeObjectInputs().getRight();
+
+    // only the URI should be passed for authorization check
+    assertEquals("input count", 0, inputs.size());
+
+    reset(mockedAuthorizer);
+    status = driver.compile("select * from " + tableName);
+    assertEquals(0, status);
+
+    inputs = getHivePrivilegeObjectInputs().getLeft();
+    outputs = getHivePrivilegeObjectInputs().getRight();
+
+    // temp tables should be skipped from authorization
+    assertEquals("input count", 1, inputs.size());
+    assertEquals("output count", 0, outputs.size());
+
+  }
+
+  private String getDefaultTmp() {
+    return System.getProperty("test.tmp.dir",
+        "target" + File.separator + "test" + File.separator + "tmp");
+  }
+
+  @Test
   public void testUpdateSomeColumnsUsed() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("update " + acidTableName + " set i = 5 where j = 3");

http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/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 762e57c..bf4d29c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -41,8 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.stream.Collectors;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
@@ -77,8 +75,8 @@ import org.apache.hadoop.hive.ql.exec.DagUtils;
 import org.apache.hadoop.hive.ql.exec.ExplainTask;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
-import org.apache.hadoop.hive.ql.exec.FunctionUtils;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionType;
+import org.apache.hadoop.hive.ql.exec.FunctionUtils;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -150,6 +148,7 @@ import org.apache.hive.common.util.TxnIdUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
@@ -1334,6 +1333,11 @@ public class Driver implements IDriver {
         //do not authorize temporary uris
         continue;
       }
+      if (privObject.getTyp() == Type.TABLE
+          && (privObject.getT() == null || privObject.getT().isTemporary())) {
+        // skip temporary tables from authorization
+        continue;
+      }
       //support for authorization on partitions needs to be added
       String dbname = null;
       String objName = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/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 f0061c0..14e60f0 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
@@ -30,7 +30,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -75,6 +74,8 @@ import org.apache.hive.common.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * A Hive Table: is a fundamental unit of data in Hive that shares a common schema/DDL.
  *
@@ -1010,6 +1011,10 @@ public class Table implements Serializable {
     return tTable.isTemporary();
   }
 
+  public void setTemporary(boolean isTemporary) {
+    tTable.setTemporary(isTemporary);
+  }
+
   public static boolean hasMetastoreBasedSchema(HiveConf conf, String serdeLib) {
     return StringUtils.isEmpty(serdeLib) ||
         conf.getStringCollection(ConfVars.SERDESUSINGMETASTOREFORSCHEMA.varname).contains(serdeLib);

http://git-wip-us.apache.org/repos/asf/hive/blob/285a9b4d/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 2731f19..576f337 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
@@ -13176,7 +13176,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     case CREATE_TABLE: // REGULAR CREATE TABLE DDL
       tblProps = addDefaultProperties(
           tblProps, isExt, storageFormat, dbDotTab, sortCols, isMaterialization, isTemporary);
-      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, tblProps);
+      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, isTemporary, tblProps);
 
       CreateTableDesc crtTblDesc = new CreateTableDesc(dbDotTab, isExt, isTemporary, cols, partCols,
           bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
@@ -13200,7 +13200,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     case CTLT: // create table like <tbl_name>
       tblProps = addDefaultProperties(
           tblProps, isExt, storageFormat, dbDotTab, sortCols, isMaterialization, isTemporary);
-      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, tblProps);
+      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, isTemporary, tblProps);
 
       if (isTemporary) {
         Table likeTable = getTable(likeTableName, false);
@@ -13280,7 +13280,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
       tblProps = addDefaultProperties(
           tblProps, isExt, storageFormat, dbDotTab, sortCols, isMaterialization, isTemporary);
-      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, tblProps);
+      addDbAndTabToOutputs(qualifiedTabName, TableType.MANAGED_TABLE, isTemporary, tblProps);
       tableDesc = new CreateTableDesc(qualifiedTabName[0], dbDotTab, isExt, isTemporary, cols,
           partCols, bucketCols, sortCols, numBuckets, rowFormatParams.fieldDelim,
           rowFormatParams.fieldEscape, rowFormatParams.collItemDelim, rowFormatParams.mapKeyDelim,
@@ -13304,13 +13304,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
   /** Adds entities for create table/create view. */
   private void addDbAndTabToOutputs(String[] qualifiedTabName, TableType type,
-      Map<String, String> tblProps) throws SemanticException {
+      boolean isTemporary, Map<String, String> tblProps) throws SemanticException {
     Database database  = getDatabase(qualifiedTabName[0]);
     outputs.add(new WriteEntity(database, WriteEntity.WriteType.DDL_SHARED));
 
     Table t = new Table(qualifiedTabName[0], qualifiedTabName[1]);
     t.setParameters(tblProps);
     t.setTableType(type);
+    t.setTemporary(isTemporary);
     outputs.add(new WriteEntity(t, WriteEntity.WriteType.DDL_NO_LOCK));
   }
 
@@ -13411,7 +13412,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getInputFormat(), storageFormat.getOutputFormat(),
           location, storageFormat.getSerde(), storageFormat.getStorageHandler(),
           storageFormat.getSerdeProps());
-      addDbAndTabToOutputs(qualTabName, TableType.MATERIALIZED_VIEW, tblProps);
+      addDbAndTabToOutputs(qualTabName, TableType.MATERIALIZED_VIEW, false, tblProps);
       queryState.setCommandType(HiveOperation.CREATE_MATERIALIZED_VIEW);
     } else {
       createVwDesc = new CreateViewDesc(
@@ -13420,7 +13421,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           storageFormat.getOutputFormat(), storageFormat.getSerde());
       rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
           createVwDesc)));
-      addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW, tblProps);
+      addDbAndTabToOutputs(qualTabName, TableType.VIRTUAL_VIEW, false, tblProps);
       queryState.setCommandType(HiveOperation.CREATEVIEW);
     }
     qb.setViewDesc(createVwDesc);


[06/46] hive git commit: HIVE-19951: Vectorization: Need to disable encoded LLAP I/O for ORC when there is data type conversion (Schema Evolution) (Matt McCline, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-19951: Vectorization: Need to disable encoded LLAP I/O for ORC when there is data type conversion (Schema Evolution) (Matt McCline, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/master-txnstats
Commit: d78d6465a969ef7e7b3363a257f2c4e6f748d0df
Parents: bb531be
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon Jul 2 08:40:39 2018 -0500
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Mon Jul 2 08:40:39 2018 -0500

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../hive/llap/io/api/impl/LlapRecordReader.java |  64 +++++++
 .../vector_llap_io_data_conversion.q            |  19 ++
 .../llap/vector_llap_io_data_conversion.q.out   | 187 +++++++++++++++++++
 4 files changed, 271 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d78d6465/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 35fad2c..d415b7d 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -761,6 +761,7 @@ minillaplocal.query.files=\
   vector_join_filters.q,\
   vector_leftsemi_mapjoin.q,\
   vector_like_2.q,\
+  vector_llap_io_data_conversion.q,\
   vector_llap_text_1.q,\
   vector_mapjoin_reduce.q,\
   vector_null_map.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/d78d6465/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
index 201c097..be748e9 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapRecordReader.java
@@ -289,8 +289,72 @@ class LlapRecordReader
     executor.submit(rp.getReadCallable());
   }
 
+  private boolean hasSchemaEvolutionStringFamilyTruncateIssue(SchemaEvolution evolution) {
+    return hasStringFamilyTruncateTypeIssue(evolution, evolution.getReaderSchema());
+  }
+
+  // We recurse through the types.
+  private boolean hasStringFamilyTruncateTypeIssue(SchemaEvolution evolution,
+      TypeDescription readerType) {
+    TypeDescription fileType = evolution.getFileType(readerType);
+    if (fileType == null) {
+      return false;
+    }
+    switch (fileType.getCategory()) {
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case LONG:
+    case DOUBLE:
+    case FLOAT:
+    case STRING:
+    case TIMESTAMP:
+    case BINARY:
+    case DATE:
+    case DECIMAL:
+      // We are only looking for the CHAR/VARCHAR truncate issue.
+      return false;
+    case CHAR:
+    case VARCHAR:
+      if (readerType.getCategory().equals(TypeDescription.Category.CHAR) ||
+          readerType.getCategory().equals(TypeDescription.Category.VARCHAR)) {
+        return (fileType.getMaxLength() > readerType.getMaxLength());
+      }
+      return false;
+    case UNION:
+    case MAP:
+    case LIST:
+    case STRUCT:
+      {
+        List<TypeDescription> readerChildren = readerType.getChildren();
+        final int childCount = readerChildren.size();
+        for (int i = 0; i < childCount; ++i) {
+          if (hasStringFamilyTruncateTypeIssue(evolution, readerChildren.get(i))) {
+            return true;
+          }
+        }
+      }
+      return false;
+    default:
+      throw new IllegalArgumentException("Unknown type " + fileType);
+    }
+  }
+
   private boolean checkOrcSchemaEvolution() {
     SchemaEvolution evolution = rp.getSchemaEvolution();
+
+    /*
+     * FUTURE: When SchemaEvolution.isOnlyImplicitConversion becomes available:
+     *  1) Replace the hasSchemaEvolutionStringFamilyTruncateIssue call with
+     *     !isOnlyImplicitConversion.
+     *  2) Delete hasSchemaEvolutionStringFamilyTruncateIssue code.
+     */
+    if (evolution.hasConversion() && hasSchemaEvolutionStringFamilyTruncateIssue(evolution)) {
+
+      // We do not support data type conversion when reading encoded ORC data.
+      return false;
+    }
     // TODO: should this just use physical IDs?
     for (int i = 0; i < includes.getReaderLogicalColumnIds().size(); ++i) {
       int projectedColId = includes.getReaderLogicalColumnIds().get(i);

http://git-wip-us.apache.org/repos/asf/hive/blob/d78d6465/ql/src/test/queries/clientpositive/vector_llap_io_data_conversion.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_llap_io_data_conversion.q b/ql/src/test/queries/clientpositive/vector_llap_io_data_conversion.q
new file mode 100644
index 0000000..f40c4b9
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_llap_io_data_conversion.q
@@ -0,0 +1,19 @@
+--! qt:dataset:alltypesorc
+set hive.explain.user=false;
+SET hive.vectorized.execution.enabled=true;
+
+set hive.llap.io.enabled=true;
+set hive.llap.io.encode.enabled=true;
+
+create table varchar_single_partition(vt varchar(10), vsi varchar(10), vi varchar(20), vb varchar(30), vf varchar(20),vd varchar(20),vs varchar(50))
+    partitioned by(s varchar(50)) stored as orc;
+insert into table varchar_single_partition partition(s='positive') select ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1 from alltypesorc where cint>0 limit 10;
+insert into table varchar_single_partition partition(s='negative') select ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1 from alltypesorc where cint<0 limit 10;
+alter table varchar_single_partition change column vs vs varchar(10);
+
+create table varchar_ctas_1 stored as orc as select vs, length(vs) as c1,reverse(vs) as c2 from varchar_single_partition where s='positive';
+
+explain vectorization detail
+select * from varchar_ctas_1 order by vs, c1, c2;
+
+select * from varchar_ctas_1 order by vs, c1, c2;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/d78d6465/ql/src/test/results/clientpositive/llap/vector_llap_io_data_conversion.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_llap_io_data_conversion.q.out b/ql/src/test/results/clientpositive/llap/vector_llap_io_data_conversion.q.out
new file mode 100644
index 0000000..f503761
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vector_llap_io_data_conversion.q.out
@@ -0,0 +1,187 @@
+PREHOOK: query: create table varchar_single_partition(vt varchar(10), vsi varchar(10), vi varchar(20), vb varchar(30), vf varchar(20),vd varchar(20),vs varchar(50))
+    partitioned by(s varchar(50)) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@varchar_single_partition
+POSTHOOK: query: create table varchar_single_partition(vt varchar(10), vsi varchar(10), vi varchar(20), vb varchar(30), vf varchar(20),vd varchar(20),vs varchar(50))
+    partitioned by(s varchar(50)) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@varchar_single_partition
+PREHOOK: query: insert into table varchar_single_partition partition(s='positive') select ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1 from alltypesorc where cint>0 limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@varchar_single_partition@s=positive
+POSTHOOK: query: insert into table varchar_single_partition partition(s='positive') select ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1 from alltypesorc where cint>0 limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@varchar_single_partition@s=positive
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vb EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vd EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vf EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vi EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vs EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cstring1, type:string, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vsi EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=positive).vt EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
+PREHOOK: query: insert into table varchar_single_partition partition(s='negative') select ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1 from alltypesorc where cint<0 limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@varchar_single_partition@s=negative
+POSTHOOK: query: insert into table varchar_single_partition partition(s='negative') select ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1 from alltypesorc where cint<0 limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@varchar_single_partition@s=negative
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vb EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vd EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vf EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vi EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vs EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:cstring1, type:string, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vsi EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: varchar_single_partition PARTITION(s=negative).vt EXPRESSION [(alltypesorc)alltypesorc.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
+PREHOOK: query: alter table varchar_single_partition change column vs vs varchar(10)
+PREHOOK: type: ALTERTABLE_RENAMECOL
+PREHOOK: Input: default@varchar_single_partition
+PREHOOK: Output: default@varchar_single_partition
+POSTHOOK: query: alter table varchar_single_partition change column vs vs varchar(10)
+POSTHOOK: type: ALTERTABLE_RENAMECOL
+POSTHOOK: Input: default@varchar_single_partition
+POSTHOOK: Output: default@varchar_single_partition
+PREHOOK: query: create table varchar_ctas_1 stored as orc as select vs, length(vs) as c1,reverse(vs) as c2 from varchar_single_partition where s='positive'
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@varchar_single_partition
+PREHOOK: Input: default@varchar_single_partition@s=positive
+PREHOOK: Output: database:default
+PREHOOK: Output: default@varchar_ctas_1
+POSTHOOK: query: create table varchar_ctas_1 stored as orc as select vs, length(vs) as c1,reverse(vs) as c2 from varchar_single_partition where s='positive'
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@varchar_single_partition
+POSTHOOK: Input: default@varchar_single_partition@s=positive
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@varchar_ctas_1
+POSTHOOK: Lineage: varchar_ctas_1.c1 EXPRESSION [(varchar_single_partition)varchar_single_partition.FieldSchema(name:vs, type:varchar(10), comment:null), ]
+POSTHOOK: Lineage: varchar_ctas_1.c2 EXPRESSION [(varchar_single_partition)varchar_single_partition.FieldSchema(name:vs, type:varchar(10), comment:null), ]
+POSTHOOK: Lineage: varchar_ctas_1.vs SIMPLE [(varchar_single_partition)varchar_single_partition.FieldSchema(name:vs, type:varchar(10), comment:null), ]
+PREHOOK: query: explain vectorization detail
+select * from varchar_ctas_1 order by vs, c1, c2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain vectorization detail
+select * from varchar_ctas_1 order by vs, c1, c2
+POSTHOOK: type: QUERY
+PLAN VECTORIZATION:
+  enabled: true
+  enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: varchar_ctas_1
+                  Statistics: Num rows: 10 Data size: 2820 Basic stats: COMPLETE Column stats: NONE
+                  TableScan Vectorization:
+                      native: true
+                      vectorizationSchemaColumns: [0:vs:varchar(10), 1:c1:int, 2:c2:string, 3:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+                  Select Operator
+                    expressions: vs (type: varchar(10)), c1 (type: int), c2 (type: string)
+                    outputColumnNames: _col0, _col1, _col2
+                    Select Vectorization:
+                        className: VectorSelectOperator
+                        native: true
+                        projectedOutputColumnNums: [0, 1, 2]
+                    Statistics: Num rows: 10 Data size: 2820 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: varchar(10)), _col1 (type: int), _col2 (type: string)
+                      sort order: +++
+                      Reduce Sink Vectorization:
+                          className: VectorReduceSinkObjectHashOperator
+                          keyColumnNums: [0, 1, 2]
+                          native: true
+                          nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumnNums: []
+                      Statistics: Num rows: 10 Data size: 2820 Basic stats: COMPLETE Column stats: NONE
+            Execution mode: vectorized, llap
+            LLAP IO: all inputs
+            Map Vectorization:
+                enabled: true
+                enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+                inputFormatFeatureSupport: [DECIMAL_64]
+                featureSupportInUse: [DECIMAL_64]
+                inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                allNative: true
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 3
+                    includeColumns: [0, 1, 2]
+                    dataColumns: vs:varchar(10), c1:int, c2:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Vectorization:
+                enabled: true
+                enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+                reduceColumnNullOrder: aaa
+                reduceColumnSortOrder: +++
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 3
+                    dataColumns: KEY.reducesinkkey0:varchar(10), KEY.reducesinkkey1:int, KEY.reducesinkkey2:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: varchar(10)), KEY.reducesinkkey1 (type: int), KEY.reducesinkkey2 (type: string)
+                outputColumnNames: _col0, _col1, _col2
+                Select Vectorization:
+                    className: VectorSelectOperator
+                    native: true
+                    projectedOutputColumnNums: [0, 1, 2]
+                Statistics: Num rows: 10 Data size: 2820 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  File Sink Vectorization:
+                      className: VectorFileSinkOperator
+                      native: false
+                  Statistics: Num rows: 10 Data size: 2820 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select * from varchar_ctas_1 order by vs, c1, c2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@varchar_ctas_1
+#### A masked pattern was here ####
+POSTHOOK: query: select * from varchar_ctas_1 order by vs, c1, c2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@varchar_ctas_1
+#### A masked pattern was here ####
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc
+cvLH6Eat2y	10	y2taE6HLvc


[36/46] hive git commit: HIVE-20074: Disable TestTriggersWorkloadManager as it is unstable again (Prasanth Jayachandran reviewed by Jesus Camacho Rodriguez)

Posted by se...@apache.org.
HIVE-20074: Disable TestTriggersWorkloadManager as it is unstable again (Prasanth Jayachandran reviewed by Jesus Camacho Rodriguez)


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

Branch: refs/heads/master-txnstats
Commit: 4dbf991b48f5e7b34428c7c23fe34a77e0a9987a
Parents: d6fddac
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Jul 3 09:55:55 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Jul 3 09:55:55 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/jdbc/TestTriggersWorkloadManager.java     | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4dbf991b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersWorkloadManager.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersWorkloadManager.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersWorkloadManager.java
index 53e7347..6a59ff1 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersWorkloadManager.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestTriggersWorkloadManager.java
@@ -35,11 +35,13 @@ import org.apache.hadoop.hive.ql.wm.Trigger;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.rules.TestName;
 
 import com.google.common.collect.Lists;
 
+@Ignore("Disabled in HIVE-20074 temporary as it is unstable, Will re-enable in HIVE-20075.")
 public class TestTriggersWorkloadManager extends TestTriggersTezSessionPoolManager {
   @Rule
   public TestName testName = new TestName();


[35/46] hive git commit: HIVE-19792: Upgrade orc to 1.5.2 and enable decimal_64 schema evolution tests (Prasanth Jayachandran reviewed by Matt McCline)

Posted by se...@apache.org.
HIVE-19792: Upgrade orc to 1.5.2 and enable decimal_64 schema evolution tests (Prasanth Jayachandran 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/d6fddac8
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/d6fddac8
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/d6fddac8

Branch: refs/heads/master-txnstats
Commit: d6fddac8e5b3a87b77f754d704643922197210ad
Parents: f519db7
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Jul 3 08:49:45 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Jul 3 08:49:51 2018 -0700

----------------------------------------------------------------------
 pom.xml                                         |  2 +-
 .../clientpositive/orc_schema_evolution_float.q |  2 -
 .../schema_evol_orc_nonvec_part_all_primitive.q |  2 -
 ...evol_orc_nonvec_part_all_primitive_llap_io.q |  2 -
 .../schema_evol_orc_vec_part_all_primitive.q    |  2 -
 ...ma_evol_orc_vec_part_all_primitive_llap_io.q |  2 -
 .../clientpositive/type_change_test_int.q       |  3 -
 .../type_change_test_int_vectorized.q           |  2 -
 .../clientpositive/llap/orc_merge11.q.out       | 54 ++++++-------
 ...schema_evol_orc_vec_part_all_primitive.q.out | 19 ++---
 ...vol_orc_vec_part_all_primitive_llap_io.q.out | 19 ++---
 .../results/clientpositive/orc_file_dump.q.out  | 24 +++---
 .../results/clientpositive/orc_merge11.q.out    | 54 ++++++-------
 .../results/clientpositive/orc_merge12.q.out    | 84 ++++++++++----------
 .../clientpositive/tez/orc_merge12.q.out        | 84 ++++++++++----------
 15 files changed, 165 insertions(+), 190 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5202248..28ad152 100644
--- a/pom.xml
+++ b/pom.xml
@@ -184,7 +184,7 @@
     <libthrift.version>0.9.3</libthrift.version>
     <log4j2.version>2.10.0</log4j2.version>
     <opencsv.version>2.3</opencsv.version>
-    <orc.version>1.5.1</orc.version>
+    <orc.version>1.5.2</orc.version>
     <mockito-all.version>1.10.19</mockito-all.version>
     <mina.version>2.0.0-M5</mina.version>
     <netty.version>4.1.17.Final</netty.version>

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/orc_schema_evolution_float.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_schema_evolution_float.q b/ql/src/test/queries/clientpositive/orc_schema_evolution_float.q
index ca5dc6f..c2d9840 100644
--- a/ql/src/test/queries/clientpositive/orc_schema_evolution_float.q
+++ b/ql/src/test/queries/clientpositive/orc_schema_evolution_float.q
@@ -1,8 +1,6 @@
 set hive.vectorized.execution.enabled=false;
 set hive.optimize.index.filter=false;
 set hive.metastore.disallow.incompatible.col.type.changes=false;
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
 
 drop table float_text;
 create table float_text(f float);

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive.q b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive.q
index 53c16e0..427734f 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive.q
@@ -12,8 +12,6 @@ set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.metastore.disallow.incompatible.col.type.changes=false;
 set hive.default.fileformat=orc;
 set hive.llap.io.enabled=false;
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
 
 -- SORT_QUERY_RESULTS
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive_llap_io.q
index f2fb2f0..1eca9e3 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_part_all_primitive_llap_io.q
@@ -13,8 +13,6 @@ set hive.metastore.disallow.incompatible.col.type.changes=false;
 set hive.default.fileformat=orc;
 set hive.llap.io.enabled=true;
 set hive.llap.io.encode.enabled=true;
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
 
 -- SORT_QUERY_RESULTS
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
index e811f1d..6e35f5a 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive.q
@@ -12,8 +12,6 @@ set hive.exec.dynamic.partition.mode=nonstrict;
 set hive.metastore.disallow.incompatible.col.type.changes=false;
 set hive.default.fileformat=orc;
 set hive.llap.io.enabled=false;
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
 
 -- SORT_QUERY_RESULTS
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive_llap_io.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive_llap_io.q b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive_llap_io.q
index bae6cc8..576f994 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive_llap_io.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_vec_part_all_primitive_llap_io.q
@@ -13,8 +13,6 @@ set hive.metastore.disallow.incompatible.col.type.changes=false;
 set hive.default.fileformat=orc;
 set hive.llap.io.enabled=true;
 set hive.llap.io.encode.enabled=true;
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
 
 -- SORT_QUERY_RESULTS
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/type_change_test_int.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/type_change_test_int.q b/ql/src/test/queries/clientpositive/type_change_test_int.q
index 2a49871..112a674 100644
--- a/ql/src/test/queries/clientpositive/type_change_test_int.q
+++ b/ql/src/test/queries/clientpositive/type_change_test_int.q
@@ -1,6 +1,3 @@
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
-
 -- Create a base table to be used for loading data: Begin
 drop table if exists testAltCol_n1;
 create table testAltCol_n1

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/queries/clientpositive/type_change_test_int_vectorized.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/type_change_test_int_vectorized.q b/ql/src/test/queries/clientpositive/type_change_test_int_vectorized.q
index 6a940ac..9e93a2f 100644
--- a/ql/src/test/queries/clientpositive/type_change_test_int_vectorized.q
+++ b/ql/src/test/queries/clientpositive/type_change_test_int_vectorized.q
@@ -1,5 +1,3 @@
--- set this to 'decimal_64' after resolving HIVE-19792
-set hive.vectorized.input.format.supports.enabled=none;
 -- Create a base table to be used for loading data: Begin
 drop table if exists testAltCol;
 create table testAltCol

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/llap/orc_merge11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/orc_merge11.q.out b/ql/src/test/results/clientpositive/llap/orc_merge11.q.out
index 5327299..cfb2337 100644
--- a/ql/src/test/results/clientpositive/llap/orc_merge11.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_merge11.q.out
@@ -85,7 +85,7 @@ Stripe Statistics:
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 File Statistics:
   Column 0: count: 50000 hasNull: false
@@ -93,7 +93,7 @@ File Statistics:
   Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
   Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
   Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 Stripes:
   Stripe: offset: 3 data: 5761 rows: 50000 tail: 99 index: 433
@@ -149,11 +149,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
 
 File length: 6672 bytes
 Padding length: 0 bytes
@@ -176,7 +176,7 @@ Stripe Statistics:
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 File Statistics:
   Column 0: count: 50000 hasNull: false
@@ -184,7 +184,7 @@ File Statistics:
   Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
   Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
   Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 Stripes:
   Stripe: offset: 3 data: 5761 rows: 50000 tail: 99 index: 433
@@ -240,11 +240,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
 
 File length: 6672 bytes
 Padding length: 0 bytes
@@ -288,14 +288,14 @@ Stripe Statistics:
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
   Stripe 2:
     Column 0: count: 50000 hasNull: false
     Column 1: count: 50000 hasNull: false bytesOnDisk: 30 min: 2 max: 100 sum: 4999238
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 File Statistics:
   Column 0: count: 100000 hasNull: false
@@ -303,7 +303,7 @@ File Statistics:
   Column 2: count: 100000 hasNull: false bytesOnDisk: 110 min: bar max: zebra sum: 499960
   Column 3: count: 100000 hasNull: false bytesOnDisk: 10228 min: 0.8 max: 80.0 sum: 800205.6
   Column 4: count: 100000 hasNull: false bytesOnDisk: 996 min: 0 max: 6 sum: 64
-  Column 5: count: 100000 hasNull: false bytesOnDisk: 128 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+  Column 5: count: 100000 hasNull: false bytesOnDisk: 128 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 Stripes:
   Stripe: offset: 3 data: 5761 rows: 50000 tail: 99 index: 433
@@ -359,11 +359,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
   Stripe: offset: 6296 data: 5761 rows: 50000 tail: 99 index: 433
     Stream: column 0 section ROW_INDEX start: 6296 length 17
     Stream: column 1 section ROW_INDEX start: 6313 length 73
@@ -417,11 +417,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
 
 File length: 12978 bytes
 Padding length: 0 bytes

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
index 92f6f3a..ae483ca 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
@@ -293,8 +293,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -545,8 +544,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -719,8 +717,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -877,8 +874,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -1092,7 +1088,7 @@ STAGE PLANS:
                   Statistics: Num rows: 13 Data size: 9347 Basic stats: COMPLETE Column stats: PARTIAL
                   TableScan Vectorization:
                       native: true
-                      vectorizationSchemaColumns: [0:insert_num:int, 1:c1:char(8), 2:c2:char(32), 3:c3:varchar(15), 4:c4:varchar(18), 5:c5:decimal(10,2), 6:c6:decimal(25,15), 7:b:string, 8:part:int, 9:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+                      vectorizationSchemaColumns: [0:insert_num:int, 1:c1:char(8), 2:c2:char(32), 3:c3:varchar(15), 4:c4:varchar(18), 5:c5:decimal(10,2)/DECIMAL_64, 6:c6:decimal(25,15), 7:b:string, 8:part:int, 9:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
                   Select Operator
                     expressions: insert_num (type: int), part (type: int), c1 (type: char(8)), c2 (type: char(32)), c3 (type: varchar(15)), c4 (type: varchar(18)), c5 (type: decimal(10,2)), c6 (type: decimal(25,15)), b (type: string)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -1116,8 +1112,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -1125,7 +1120,7 @@ STAGE PLANS:
                 rowBatchContext:
                     dataColumnCount: 8
                     includeColumns: [0, 1, 2, 3, 4, 5, 6, 7]
-                    dataColumns: insert_num:int, c1:char(8), c2:char(32), c3:varchar(15), c4:varchar(18), c5:decimal(10,2), c6:decimal(25,15), b:string
+                    dataColumns: insert_num:int, c1:char(8), c2:char(32), c3:varchar(15), c4:varchar(18), c5:decimal(10,2)/DECIMAL_64, c6:decimal(25,15), b:string
                     partitionColumnCount: 1
                     partitionColumns: part:int
                     scratchColumnTypeNames: []

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive_llap_io.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive_llap_io.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive_llap_io.q.out
index 6cfbb2f..fb1a4d2 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive_llap_io.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive_llap_io.q.out
@@ -294,8 +294,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -547,8 +546,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -722,8 +720,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -881,8 +878,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -1096,7 +1092,7 @@ STAGE PLANS:
                   Statistics: Num rows: 13 Data size: 9347 Basic stats: COMPLETE Column stats: PARTIAL
                   TableScan Vectorization:
                       native: true
-                      vectorizationSchemaColumns: [0:insert_num:int, 1:c1:char(8), 2:c2:char(32), 3:c3:varchar(15), 4:c4:varchar(18), 5:c5:decimal(10,2), 6:c6:decimal(25,15), 7:b:string, 8:part:int, 9:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+                      vectorizationSchemaColumns: [0:insert_num:int, 1:c1:char(8), 2:c2:char(32), 3:c3:varchar(15), 4:c4:varchar(18), 5:c5:decimal(10,2)/DECIMAL_64, 6:c6:decimal(25,15), 7:b:string, 8:part:int, 9:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
                   Select Operator
                     expressions: insert_num (type: int), part (type: int), c1 (type: char(8)), c2 (type: char(32)), c3 (type: varchar(15)), c4 (type: varchar(18)), c5 (type: decimal(10,2)), c6 (type: decimal(25,15)), b (type: string)
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -1121,8 +1117,7 @@ STAGE PLANS:
                 enabled: true
                 enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
                 inputFormatFeatureSupport: [DECIMAL_64]
-                vectorizationSupportRemovedReasons: [[] is disabled because it is not in hive.vectorized.input.format.supports.enabled []]
-                featureSupportInUse: []
+                featureSupportInUse: [DECIMAL_64]
                 inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
                 allNative: false
                 usesVectorUDFAdaptor: false
@@ -1130,7 +1125,7 @@ STAGE PLANS:
                 rowBatchContext:
                     dataColumnCount: 8
                     includeColumns: [0, 1, 2, 3, 4, 5, 6, 7]
-                    dataColumns: insert_num:int, c1:char(8), c2:char(32), c3:varchar(15), c4:varchar(18), c5:decimal(10,2), c6:decimal(25,15), b:string
+                    dataColumns: insert_num:int, c1:char(8), c2:char(32), c3:varchar(15), c4:varchar(18), c5:decimal(10,2)/DECIMAL_64, c6:decimal(25,15), b:string
                     partitionColumnCount: 1
                     partitionColumns: part:int
                     scratchColumnTypeNames: []

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/orc_file_dump.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_file_dump.q.out b/ql/src/test/results/clientpositive/orc_file_dump.q.out
index 77be7a5..fde2785 100644
--- a/ql/src/test/results/clientpositive/orc_file_dump.q.out
+++ b/ql/src/test/results/clientpositive/orc_file_dump.q.out
@@ -110,7 +110,7 @@ Stripe Statistics:
     Column 6: count: 1049 hasNull: false bytesOnDisk: 3323 min: 0.02 max: 49.85 sum: 26286.349999999977
     Column 7: count: 1049 hasNull: false bytesOnDisk: 137 true: 526
     Column 8: count: 1049 hasNull: false bytesOnDisk: 3430 min:  max: zach zipper sum: 13443
-    Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703
+    Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703
     Column 10: count: 1049 hasNull: false bytesOnDisk: 2181 min: 0 max: 99.94 sum: 53646.16
     Column 11: count: 1049 hasNull: false bytesOnDisk: 2468 sum: 13278
 
@@ -124,7 +124,7 @@ File Statistics:
   Column 6: count: 1049 hasNull: false bytesOnDisk: 3323 min: 0.02 max: 49.85 sum: 26286.349999999977
   Column 7: count: 1049 hasNull: false bytesOnDisk: 137 true: 526
   Column 8: count: 1049 hasNull: false bytesOnDisk: 3430 min:  max: zach zipper sum: 13443
-  Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703
+  Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703
   Column 10: count: 1049 hasNull: false bytesOnDisk: 2181 min: 0 max: 99.94 sum: 53646.16
   Column 11: count: 1049 hasNull: false bytesOnDisk: 2468 sum: 13278
 
@@ -248,8 +248,8 @@ Stripes:
       Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 182 loadFactor: 0.029 expectedFpp: 7.090246E-7
       Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 1772 loadFactor: 0.2825 expectedFpp: 0.0063713384
     Row group indices for column 9:
-      Entry 0: count: 1000 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703 positions: 0,0,0,0,0,0
-      Entry 1: count: 49 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703 positions: 0,7,488,0,1538,488
+      Entry 0: count: 1000 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 positions: 0,0,0,0,0,0
+      Entry 1: count: 49 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 positions: 0,7,488,0,1538,488
     Bloom filters for column 9:
       Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 4 loadFactor: 0.0006 expectedFpp: 1.6543056E-13
       Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 4 loadFactor: 0.0006 expectedFpp: 1.6543056E-13
@@ -307,7 +307,7 @@ Stripe Statistics:
     Column 6: count: 1049 hasNull: false bytesOnDisk: 3323 min: 0.02 max: 49.85 sum: 26286.349999999977
     Column 7: count: 1049 hasNull: false bytesOnDisk: 137 true: 526
     Column 8: count: 1049 hasNull: false bytesOnDisk: 3430 min:  max: zach zipper sum: 13443
-    Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703
+    Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703
     Column 10: count: 1049 hasNull: false bytesOnDisk: 2181 min: 0 max: 99.94 sum: 53646.16
     Column 11: count: 1049 hasNull: false bytesOnDisk: 2468 sum: 13278
 
@@ -321,7 +321,7 @@ File Statistics:
   Column 6: count: 1049 hasNull: false bytesOnDisk: 3323 min: 0.02 max: 49.85 sum: 26286.349999999977
   Column 7: count: 1049 hasNull: false bytesOnDisk: 137 true: 526
   Column 8: count: 1049 hasNull: false bytesOnDisk: 3430 min:  max: zach zipper sum: 13443
-  Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703
+  Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703
   Column 10: count: 1049 hasNull: false bytesOnDisk: 2181 min: 0 max: 99.94 sum: 53646.16
   Column 11: count: 1049 hasNull: false bytesOnDisk: 2468 sum: 13278
 
@@ -445,8 +445,8 @@ Stripes:
       Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 317 loadFactor: 0.033 expectedFpp: 4.2807122E-11
       Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 3015 loadFactor: 0.3141 expectedFpp: 3.0137875E-4
     Row group indices for column 9:
-      Entry 0: count: 1000 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703 positions: 0,0,0,0,0,0
-      Entry 1: count: 49 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703 positions: 0,7,488,0,1538,488
+      Entry 0: count: 1000 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 positions: 0,0,0,0,0,0
+      Entry 1: count: 49 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 positions: 0,7,488,0,1538,488
     Bloom filters for column 9:
       Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 7 loadFactor: 0.0007 expectedFpp: 1.0959422E-22
       Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 7 loadFactor: 0.0007 expectedFpp: 1.0959422E-22
@@ -516,7 +516,7 @@ Stripe Statistics:
     Column 6: count: 1049 hasNull: false bytesOnDisk: 3323 min: 0.02 max: 49.85 sum: 26286.349999999977
     Column 7: count: 1049 hasNull: false bytesOnDisk: 137 true: 526
     Column 8: count: 1049 hasNull: false bytesOnDisk: 3430 min:  max: zach zipper sum: 13443
-    Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703
+    Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703
     Column 10: count: 1049 hasNull: false bytesOnDisk: 2181 min: 0 max: 99.94 sum: 53646.16
     Column 11: count: 1049 hasNull: false bytesOnDisk: 2468 sum: 13278
 
@@ -530,7 +530,7 @@ File Statistics:
   Column 6: count: 1049 hasNull: false bytesOnDisk: 3323 min: 0.02 max: 49.85 sum: 26286.349999999977
   Column 7: count: 1049 hasNull: false bytesOnDisk: 137 true: 526
   Column 8: count: 1049 hasNull: false bytesOnDisk: 3430 min:  max: zach zipper sum: 13443
-  Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703
+  Column 9: count: 1049 hasNull: false bytesOnDisk: 1802 min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703
   Column 10: count: 1049 hasNull: false bytesOnDisk: 2181 min: 0 max: 99.94 sum: 53646.16
   Column 11: count: 1049 hasNull: false bytesOnDisk: 2468 sum: 13278
 
@@ -654,8 +654,8 @@ Stripes:
       Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 182 loadFactor: 0.029 expectedFpp: 7.090246E-7
       Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 1772 loadFactor: 0.2825 expectedFpp: 0.0063713384
     Row group indices for column 9:
-      Entry 0: count: 1000 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703 positions: 0,0,0,0,0,0
-      Entry 1: count: 49 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 min UTC: 2013-03-01 01:11:58.703 max UTC: 2013-03-01 01:11:58.703 positions: 0,7,488,0,1538,488
+      Entry 0: count: 1000 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 positions: 0,0,0,0,0,0
+      Entry 1: count: 49 hasNull: false min: 2013-03-01 09:11:58.703 max: 2013-03-01 09:11:58.703 positions: 0,7,488,0,1538,488
     Bloom filters for column 9:
       Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 4 loadFactor: 0.0006 expectedFpp: 1.6543056E-13
       Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 4 loadFactor: 0.0006 expectedFpp: 1.6543056E-13

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/orc_merge11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_merge11.q.out b/ql/src/test/results/clientpositive/orc_merge11.q.out
index 5327299..cfb2337 100644
--- a/ql/src/test/results/clientpositive/orc_merge11.q.out
+++ b/ql/src/test/results/clientpositive/orc_merge11.q.out
@@ -85,7 +85,7 @@ Stripe Statistics:
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 File Statistics:
   Column 0: count: 50000 hasNull: false
@@ -93,7 +93,7 @@ File Statistics:
   Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
   Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
   Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 Stripes:
   Stripe: offset: 3 data: 5761 rows: 50000 tail: 99 index: 433
@@ -149,11 +149,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
 
 File length: 6672 bytes
 Padding length: 0 bytes
@@ -176,7 +176,7 @@ Stripe Statistics:
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 File Statistics:
   Column 0: count: 50000 hasNull: false
@@ -184,7 +184,7 @@ File Statistics:
   Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
   Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
   Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+  Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 Stripes:
   Stripe: offset: 3 data: 5761 rows: 50000 tail: 99 index: 433
@@ -240,11 +240,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
 
 File length: 6672 bytes
 Padding length: 0 bytes
@@ -288,14 +288,14 @@ Stripe Statistics:
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
   Stripe 2:
     Column 0: count: 50000 hasNull: false
     Column 1: count: 50000 hasNull: false bytesOnDisk: 30 min: 2 max: 100 sum: 4999238
     Column 2: count: 50000 hasNull: false bytesOnDisk: 55 min: bar max: zebra sum: 249980
     Column 3: count: 50000 hasNull: false bytesOnDisk: 5114 min: 0.8 max: 80.0 sum: 400102.8
     Column 4: count: 50000 hasNull: false bytesOnDisk: 498 min: 0 max: 6 sum: 32
-    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+    Column 5: count: 50000 hasNull: false bytesOnDisk: 64 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 File Statistics:
   Column 0: count: 100000 hasNull: false
@@ -303,7 +303,7 @@ File Statistics:
   Column 2: count: 100000 hasNull: false bytesOnDisk: 110 min: bar max: zebra sum: 499960
   Column 3: count: 100000 hasNull: false bytesOnDisk: 10228 min: 0.8 max: 80.0 sum: 800205.6
   Column 4: count: 100000 hasNull: false bytesOnDisk: 996 min: 0 max: 6 sum: 64
-  Column 5: count: 100000 hasNull: false bytesOnDisk: 128 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0
+  Column 5: count: 100000 hasNull: false bytesOnDisk: 128 min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0
 
 Stripes:
   Stripe: offset: 3 data: 5761 rows: 50000 tail: 99 index: 433
@@ -359,11 +359,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
   Stripe: offset: 6296 data: 5761 rows: 50000 tail: 99 index: 433
     Stream: column 0 section ROW_INDEX start: 6296 length 17
     Stream: column 1 section ROW_INDEX start: 6313 length 73
@@ -417,11 +417,11 @@ Stripes:
       Entry 3: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 274,1328,0,232,304
       Entry 4: count: 10000 hasNull: false min: 0 max: 0 sum: 0 positions: 350,3136,0,312,64
     Row group indices for column 5:
-      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:00:00.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,0,0,0,0,0
-      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,194,262,0,76,272
-      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,354,22,0,156,32
-      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,506,294,0,232,304
-      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 min UTC: 1969-12-31 08:04:10.0 max UTC: 1969-12-31 08:04:10.0 positions: 0,666,54,0,312,64
+      Entry 0: count: 10000 hasNull: false min: 1969-12-31 16:00:00.0 max: 1969-12-31 16:04:10.0 positions: 0,0,0,0,0,0
+      Entry 1: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,194,262,0,76,272
+      Entry 2: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,354,22,0,156,32
+      Entry 3: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,506,294,0,232,304
+      Entry 4: count: 10000 hasNull: false min: 1969-12-31 16:04:10.0 max: 1969-12-31 16:04:10.0 positions: 0,666,54,0,312,64
 
 File length: 12978 bytes
 Padding length: 0 bytes

http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/orc_merge12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_merge12.q.out b/ql/src/test/results/clientpositive/orc_merge12.q.out
index 045ba42..6bc2a81 100644
--- a/ql/src/test/results/clientpositive/orc_merge12.q.out
+++ b/ql/src/test/results/clientpositive/orc_merge12.q.out
@@ -161,8 +161,8 @@ Stripe Statistics:
     Column 6: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 7: count: 12288 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 442368
     Column 8: count: 12288 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 884736
-    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 11: count: 9174 hasNull: true true: 6138
     Column 12: count: 9173 hasNull: true true: 3983
     Column 13: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -173,8 +173,8 @@ Stripe Statistics:
     Column 18: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 19: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 20: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 23: count: 9174 hasNull: true true: 6138
     Column 24: count: 9173 hasNull: true true: 3983
     Column 25: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -185,8 +185,8 @@ Stripe Statistics:
     Column 30: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 31: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 32: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 35: count: 9174 hasNull: true true: 6138
     Column 36: count: 9173 hasNull: true true: 3983
   Stripe 2:
@@ -199,8 +199,8 @@ Stripe Statistics:
     Column 6: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 7: count: 12288 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 442368
     Column 8: count: 12288 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 884736
-    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 11: count: 9174 hasNull: true true: 6138
     Column 12: count: 9173 hasNull: true true: 3983
     Column 13: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -211,8 +211,8 @@ Stripe Statistics:
     Column 18: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 19: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 20: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 23: count: 9174 hasNull: true true: 6138
     Column 24: count: 9173 hasNull: true true: 3983
     Column 25: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -223,8 +223,8 @@ Stripe Statistics:
     Column 30: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 31: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 32: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 35: count: 9174 hasNull: true true: 6138
     Column 36: count: 9173 hasNull: true true: 3983
 
@@ -238,8 +238,8 @@ File Statistics:
   Column 6: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
   Column 7: count: 24576 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 884736
   Column 8: count: 24576 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 1769472
-  Column 9: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-  Column 10: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+  Column 9: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+  Column 10: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
   Column 11: count: 18348 hasNull: true true: 12276
   Column 12: count: 18346 hasNull: true true: 7966
   Column 13: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
@@ -250,8 +250,8 @@ File Statistics:
   Column 18: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
   Column 19: count: 18348 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 255762
   Column 20: count: 18346 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 298268
-  Column 21: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-  Column 22: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+  Column 21: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+  Column 22: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
   Column 23: count: 18348 hasNull: true true: 12276
   Column 24: count: 18346 hasNull: true true: 7966
   Column 25: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
@@ -262,8 +262,8 @@ File Statistics:
   Column 30: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
   Column 31: count: 18348 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 255762
   Column 32: count: 18346 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 298268
-  Column 33: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-  Column 34: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+  Column 33: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+  Column 34: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
   Column 35: count: 18348 hasNull: true true: 12276
   Column 36: count: 18346 hasNull: true true: 7966
 
@@ -457,11 +457,11 @@ Stripes:
       Entry 0: count: 10000 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 720000 positions: 0,0,0,0,0
       Entry 1: count: 2288 hasNull: false min: 00124556-8383-44c4-a28b-7a413de74ccc4137606f-2cf7-43fb-beff-b6d374fd15ec max: ffde3bce-bb56-4fa9-81d7-146ca2eab946225c18e0-0002-4d07-9853-12c92c0f5637 sum: 164736 positions: 384237,64640,0,76,272
     Row group indices for column 9:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 10:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 11:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -493,11 +493,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 21:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 22:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 23:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -529,11 +529,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 33:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 34:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 35:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -729,11 +729,11 @@ Stripes:
       Entry 0: count: 10000 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 720000 positions: 0,0,0,0,0
       Entry 1: count: 2288 hasNull: false min: 00124556-8383-44c4-a28b-7a413de74ccc4137606f-2cf7-43fb-beff-b6d374fd15ec max: ffde3bce-bb56-4fa9-81d7-146ca2eab946225c18e0-0002-4d07-9853-12c92c0f5637 sum: 164736 positions: 384237,64640,0,76,272
     Row group indices for column 9:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 10:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 11:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -765,11 +765,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 21:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 22:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 23:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -801,11 +801,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 33:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 34:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 35:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4


[43/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
index 314ea90,0000000..cc6ecdf
mode 100644,000000..100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AlterPartitionsRequest.java
@@@ -1,1067 -1,0 +1,1067 @@@
 +/**
 + * Autogenerated by Thrift Compiler (0.9.3)
 + *
 + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
 + *  @generated
 + */
 +package org.apache.hadoop.hive.metastore.api;
 +
 +import org.apache.thrift.scheme.IScheme;
 +import org.apache.thrift.scheme.SchemeFactory;
 +import org.apache.thrift.scheme.StandardScheme;
 +
 +import org.apache.thrift.scheme.TupleScheme;
 +import org.apache.thrift.protocol.TTupleProtocol;
 +import org.apache.thrift.protocol.TProtocolException;
 +import org.apache.thrift.EncodingUtils;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.async.AsyncMethodCallback;
 +import org.apache.thrift.server.AbstractNonblockingServer.*;
 +import java.util.List;
 +import java.util.ArrayList;
 +import java.util.Map;
 +import java.util.HashMap;
 +import java.util.EnumMap;
 +import java.util.Set;
 +import java.util.HashSet;
 +import java.util.EnumSet;
 +import java.util.Collections;
 +import java.util.BitSet;
 +import java.nio.ByteBuffer;
 +import java.util.Arrays;
 +import javax.annotation.Generated;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
 +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
 +@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class AlterPartitionsRequest implements org.apache.thrift.TBase<AlterPartitionsRequest, AlterPartitionsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<AlterPartitionsRequest> {
 +  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("AlterPartitionsRequest");
 +
 +  private static final org.apache.thrift.protocol.TField DB_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbName", org.apache.thrift.protocol.TType.STRING, (short)1);
 +  private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)2);
 +  private static final org.apache.thrift.protocol.TField PARTITIONS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitions", org.apache.thrift.protocol.TType.LIST, (short)3);
 +  private static final org.apache.thrift.protocol.TField ENVIRONMENT_CONTEXT_FIELD_DESC = new org.apache.thrift.protocol.TField("environmentContext", org.apache.thrift.protocol.TType.STRUCT, (short)4);
 +  private static final org.apache.thrift.protocol.TField TXN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnId", org.apache.thrift.protocol.TType.I64, (short)5);
 +  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)6);
 +  private static final org.apache.thrift.protocol.TField VALID_WRITE_ID_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("validWriteIdList", org.apache.thrift.protocol.TType.STRING, (short)7);
 +
 +  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
 +  static {
 +    schemes.put(StandardScheme.class, new AlterPartitionsRequestStandardSchemeFactory());
 +    schemes.put(TupleScheme.class, new AlterPartitionsRequestTupleSchemeFactory());
 +  }
 +
 +  private String dbName; // required
 +  private String tableName; // required
 +  private List<Partition> partitions; // required
 +  private EnvironmentContext environmentContext; // required
 +  private long txnId; // optional
 +  private long writeId; // optional
 +  private String validWriteIdList; // optional
 +
 +  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
 +  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
 +    DB_NAME((short)1, "dbName"),
 +    TABLE_NAME((short)2, "tableName"),
 +    PARTITIONS((short)3, "partitions"),
 +    ENVIRONMENT_CONTEXT((short)4, "environmentContext"),
 +    TXN_ID((short)5, "txnId"),
 +    WRITE_ID((short)6, "writeId"),
 +    VALID_WRITE_ID_LIST((short)7, "validWriteIdList");
 +
 +    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 +
 +    static {
 +      for (_Fields field : EnumSet.allOf(_Fields.class)) {
 +        byName.put(field.getFieldName(), field);
 +      }
 +    }
 +
 +    /**
 +     * Find the _Fields constant that matches fieldId, or null if its not found.
 +     */
 +    public static _Fields findByThriftId(int fieldId) {
 +      switch(fieldId) {
 +        case 1: // DB_NAME
 +          return DB_NAME;
 +        case 2: // TABLE_NAME
 +          return TABLE_NAME;
 +        case 3: // PARTITIONS
 +          return PARTITIONS;
 +        case 4: // ENVIRONMENT_CONTEXT
 +          return ENVIRONMENT_CONTEXT;
 +        case 5: // TXN_ID
 +          return TXN_ID;
 +        case 6: // WRITE_ID
 +          return WRITE_ID;
 +        case 7: // VALID_WRITE_ID_LIST
 +          return VALID_WRITE_ID_LIST;
 +        default:
 +          return null;
 +      }
 +    }
 +
 +    /**
 +     * Find the _Fields constant that matches fieldId, throwing an exception
 +     * if it is not found.
 +     */
 +    public static _Fields findByThriftIdOrThrow(int fieldId) {
 +      _Fields fields = findByThriftId(fieldId);
 +      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
 +      return fields;
 +    }
 +
 +    /**
 +     * Find the _Fields constant that matches name, or null if its not found.
 +     */
 +    public static _Fields findByName(String name) {
 +      return byName.get(name);
 +    }
 +
 +    private final short _thriftId;
 +    private final String _fieldName;
 +
 +    _Fields(short thriftId, String fieldName) {
 +      _thriftId = thriftId;
 +      _fieldName = fieldName;
 +    }
 +
 +    public short getThriftFieldId() {
 +      return _thriftId;
 +    }
 +
 +    public String getFieldName() {
 +      return _fieldName;
 +    }
 +  }
 +
 +  // isset id assignments
 +  private static final int __TXNID_ISSET_ID = 0;
 +  private static final int __WRITEID_ISSET_ID = 1;
 +  private byte __isset_bitfield = 0;
 +  private static final _Fields optionals[] = {_Fields.TXN_ID,_Fields.WRITE_ID,_Fields.VALID_WRITE_ID_LIST};
 +  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
 +  static {
 +    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
 +    tmpMap.put(_Fields.DB_NAME, new org.apache.thrift.meta_data.FieldMetaData("dbName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
 +        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
 +    tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
 +        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
 +    tmpMap.put(_Fields.PARTITIONS, new org.apache.thrift.meta_data.FieldMetaData("partitions", org.apache.thrift.TFieldRequirementType.REQUIRED, 
 +        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
 +            new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Partition.class))));
 +    tmpMap.put(_Fields.ENVIRONMENT_CONTEXT, new org.apache.thrift.meta_data.FieldMetaData("environmentContext", org.apache.thrift.TFieldRequirementType.REQUIRED, 
 +        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, EnvironmentContext.class)));
 +    tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
 +        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
 +    tmpMap.put(_Fields.WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("writeId", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
 +        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
 +    tmpMap.put(_Fields.VALID_WRITE_ID_LIST, new org.apache.thrift.meta_data.FieldMetaData("validWriteIdList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
 +        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
 +    metaDataMap = Collections.unmodifiableMap(tmpMap);
 +    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AlterPartitionsRequest.class, metaDataMap);
 +  }
 +
 +  public AlterPartitionsRequest() {
 +    this.txnId = -1L;
 +
 +    this.writeId = -1L;
 +
 +  }
 +
 +  public AlterPartitionsRequest(
 +    String dbName,
 +    String tableName,
 +    List<Partition> partitions,
 +    EnvironmentContext environmentContext)
 +  {
 +    this();
 +    this.dbName = dbName;
 +    this.tableName = tableName;
 +    this.partitions = partitions;
 +    this.environmentContext = environmentContext;
 +  }
 +
 +  /**
 +   * Performs a deep copy on <i>other</i>.
 +   */
 +  public AlterPartitionsRequest(AlterPartitionsRequest other) {
 +    __isset_bitfield = other.__isset_bitfield;
 +    if (other.isSetDbName()) {
 +      this.dbName = other.dbName;
 +    }
 +    if (other.isSetTableName()) {
 +      this.tableName = other.tableName;
 +    }
 +    if (other.isSetPartitions()) {
 +      List<Partition> __this__partitions = new ArrayList<Partition>(other.partitions.size());
 +      for (Partition other_element : other.partitions) {
 +        __this__partitions.add(new Partition(other_element));
 +      }
 +      this.partitions = __this__partitions;
 +    }
 +    if (other.isSetEnvironmentContext()) {
 +      this.environmentContext = new EnvironmentContext(other.environmentContext);
 +    }
 +    this.txnId = other.txnId;
 +    this.writeId = other.writeId;
 +    if (other.isSetValidWriteIdList()) {
 +      this.validWriteIdList = other.validWriteIdList;
 +    }
 +  }
 +
 +  public AlterPartitionsRequest deepCopy() {
 +    return new AlterPartitionsRequest(this);
 +  }
 +
 +  @Override
 +  public void clear() {
 +    this.dbName = null;
 +    this.tableName = null;
 +    this.partitions = null;
 +    this.environmentContext = null;
 +    this.txnId = -1L;
 +
 +    this.writeId = -1L;
 +
 +    this.validWriteIdList = null;
 +  }
 +
 +  public String getDbName() {
 +    return this.dbName;
 +  }
 +
 +  public void setDbName(String dbName) {
 +    this.dbName = dbName;
 +  }
 +
 +  public void unsetDbName() {
 +    this.dbName = null;
 +  }
 +
 +  /** Returns true if field dbName is set (has been assigned a value) and false otherwise */
 +  public boolean isSetDbName() {
 +    return this.dbName != null;
 +  }
 +
 +  public void setDbNameIsSet(boolean value) {
 +    if (!value) {
 +      this.dbName = null;
 +    }
 +  }
 +
 +  public String getTableName() {
 +    return this.tableName;
 +  }
 +
 +  public void setTableName(String tableName) {
 +    this.tableName = tableName;
 +  }
 +
 +  public void unsetTableName() {
 +    this.tableName = null;
 +  }
 +
 +  /** Returns true if field tableName is set (has been assigned a value) and false otherwise */
 +  public boolean isSetTableName() {
 +    return this.tableName != null;
 +  }
 +
 +  public void setTableNameIsSet(boolean value) {
 +    if (!value) {
 +      this.tableName = null;
 +    }
 +  }
 +
 +  public int getPartitionsSize() {
 +    return (this.partitions == null) ? 0 : this.partitions.size();
 +  }
 +
 +  public java.util.Iterator<Partition> getPartitionsIterator() {
 +    return (this.partitions == null) ? null : this.partitions.iterator();
 +  }
 +
 +  public void addToPartitions(Partition elem) {
 +    if (this.partitions == null) {
 +      this.partitions = new ArrayList<Partition>();
 +    }
 +    this.partitions.add(elem);
 +  }
 +
 +  public List<Partition> getPartitions() {
 +    return this.partitions;
 +  }
 +
 +  public void setPartitions(List<Partition> partitions) {
 +    this.partitions = partitions;
 +  }
 +
 +  public void unsetPartitions() {
 +    this.partitions = null;
 +  }
 +
 +  /** Returns true if field partitions is set (has been assigned a value) and false otherwise */
 +  public boolean isSetPartitions() {
 +    return this.partitions != null;
 +  }
 +
 +  public void setPartitionsIsSet(boolean value) {
 +    if (!value) {
 +      this.partitions = null;
 +    }
 +  }
 +
 +  public EnvironmentContext getEnvironmentContext() {
 +    return this.environmentContext;
 +  }
 +
 +  public void setEnvironmentContext(EnvironmentContext environmentContext) {
 +    this.environmentContext = environmentContext;
 +  }
 +
 +  public void unsetEnvironmentContext() {
 +    this.environmentContext = null;
 +  }
 +
 +  /** Returns true if field environmentContext is set (has been assigned a value) and false otherwise */
 +  public boolean isSetEnvironmentContext() {
 +    return this.environmentContext != null;
 +  }
 +
 +  public void setEnvironmentContextIsSet(boolean value) {
 +    if (!value) {
 +      this.environmentContext = null;
 +    }
 +  }
 +
 +  public long getTxnId() {
 +    return this.txnId;
 +  }
 +
 +  public void setTxnId(long txnId) {
 +    this.txnId = txnId;
 +    setTxnIdIsSet(true);
 +  }
 +
 +  public void unsetTxnId() {
 +    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID);
 +  }
 +
 +  /** Returns true if field txnId is set (has been assigned a value) and false otherwise */
 +  public boolean isSetTxnId() {
 +    return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID);
 +  }
 +
 +  public void setTxnIdIsSet(boolean value) {
 +    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value);
 +  }
 +
 +  public long getWriteId() {
 +    return this.writeId;
 +  }
 +
 +  public void setWriteId(long writeId) {
 +    this.writeId = writeId;
 +    setWriteIdIsSet(true);
 +  }
 +
 +  public void unsetWriteId() {
 +    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEID_ISSET_ID);
 +  }
 +
 +  /** Returns true if field writeId is set (has been assigned a value) and false otherwise */
 +  public boolean isSetWriteId() {
 +    return EncodingUtils.testBit(__isset_bitfield, __WRITEID_ISSET_ID);
 +  }
 +
 +  public void setWriteIdIsSet(boolean value) {
 +    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEID_ISSET_ID, value);
 +  }
 +
 +  public String getValidWriteIdList() {
 +    return this.validWriteIdList;
 +  }
 +
 +  public void setValidWriteIdList(String validWriteIdList) {
 +    this.validWriteIdList = validWriteIdList;
 +  }
 +
 +  public void unsetValidWriteIdList() {
 +    this.validWriteIdList = null;
 +  }
 +
 +  /** Returns true if field validWriteIdList is set (has been assigned a value) and false otherwise */
 +  public boolean isSetValidWriteIdList() {
 +    return this.validWriteIdList != null;
 +  }
 +
 +  public void setValidWriteIdListIsSet(boolean value) {
 +    if (!value) {
 +      this.validWriteIdList = null;
 +    }
 +  }
 +
 +  public void setFieldValue(_Fields field, Object value) {
 +    switch (field) {
 +    case DB_NAME:
 +      if (value == null) {
 +        unsetDbName();
 +      } else {
 +        setDbName((String)value);
 +      }
 +      break;
 +
 +    case TABLE_NAME:
 +      if (value == null) {
 +        unsetTableName();
 +      } else {
 +        setTableName((String)value);
 +      }
 +      break;
 +
 +    case PARTITIONS:
 +      if (value == null) {
 +        unsetPartitions();
 +      } else {
 +        setPartitions((List<Partition>)value);
 +      }
 +      break;
 +
 +    case ENVIRONMENT_CONTEXT:
 +      if (value == null) {
 +        unsetEnvironmentContext();
 +      } else {
 +        setEnvironmentContext((EnvironmentContext)value);
 +      }
 +      break;
 +
 +    case TXN_ID:
 +      if (value == null) {
 +        unsetTxnId();
 +      } else {
 +        setTxnId((Long)value);
 +      }
 +      break;
 +
 +    case WRITE_ID:
 +      if (value == null) {
 +        unsetWriteId();
 +      } else {
 +        setWriteId((Long)value);
 +      }
 +      break;
 +
 +    case VALID_WRITE_ID_LIST:
 +      if (value == null) {
 +        unsetValidWriteIdList();
 +      } else {
 +        setValidWriteIdList((String)value);
 +      }
 +      break;
 +
 +    }
 +  }
 +
 +  public Object getFieldValue(_Fields field) {
 +    switch (field) {
 +    case DB_NAME:
 +      return getDbName();
 +
 +    case TABLE_NAME:
 +      return getTableName();
 +
 +    case PARTITIONS:
 +      return getPartitions();
 +
 +    case ENVIRONMENT_CONTEXT:
 +      return getEnvironmentContext();
 +
 +    case TXN_ID:
 +      return getTxnId();
 +
 +    case WRITE_ID:
 +      return getWriteId();
 +
 +    case VALID_WRITE_ID_LIST:
 +      return getValidWriteIdList();
 +
 +    }
 +    throw new IllegalStateException();
 +  }
 +
 +  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
 +  public boolean isSet(_Fields field) {
 +    if (field == null) {
 +      throw new IllegalArgumentException();
 +    }
 +
 +    switch (field) {
 +    case DB_NAME:
 +      return isSetDbName();
 +    case TABLE_NAME:
 +      return isSetTableName();
 +    case PARTITIONS:
 +      return isSetPartitions();
 +    case ENVIRONMENT_CONTEXT:
 +      return isSetEnvironmentContext();
 +    case TXN_ID:
 +      return isSetTxnId();
 +    case WRITE_ID:
 +      return isSetWriteId();
 +    case VALID_WRITE_ID_LIST:
 +      return isSetValidWriteIdList();
 +    }
 +    throw new IllegalStateException();
 +  }
 +
 +  @Override
 +  public boolean equals(Object that) {
 +    if (that == null)
 +      return false;
 +    if (that instanceof AlterPartitionsRequest)
 +      return this.equals((AlterPartitionsRequest)that);
 +    return false;
 +  }
 +
 +  public boolean equals(AlterPartitionsRequest that) {
 +    if (that == null)
 +      return false;
 +
 +    boolean this_present_dbName = true && this.isSetDbName();
 +    boolean that_present_dbName = true && that.isSetDbName();
 +    if (this_present_dbName || that_present_dbName) {
 +      if (!(this_present_dbName && that_present_dbName))
 +        return false;
 +      if (!this.dbName.equals(that.dbName))
 +        return false;
 +    }
 +
 +    boolean this_present_tableName = true && this.isSetTableName();
 +    boolean that_present_tableName = true && that.isSetTableName();
 +    if (this_present_tableName || that_present_tableName) {
 +      if (!(this_present_tableName && that_present_tableName))
 +        return false;
 +      if (!this.tableName.equals(that.tableName))
 +        return false;
 +    }
 +
 +    boolean this_present_partitions = true && this.isSetPartitions();
 +    boolean that_present_partitions = true && that.isSetPartitions();
 +    if (this_present_partitions || that_present_partitions) {
 +      if (!(this_present_partitions && that_present_partitions))
 +        return false;
 +      if (!this.partitions.equals(that.partitions))
 +        return false;
 +    }
 +
 +    boolean this_present_environmentContext = true && this.isSetEnvironmentContext();
 +    boolean that_present_environmentContext = true && that.isSetEnvironmentContext();
 +    if (this_present_environmentContext || that_present_environmentContext) {
 +      if (!(this_present_environmentContext && that_present_environmentContext))
 +        return false;
 +      if (!this.environmentContext.equals(that.environmentContext))
 +        return false;
 +    }
 +
 +    boolean this_present_txnId = true && this.isSetTxnId();
 +    boolean that_present_txnId = true && that.isSetTxnId();
 +    if (this_present_txnId || that_present_txnId) {
 +      if (!(this_present_txnId && that_present_txnId))
 +        return false;
 +      if (this.txnId != that.txnId)
 +        return false;
 +    }
 +
 +    boolean this_present_writeId = true && this.isSetWriteId();
 +    boolean that_present_writeId = true && that.isSetWriteId();
 +    if (this_present_writeId || that_present_writeId) {
 +      if (!(this_present_writeId && that_present_writeId))
 +        return false;
 +      if (this.writeId != that.writeId)
 +        return false;
 +    }
 +
 +    boolean this_present_validWriteIdList = true && this.isSetValidWriteIdList();
 +    boolean that_present_validWriteIdList = true && that.isSetValidWriteIdList();
 +    if (this_present_validWriteIdList || that_present_validWriteIdList) {
 +      if (!(this_present_validWriteIdList && that_present_validWriteIdList))
 +        return false;
 +      if (!this.validWriteIdList.equals(that.validWriteIdList))
 +        return false;
 +    }
 +
 +    return true;
 +  }
 +
 +  @Override
 +  public int hashCode() {
 +    List<Object> list = new ArrayList<Object>();
 +
 +    boolean present_dbName = true && (isSetDbName());
 +    list.add(present_dbName);
 +    if (present_dbName)
 +      list.add(dbName);
 +
 +    boolean present_tableName = true && (isSetTableName());
 +    list.add(present_tableName);
 +    if (present_tableName)
 +      list.add(tableName);
 +
 +    boolean present_partitions = true && (isSetPartitions());
 +    list.add(present_partitions);
 +    if (present_partitions)
 +      list.add(partitions);
 +
 +    boolean present_environmentContext = true && (isSetEnvironmentContext());
 +    list.add(present_environmentContext);
 +    if (present_environmentContext)
 +      list.add(environmentContext);
 +
 +    boolean present_txnId = true && (isSetTxnId());
 +    list.add(present_txnId);
 +    if (present_txnId)
 +      list.add(txnId);
 +
 +    boolean present_writeId = true && (isSetWriteId());
 +    list.add(present_writeId);
 +    if (present_writeId)
 +      list.add(writeId);
 +
 +    boolean present_validWriteIdList = true && (isSetValidWriteIdList());
 +    list.add(present_validWriteIdList);
 +    if (present_validWriteIdList)
 +      list.add(validWriteIdList);
 +
 +    return list.hashCode();
 +  }
 +
 +  @Override
 +  public int compareTo(AlterPartitionsRequest other) {
 +    if (!getClass().equals(other.getClass())) {
 +      return getClass().getName().compareTo(other.getClass().getName());
 +    }
 +
 +    int lastComparison = 0;
 +
 +    lastComparison = Boolean.valueOf(isSetDbName()).compareTo(other.isSetDbName());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetDbName()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.dbName, other.dbName);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetTableName()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    lastComparison = Boolean.valueOf(isSetPartitions()).compareTo(other.isSetPartitions());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetPartitions()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitions, other.partitions);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    lastComparison = Boolean.valueOf(isSetEnvironmentContext()).compareTo(other.isSetEnvironmentContext());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetEnvironmentContext()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.environmentContext, other.environmentContext);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    lastComparison = Boolean.valueOf(isSetTxnId()).compareTo(other.isSetTxnId());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetTxnId()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnId, other.txnId);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    lastComparison = Boolean.valueOf(isSetWriteId()).compareTo(other.isSetWriteId());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetWriteId()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeId, other.writeId);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    lastComparison = Boolean.valueOf(isSetValidWriteIdList()).compareTo(other.isSetValidWriteIdList());
 +    if (lastComparison != 0) {
 +      return lastComparison;
 +    }
 +    if (isSetValidWriteIdList()) {
 +      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.validWriteIdList, other.validWriteIdList);
 +      if (lastComparison != 0) {
 +        return lastComparison;
 +      }
 +    }
 +    return 0;
 +  }
 +
 +  public _Fields fieldForId(int fieldId) {
 +    return _Fields.findByThriftId(fieldId);
 +  }
 +
 +  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
 +    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
 +  }
 +
 +  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
 +    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
 +  }
 +
 +  @Override
 +  public String toString() {
 +    StringBuilder sb = new StringBuilder("AlterPartitionsRequest(");
 +    boolean first = true;
 +
 +    sb.append("dbName:");
 +    if (this.dbName == null) {
 +      sb.append("null");
 +    } else {
 +      sb.append(this.dbName);
 +    }
 +    first = false;
 +    if (!first) sb.append(", ");
 +    sb.append("tableName:");
 +    if (this.tableName == null) {
 +      sb.append("null");
 +    } else {
 +      sb.append(this.tableName);
 +    }
 +    first = false;
 +    if (!first) sb.append(", ");
 +    sb.append("partitions:");
 +    if (this.partitions == null) {
 +      sb.append("null");
 +    } else {
 +      sb.append(this.partitions);
 +    }
 +    first = false;
 +    if (!first) sb.append(", ");
 +    sb.append("environmentContext:");
 +    if (this.environmentContext == null) {
 +      sb.append("null");
 +    } else {
 +      sb.append(this.environmentContext);
 +    }
 +    first = false;
 +    if (isSetTxnId()) {
 +      if (!first) sb.append(", ");
 +      sb.append("txnId:");
 +      sb.append(this.txnId);
 +      first = false;
 +    }
 +    if (isSetWriteId()) {
 +      if (!first) sb.append(", ");
 +      sb.append("writeId:");
 +      sb.append(this.writeId);
 +      first = false;
 +    }
 +    if (isSetValidWriteIdList()) {
 +      if (!first) sb.append(", ");
 +      sb.append("validWriteIdList:");
 +      if (this.validWriteIdList == null) {
 +        sb.append("null");
 +      } else {
 +        sb.append(this.validWriteIdList);
 +      }
 +      first = false;
 +    }
 +    sb.append(")");
 +    return sb.toString();
 +  }
 +
 +  public void validate() throws org.apache.thrift.TException {
 +    // check for required fields
 +    if (!isSetDbName()) {
 +      throw new org.apache.thrift.protocol.TProtocolException("Required field 'dbName' is unset! Struct:" + toString());
 +    }
 +
 +    if (!isSetTableName()) {
 +      throw new org.apache.thrift.protocol.TProtocolException("Required field 'tableName' is unset! Struct:" + toString());
 +    }
 +
 +    if (!isSetPartitions()) {
 +      throw new org.apache.thrift.protocol.TProtocolException("Required field 'partitions' is unset! Struct:" + toString());
 +    }
 +
 +    if (!isSetEnvironmentContext()) {
 +      throw new org.apache.thrift.protocol.TProtocolException("Required field 'environmentContext' is unset! Struct:" + toString());
 +    }
 +
 +    // check for sub-struct validity
 +    if (environmentContext != null) {
 +      environmentContext.validate();
 +    }
 +  }
 +
 +  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
 +    try {
 +      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
 +    } catch (org.apache.thrift.TException te) {
 +      throw new java.io.IOException(te);
 +    }
 +  }
 +
 +  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
 +    try {
 +      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
 +      __isset_bitfield = 0;
 +      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
 +    } catch (org.apache.thrift.TException te) {
 +      throw new java.io.IOException(te);
 +    }
 +  }
 +
 +  private static class AlterPartitionsRequestStandardSchemeFactory implements SchemeFactory {
 +    public AlterPartitionsRequestStandardScheme getScheme() {
 +      return new AlterPartitionsRequestStandardScheme();
 +    }
 +  }
 +
 +  private static class AlterPartitionsRequestStandardScheme extends StandardScheme<AlterPartitionsRequest> {
 +
 +    public void read(org.apache.thrift.protocol.TProtocol iprot, AlterPartitionsRequest struct) throws org.apache.thrift.TException {
 +      org.apache.thrift.protocol.TField schemeField;
 +      iprot.readStructBegin();
 +      while (true)
 +      {
 +        schemeField = iprot.readFieldBegin();
 +        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
 +          break;
 +        }
 +        switch (schemeField.id) {
 +          case 1: // DB_NAME
 +            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
 +              struct.dbName = iprot.readString();
 +              struct.setDbNameIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          case 2: // TABLE_NAME
 +            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
 +              struct.tableName = iprot.readString();
 +              struct.setTableNameIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          case 3: // PARTITIONS
 +            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
 +              {
-                 org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
-                 struct.partitions = new ArrayList<Partition>(_list928.size);
-                 Partition _elem929;
-                 for (int _i930 = 0; _i930 < _list928.size; ++_i930)
++                org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
++                struct.partitions = new ArrayList<Partition>(_list952.size);
++                Partition _elem953;
++                for (int _i954 = 0; _i954 < _list952.size; ++_i954)
 +                {
-                   _elem929 = new Partition();
-                   _elem929.read(iprot);
-                   struct.partitions.add(_elem929);
++                  _elem953 = new Partition();
++                  _elem953.read(iprot);
++                  struct.partitions.add(_elem953);
 +                }
 +                iprot.readListEnd();
 +              }
 +              struct.setPartitionsIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          case 4: // ENVIRONMENT_CONTEXT
 +            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
 +              struct.environmentContext = new EnvironmentContext();
 +              struct.environmentContext.read(iprot);
 +              struct.setEnvironmentContextIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          case 5: // TXN_ID
 +            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
 +              struct.txnId = iprot.readI64();
 +              struct.setTxnIdIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          case 6: // WRITE_ID
 +            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
 +              struct.writeId = iprot.readI64();
 +              struct.setWriteIdIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          case 7: // VALID_WRITE_ID_LIST
 +            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
 +              struct.validWriteIdList = iprot.readString();
 +              struct.setValidWriteIdListIsSet(true);
 +            } else { 
 +              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +            }
 +            break;
 +          default:
 +            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
 +        }
 +        iprot.readFieldEnd();
 +      }
 +      iprot.readStructEnd();
 +      struct.validate();
 +    }
 +
 +    public void write(org.apache.thrift.protocol.TProtocol oprot, AlterPartitionsRequest struct) throws org.apache.thrift.TException {
 +      struct.validate();
 +
 +      oprot.writeStructBegin(STRUCT_DESC);
 +      if (struct.dbName != null) {
 +        oprot.writeFieldBegin(DB_NAME_FIELD_DESC);
 +        oprot.writeString(struct.dbName);
 +        oprot.writeFieldEnd();
 +      }
 +      if (struct.tableName != null) {
 +        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
 +        oprot.writeString(struct.tableName);
 +        oprot.writeFieldEnd();
 +      }
 +      if (struct.partitions != null) {
 +        oprot.writeFieldBegin(PARTITIONS_FIELD_DESC);
 +        {
 +          oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.partitions.size()));
-           for (Partition _iter931 : struct.partitions)
++          for (Partition _iter955 : struct.partitions)
 +          {
-             _iter931.write(oprot);
++            _iter955.write(oprot);
 +          }
 +          oprot.writeListEnd();
 +        }
 +        oprot.writeFieldEnd();
 +      }
 +      if (struct.environmentContext != null) {
 +        oprot.writeFieldBegin(ENVIRONMENT_CONTEXT_FIELD_DESC);
 +        struct.environmentContext.write(oprot);
 +        oprot.writeFieldEnd();
 +      }
 +      if (struct.isSetTxnId()) {
 +        oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
 +        oprot.writeI64(struct.txnId);
 +        oprot.writeFieldEnd();
 +      }
 +      if (struct.isSetWriteId()) {
 +        oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
 +        oprot.writeI64(struct.writeId);
 +        oprot.writeFieldEnd();
 +      }
 +      if (struct.validWriteIdList != null) {
 +        if (struct.isSetValidWriteIdList()) {
 +          oprot.writeFieldBegin(VALID_WRITE_ID_LIST_FIELD_DESC);
 +          oprot.writeString(struct.validWriteIdList);
 +          oprot.writeFieldEnd();
 +        }
 +      }
 +      oprot.writeFieldStop();
 +      oprot.writeStructEnd();
 +    }
 +
 +  }
 +
 +  private static class AlterPartitionsRequestTupleSchemeFactory implements SchemeFactory {
 +    public AlterPartitionsRequestTupleScheme getScheme() {
 +      return new AlterPartitionsRequestTupleScheme();
 +    }
 +  }
 +
 +  private static class AlterPartitionsRequestTupleScheme extends TupleScheme<AlterPartitionsRequest> {
 +
 +    @Override
 +    public void write(org.apache.thrift.protocol.TProtocol prot, AlterPartitionsRequest struct) throws org.apache.thrift.TException {
 +      TTupleProtocol oprot = (TTupleProtocol) prot;
 +      oprot.writeString(struct.dbName);
 +      oprot.writeString(struct.tableName);
 +      {
 +        oprot.writeI32(struct.partitions.size());
-         for (Partition _iter932 : struct.partitions)
++        for (Partition _iter956 : struct.partitions)
 +        {
-           _iter932.write(oprot);
++          _iter956.write(oprot);
 +        }
 +      }
 +      struct.environmentContext.write(oprot);
 +      BitSet optionals = new BitSet();
 +      if (struct.isSetTxnId()) {
 +        optionals.set(0);
 +      }
 +      if (struct.isSetWriteId()) {
 +        optionals.set(1);
 +      }
 +      if (struct.isSetValidWriteIdList()) {
 +        optionals.set(2);
 +      }
 +      oprot.writeBitSet(optionals, 3);
 +      if (struct.isSetTxnId()) {
 +        oprot.writeI64(struct.txnId);
 +      }
 +      if (struct.isSetWriteId()) {
 +        oprot.writeI64(struct.writeId);
 +      }
 +      if (struct.isSetValidWriteIdList()) {
 +        oprot.writeString(struct.validWriteIdList);
 +      }
 +    }
 +
 +    @Override
 +    public void read(org.apache.thrift.protocol.TProtocol prot, AlterPartitionsRequest struct) throws org.apache.thrift.TException {
 +      TTupleProtocol iprot = (TTupleProtocol) prot;
 +      struct.dbName = iprot.readString();
 +      struct.setDbNameIsSet(true);
 +      struct.tableName = iprot.readString();
 +      struct.setTableNameIsSet(true);
 +      {
-         org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-         struct.partitions = new ArrayList<Partition>(_list933.size);
-         Partition _elem934;
-         for (int _i935 = 0; _i935 < _list933.size; ++_i935)
++        org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++        struct.partitions = new ArrayList<Partition>(_list957.size);
++        Partition _elem958;
++        for (int _i959 = 0; _i959 < _list957.size; ++_i959)
 +        {
-           _elem934 = new Partition();
-           _elem934.read(iprot);
-           struct.partitions.add(_elem934);
++          _elem958 = new Partition();
++          _elem958.read(iprot);
++          struct.partitions.add(_elem958);
 +        }
 +      }
 +      struct.setPartitionsIsSet(true);
 +      struct.environmentContext = new EnvironmentContext();
 +      struct.environmentContext.read(iprot);
 +      struct.setEnvironmentContextIsSet(true);
 +      BitSet incoming = iprot.readBitSet(3);
 +      if (incoming.get(0)) {
 +        struct.txnId = iprot.readI64();
 +        struct.setTxnIdIsSet(true);
 +      }
 +      if (incoming.get(1)) {
 +        struct.writeId = iprot.readI64();
 +        struct.setWriteIdIsSet(true);
 +      }
 +      if (incoming.get(2)) {
 +        struct.validWriteIdList = iprot.readString();
 +        struct.setValidWriteIdListIsSet(true);
 +      }
 +    }
 +  }
 +
 +}
 +


[14/46] hive git commit: HIVE-20059: Hive streaming should try shade prefix unconditionally on exception (Prasanth Jayachandran reviewed by Vineet Garg)

Posted by se...@apache.org.
HIVE-20059: Hive streaming should try shade prefix unconditionally on exception (Prasanth Jayachandran reviewed by Vineet Garg)


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

Branch: refs/heads/master-txnstats
Commit: 1cedb120954679844473fe65ec755ee0f6a6e37e
Parents: d83a0be
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Mon Jul 2 14:47:47 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Mon Jul 2 14:47:47 2018 -0700

----------------------------------------------------------------------
 .../src/java/org/apache/hive/streaming/AbstractRecordWriter.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1cedb120/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
index 281f280..2980028 100644
--- a/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
+++ b/streaming/src/java/org/apache/hive/streaming/AbstractRecordWriter.java
@@ -162,7 +162,7 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     try {
       this.acidOutputFormat = (AcidOutputFormat<?, ?>) ReflectionUtils
         .newInstance(JavaUtils.loadClass(outFormatName), conf);
-    } catch (ClassNotFoundException e) {
+    } catch (Exception e) {
       String shadePrefix = conf.getVar(HiveConf.ConfVars.HIVE_CLASSLOADER_SHADE_PREFIX);
       if (shadePrefix != null && !shadePrefix.trim().isEmpty()) {
         try {


[42/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index c6ce900,ec129ef..10e2388
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@@ -42244,13 -42384,13 +42376,13 @@@ import org.slf4j.LoggerFactory
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
-                   struct.success = new ArrayList<String>(_list936.size);
-                   String _elem937;
-                   for (int _i938 = 0; _i938 < _list936.size; ++_i938)
 -                  org.apache.thrift.protocol.TList _list952 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list952.size);
 -                  String _elem953;
 -                  for (int _i954 = 0; _i954 < _list952.size; ++_i954)
++                  org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list960.size);
++                  String _elem961;
++                  for (int _i962 = 0; _i962 < _list960.size; ++_i962)
                    {
-                     _elem937 = iprot.readString();
-                     struct.success.add(_elem937);
 -                    _elem953 = iprot.readString();
 -                    struct.success.add(_elem953);
++                    _elem961 = iprot.readString();
++                    struct.success.add(_elem961);
                    }
                    iprot.readListEnd();
                  }
@@@ -42285,9 -42425,9 +42417,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-             for (String _iter939 : struct.success)
 -            for (String _iter955 : struct.success)
++            for (String _iter963 : struct.success)
              {
-               oprot.writeString(_iter939);
 -              oprot.writeString(_iter955);
++              oprot.writeString(_iter963);
              }
              oprot.writeListEnd();
            }
@@@ -42326,9 -42466,9 +42458,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (String _iter940 : struct.success)
 -            for (String _iter956 : struct.success)
++            for (String _iter964 : struct.success)
              {
-               oprot.writeString(_iter940);
 -              oprot.writeString(_iter956);
++              oprot.writeString(_iter964);
              }
            }
          }
@@@ -42343,13 -42483,13 +42475,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.success = new ArrayList<String>(_list941.size);
-             String _elem942;
-             for (int _i943 = 0; _i943 < _list941.size; ++_i943)
 -            org.apache.thrift.protocol.TList _list957 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list957.size);
 -            String _elem958;
 -            for (int _i959 = 0; _i959 < _list957.size; ++_i959)
++            org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list965.size);
++            String _elem966;
++            for (int _i967 = 0; _i967 < _list965.size; ++_i967)
              {
-               _elem942 = iprot.readString();
-               struct.success.add(_elem942);
 -              _elem958 = iprot.readString();
 -              struct.success.add(_elem958);
++              _elem966 = iprot.readString();
++              struct.success.add(_elem966);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -43003,13 -43143,13 +43135,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
-                   struct.success = new ArrayList<String>(_list944.size);
-                   String _elem945;
-                   for (int _i946 = 0; _i946 < _list944.size; ++_i946)
 -                  org.apache.thrift.protocol.TList _list960 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list960.size);
 -                  String _elem961;
 -                  for (int _i962 = 0; _i962 < _list960.size; ++_i962)
++                  org.apache.thrift.protocol.TList _list968 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list968.size);
++                  String _elem969;
++                  for (int _i970 = 0; _i970 < _list968.size; ++_i970)
                    {
-                     _elem945 = iprot.readString();
-                     struct.success.add(_elem945);
 -                    _elem961 = iprot.readString();
 -                    struct.success.add(_elem961);
++                    _elem969 = iprot.readString();
++                    struct.success.add(_elem969);
                    }
                    iprot.readListEnd();
                  }
@@@ -43044,9 -43184,9 +43176,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-             for (String _iter947 : struct.success)
 -            for (String _iter963 : struct.success)
++            for (String _iter971 : struct.success)
              {
-               oprot.writeString(_iter947);
 -              oprot.writeString(_iter963);
++              oprot.writeString(_iter971);
              }
              oprot.writeListEnd();
            }
@@@ -43085,9 -43225,9 +43217,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (String _iter948 : struct.success)
 -            for (String _iter964 : struct.success)
++            for (String _iter972 : struct.success)
              {
-               oprot.writeString(_iter948);
 -              oprot.writeString(_iter964);
++              oprot.writeString(_iter972);
              }
            }
          }
@@@ -43102,13 -43242,13 +43234,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list949 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.success = new ArrayList<String>(_list949.size);
-             String _elem950;
-             for (int _i951 = 0; _i951 < _list949.size; ++_i951)
 -            org.apache.thrift.protocol.TList _list965 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list965.size);
 -            String _elem966;
 -            for (int _i967 = 0; _i967 < _list965.size; ++_i967)
++            org.apache.thrift.protocol.TList _list973 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list973.size);
++            String _elem974;
++            for (int _i975 = 0; _i975 < _list973.size; ++_i975)
              {
-               _elem950 = iprot.readString();
-               struct.success.add(_elem950);
 -              _elem966 = iprot.readString();
 -              struct.success.add(_elem966);
++              _elem974 = iprot.readString();
++              struct.success.add(_elem974);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -47715,16 -47855,16 +47847,16 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
                  {
-                   org.apache.thrift.protocol.TMap _map952 = iprot.readMapBegin();
-                   struct.success = new HashMap<String,Type>(2*_map952.size);
-                   String _key953;
-                   Type _val954;
-                   for (int _i955 = 0; _i955 < _map952.size; ++_i955)
 -                  org.apache.thrift.protocol.TMap _map968 = iprot.readMapBegin();
 -                  struct.success = new HashMap<String,Type>(2*_map968.size);
 -                  String _key969;
 -                  Type _val970;
 -                  for (int _i971 = 0; _i971 < _map968.size; ++_i971)
++                  org.apache.thrift.protocol.TMap _map976 = iprot.readMapBegin();
++                  struct.success = new HashMap<String,Type>(2*_map976.size);
++                  String _key977;
++                  Type _val978;
++                  for (int _i979 = 0; _i979 < _map976.size; ++_i979)
                    {
-                     _key953 = iprot.readString();
-                     _val954 = new Type();
-                     _val954.read(iprot);
-                     struct.success.put(_key953, _val954);
 -                    _key969 = iprot.readString();
 -                    _val970 = new Type();
 -                    _val970.read(iprot);
 -                    struct.success.put(_key969, _val970);
++                    _key977 = iprot.readString();
++                    _val978 = new Type();
++                    _val978.read(iprot);
++                    struct.success.put(_key977, _val978);
                    }
                    iprot.readMapEnd();
                  }
@@@ -47759,10 -47899,10 +47891,10 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-             for (Map.Entry<String, Type> _iter956 : struct.success.entrySet())
 -            for (Map.Entry<String, Type> _iter972 : struct.success.entrySet())
++            for (Map.Entry<String, Type> _iter980 : struct.success.entrySet())
              {
-               oprot.writeString(_iter956.getKey());
-               _iter956.getValue().write(oprot);
 -              oprot.writeString(_iter972.getKey());
 -              _iter972.getValue().write(oprot);
++              oprot.writeString(_iter980.getKey());
++              _iter980.getValue().write(oprot);
              }
              oprot.writeMapEnd();
            }
@@@ -47801,10 -47941,10 +47933,10 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (Map.Entry<String, Type> _iter957 : struct.success.entrySet())
 -            for (Map.Entry<String, Type> _iter973 : struct.success.entrySet())
++            for (Map.Entry<String, Type> _iter981 : struct.success.entrySet())
              {
-               oprot.writeString(_iter957.getKey());
-               _iter957.getValue().write(oprot);
 -              oprot.writeString(_iter973.getKey());
 -              _iter973.getValue().write(oprot);
++              oprot.writeString(_iter981.getKey());
++              _iter981.getValue().write(oprot);
              }
            }
          }
@@@ -47819,16 -47959,16 +47951,16 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TMap _map958 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.success = new HashMap<String,Type>(2*_map958.size);
-             String _key959;
-             Type _val960;
-             for (int _i961 = 0; _i961 < _map958.size; ++_i961)
 -            org.apache.thrift.protocol.TMap _map974 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.success = new HashMap<String,Type>(2*_map974.size);
 -            String _key975;
 -            Type _val976;
 -            for (int _i977 = 0; _i977 < _map974.size; ++_i977)
++            org.apache.thrift.protocol.TMap _map982 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new HashMap<String,Type>(2*_map982.size);
++            String _key983;
++            Type _val984;
++            for (int _i985 = 0; _i985 < _map982.size; ++_i985)
              {
-               _key959 = iprot.readString();
-               _val960 = new Type();
-               _val960.read(iprot);
-               struct.success.put(_key959, _val960);
 -              _key975 = iprot.readString();
 -              _val976 = new Type();
 -              _val976.read(iprot);
 -              struct.success.put(_key975, _val976);
++              _key983 = iprot.readString();
++              _val984 = new Type();
++              _val984.read(iprot);
++              struct.success.put(_key983, _val984);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -48863,14 -49003,14 +48995,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list962 = iprot.readListBegin();
-                   struct.success = new ArrayList<FieldSchema>(_list962.size);
-                   FieldSchema _elem963;
-                   for (int _i964 = 0; _i964 < _list962.size; ++_i964)
 -                  org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
 -                  struct.success = new ArrayList<FieldSchema>(_list978.size);
 -                  FieldSchema _elem979;
 -                  for (int _i980 = 0; _i980 < _list978.size; ++_i980)
++                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
++                  struct.success = new ArrayList<FieldSchema>(_list986.size);
++                  FieldSchema _elem987;
++                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
                    {
-                     _elem963 = new FieldSchema();
-                     _elem963.read(iprot);
-                     struct.success.add(_elem963);
 -                    _elem979 = new FieldSchema();
 -                    _elem979.read(iprot);
 -                    struct.success.add(_elem979);
++                    _elem987 = new FieldSchema();
++                    _elem987.read(iprot);
++                    struct.success.add(_elem987);
                    }
                    iprot.readListEnd();
                  }
@@@ -48923,9 -49063,9 +49055,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-             for (FieldSchema _iter965 : struct.success)
 -            for (FieldSchema _iter981 : struct.success)
++            for (FieldSchema _iter989 : struct.success)
              {
-               _iter965.write(oprot);
 -              _iter981.write(oprot);
++              _iter989.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -48980,9 -49120,9 +49112,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (FieldSchema _iter966 : struct.success)
 -            for (FieldSchema _iter982 : struct.success)
++            for (FieldSchema _iter990 : struct.success)
              {
-               _iter966.write(oprot);
 -              _iter982.write(oprot);
++              _iter990.write(oprot);
              }
            }
          }
@@@ -49003,14 -49143,14 +49135,14 @@@
          BitSet incoming = iprot.readBitSet(4);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list967 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.success = new ArrayList<FieldSchema>(_list967.size);
-             FieldSchema _elem968;
-             for (int _i969 = 0; _i969 < _list967.size; ++_i969)
 -            org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.success = new ArrayList<FieldSchema>(_list983.size);
 -            FieldSchema _elem984;
 -            for (int _i985 = 0; _i985 < _list983.size; ++_i985)
++            org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<FieldSchema>(_list991.size);
++            FieldSchema _elem992;
++            for (int _i993 = 0; _i993 < _list991.size; ++_i993)
              {
-               _elem968 = new FieldSchema();
-               _elem968.read(iprot);
-               struct.success.add(_elem968);
 -              _elem984 = new FieldSchema();
 -              _elem984.read(iprot);
 -              struct.success.add(_elem984);
++              _elem992 = new FieldSchema();
++              _elem992.read(iprot);
++              struct.success.add(_elem992);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -50164,14 -50304,14 +50296,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list970 = iprot.readListBegin();
-                   struct.success = new ArrayList<FieldSchema>(_list970.size);
-                   FieldSchema _elem971;
-                   for (int _i972 = 0; _i972 < _list970.size; ++_i972)
 -                  org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
 -                  struct.success = new ArrayList<FieldSchema>(_list986.size);
 -                  FieldSchema _elem987;
 -                  for (int _i988 = 0; _i988 < _list986.size; ++_i988)
++                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
++                  struct.success = new ArrayList<FieldSchema>(_list994.size);
++                  FieldSchema _elem995;
++                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
                    {
-                     _elem971 = new FieldSchema();
-                     _elem971.read(iprot);
-                     struct.success.add(_elem971);
 -                    _elem987 = new FieldSchema();
 -                    _elem987.read(iprot);
 -                    struct.success.add(_elem987);
++                    _elem995 = new FieldSchema();
++                    _elem995.read(iprot);
++                    struct.success.add(_elem995);
                    }
                    iprot.readListEnd();
                  }
@@@ -50224,9 -50364,9 +50356,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-             for (FieldSchema _iter973 : struct.success)
 -            for (FieldSchema _iter989 : struct.success)
++            for (FieldSchema _iter997 : struct.success)
              {
-               _iter973.write(oprot);
 -              _iter989.write(oprot);
++              _iter997.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -50281,9 -50421,9 +50413,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (FieldSchema _iter974 : struct.success)
 -            for (FieldSchema _iter990 : struct.success)
++            for (FieldSchema _iter998 : struct.success)
              {
-               _iter974.write(oprot);
 -              _iter990.write(oprot);
++              _iter998.write(oprot);
              }
            }
          }
@@@ -50304,14 -50444,14 +50436,14 @@@
          BitSet incoming = iprot.readBitSet(4);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list975 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.success = new ArrayList<FieldSchema>(_list975.size);
-             FieldSchema _elem976;
-             for (int _i977 = 0; _i977 < _list975.size; ++_i977)
 -            org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.success = new ArrayList<FieldSchema>(_list991.size);
 -            FieldSchema _elem992;
 -            for (int _i993 = 0; _i993 < _list991.size; ++_i993)
++            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<FieldSchema>(_list999.size);
++            FieldSchema _elem1000;
++            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
              {
-               _elem976 = new FieldSchema();
-               _elem976.read(iprot);
-               struct.success.add(_elem976);
 -              _elem992 = new FieldSchema();
 -              _elem992.read(iprot);
 -              struct.success.add(_elem992);
++              _elem1000 = new FieldSchema();
++              _elem1000.read(iprot);
++              struct.success.add(_elem1000);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -51356,14 -51496,14 +51488,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list978 = iprot.readListBegin();
-                   struct.success = new ArrayList<FieldSchema>(_list978.size);
-                   FieldSchema _elem979;
-                   for (int _i980 = 0; _i980 < _list978.size; ++_i980)
 -                  org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
 -                  struct.success = new ArrayList<FieldSchema>(_list994.size);
 -                  FieldSchema _elem995;
 -                  for (int _i996 = 0; _i996 < _list994.size; ++_i996)
++                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
++                  struct.success = new ArrayList<FieldSchema>(_list1002.size);
++                  FieldSchema _elem1003;
++                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
                    {
-                     _elem979 = new FieldSchema();
-                     _elem979.read(iprot);
-                     struct.success.add(_elem979);
 -                    _elem995 = new FieldSchema();
 -                    _elem995.read(iprot);
 -                    struct.success.add(_elem995);
++                    _elem1003 = new FieldSchema();
++                    _elem1003.read(iprot);
++                    struct.success.add(_elem1003);
                    }
                    iprot.readListEnd();
                  }
@@@ -51416,9 -51556,9 +51548,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-             for (FieldSchema _iter981 : struct.success)
 -            for (FieldSchema _iter997 : struct.success)
++            for (FieldSchema _iter1005 : struct.success)
              {
-               _iter981.write(oprot);
 -              _iter997.write(oprot);
++              _iter1005.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -51473,9 -51613,9 +51605,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (FieldSchema _iter982 : struct.success)
 -            for (FieldSchema _iter998 : struct.success)
++            for (FieldSchema _iter1006 : struct.success)
              {
-               _iter982.write(oprot);
 -              _iter998.write(oprot);
++              _iter1006.write(oprot);
              }
            }
          }
@@@ -51496,14 -51636,14 +51628,14 @@@
          BitSet incoming = iprot.readBitSet(4);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list983 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.success = new ArrayList<FieldSchema>(_list983.size);
-             FieldSchema _elem984;
-             for (int _i985 = 0; _i985 < _list983.size; ++_i985)
 -            org.apache.thrift.protocol.TList _list999 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.success = new ArrayList<FieldSchema>(_list999.size);
 -            FieldSchema _elem1000;
 -            for (int _i1001 = 0; _i1001 < _list999.size; ++_i1001)
++            org.apache.thrift.protocol.TList _list1007 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<FieldSchema>(_list1007.size);
++            FieldSchema _elem1008;
++            for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009)
              {
-               _elem984 = new FieldSchema();
-               _elem984.read(iprot);
-               struct.success.add(_elem984);
 -              _elem1000 = new FieldSchema();
 -              _elem1000.read(iprot);
 -              struct.success.add(_elem1000);
++              _elem1008 = new FieldSchema();
++              _elem1008.read(iprot);
++              struct.success.add(_elem1008);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -52657,14 -52797,14 +52789,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list986 = iprot.readListBegin();
-                   struct.success = new ArrayList<FieldSchema>(_list986.size);
-                   FieldSchema _elem987;
-                   for (int _i988 = 0; _i988 < _list986.size; ++_i988)
 -                  org.apache.thrift.protocol.TList _list1002 = iprot.readListBegin();
 -                  struct.success = new ArrayList<FieldSchema>(_list1002.size);
 -                  FieldSchema _elem1003;
 -                  for (int _i1004 = 0; _i1004 < _list1002.size; ++_i1004)
++                  org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin();
++                  struct.success = new ArrayList<FieldSchema>(_list1010.size);
++                  FieldSchema _elem1011;
++                  for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012)
                    {
-                     _elem987 = new FieldSchema();
-                     _elem987.read(iprot);
-                     struct.success.add(_elem987);
 -                    _elem1003 = new FieldSchema();
 -                    _elem1003.read(iprot);
 -                    struct.success.add(_elem1003);
++                    _elem1011 = new FieldSchema();
++                    _elem1011.read(iprot);
++                    struct.success.add(_elem1011);
                    }
                    iprot.readListEnd();
                  }
@@@ -52717,9 -52857,9 +52849,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-             for (FieldSchema _iter989 : struct.success)
 -            for (FieldSchema _iter1005 : struct.success)
++            for (FieldSchema _iter1013 : struct.success)
              {
-               _iter989.write(oprot);
 -              _iter1005.write(oprot);
++              _iter1013.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -52774,9 -52914,9 +52906,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (FieldSchema _iter990 : struct.success)
 -            for (FieldSchema _iter1006 : struct.success)
++            for (FieldSchema _iter1014 : struct.success)
              {
-               _iter990.write(oprot);
 -              _iter1006.write(oprot);
++              _iter1014.write(oprot);
              }
            }
          }
@@@ -52797,14 -52937,14 +52929,14 @@@
          BitSet incoming = iprot.readBitSet(4);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list991 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.success = new ArrayList<FieldSchema>(_list991.size);
-             FieldSchema _elem992;
-             for (int _i993 = 0; _i993 < _list991.size; ++_i993)
 -            org.apache.thrift.protocol.TList _list1007 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.success = new ArrayList<FieldSchema>(_list1007.size);
 -            FieldSchema _elem1008;
 -            for (int _i1009 = 0; _i1009 < _list1007.size; ++_i1009)
++            org.apache.thrift.protocol.TList _list1015 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<FieldSchema>(_list1015.size);
++            FieldSchema _elem1016;
++            for (int _i1017 = 0; _i1017 < _list1015.size; ++_i1017)
              {
-               _elem992 = new FieldSchema();
-               _elem992.read(iprot);
-               struct.success.add(_elem992);
 -              _elem1008 = new FieldSchema();
 -              _elem1008.read(iprot);
 -              struct.success.add(_elem1008);
++              _elem1016 = new FieldSchema();
++              _elem1016.read(iprot);
++              struct.success.add(_elem1016);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -55933,14 -56073,14 +56065,14 @@@
              case 2: // PRIMARY_KEYS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list994 = iprot.readListBegin();
-                   struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list994.size);
-                   SQLPrimaryKey _elem995;
-                   for (int _i996 = 0; _i996 < _list994.size; ++_i996)
 -                  org.apache.thrift.protocol.TList _list1010 = iprot.readListBegin();
 -                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1010.size);
 -                  SQLPrimaryKey _elem1011;
 -                  for (int _i1012 = 0; _i1012 < _list1010.size; ++_i1012)
++                  org.apache.thrift.protocol.TList _list1018 = iprot.readListBegin();
++                  struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1018.size);
++                  SQLPrimaryKey _elem1019;
++                  for (int _i1020 = 0; _i1020 < _list1018.size; ++_i1020)
                    {
-                     _elem995 = new SQLPrimaryKey();
-                     _elem995.read(iprot);
-                     struct.primaryKeys.add(_elem995);
 -                    _elem1011 = new SQLPrimaryKey();
 -                    _elem1011.read(iprot);
 -                    struct.primaryKeys.add(_elem1011);
++                    _elem1019 = new SQLPrimaryKey();
++                    _elem1019.read(iprot);
++                    struct.primaryKeys.add(_elem1019);
                    }
                    iprot.readListEnd();
                  }
@@@ -55952,14 -56092,14 +56084,14 @@@
              case 3: // FOREIGN_KEYS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list997 = iprot.readListBegin();
-                   struct.foreignKeys = new ArrayList<SQLForeignKey>(_list997.size);
-                   SQLForeignKey _elem998;
-                   for (int _i999 = 0; _i999 < _list997.size; ++_i999)
 -                  org.apache.thrift.protocol.TList _list1013 = iprot.readListBegin();
 -                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1013.size);
 -                  SQLForeignKey _elem1014;
 -                  for (int _i1015 = 0; _i1015 < _list1013.size; ++_i1015)
++                  org.apache.thrift.protocol.TList _list1021 = iprot.readListBegin();
++                  struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1021.size);
++                  SQLForeignKey _elem1022;
++                  for (int _i1023 = 0; _i1023 < _list1021.size; ++_i1023)
                    {
-                     _elem998 = new SQLForeignKey();
-                     _elem998.read(iprot);
-                     struct.foreignKeys.add(_elem998);
 -                    _elem1014 = new SQLForeignKey();
 -                    _elem1014.read(iprot);
 -                    struct.foreignKeys.add(_elem1014);
++                    _elem1022 = new SQLForeignKey();
++                    _elem1022.read(iprot);
++                    struct.foreignKeys.add(_elem1022);
                    }
                    iprot.readListEnd();
                  }
@@@ -55971,14 -56111,14 +56103,14 @@@
              case 4: // UNIQUE_CONSTRAINTS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1000 = iprot.readListBegin();
-                   struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1000.size);
-                   SQLUniqueConstraint _elem1001;
-                   for (int _i1002 = 0; _i1002 < _list1000.size; ++_i1002)
 -                  org.apache.thrift.protocol.TList _list1016 = iprot.readListBegin();
 -                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1016.size);
 -                  SQLUniqueConstraint _elem1017;
 -                  for (int _i1018 = 0; _i1018 < _list1016.size; ++_i1018)
++                  org.apache.thrift.protocol.TList _list1024 = iprot.readListBegin();
++                  struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1024.size);
++                  SQLUniqueConstraint _elem1025;
++                  for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026)
                    {
-                     _elem1001 = new SQLUniqueConstraint();
-                     _elem1001.read(iprot);
-                     struct.uniqueConstraints.add(_elem1001);
 -                    _elem1017 = new SQLUniqueConstraint();
 -                    _elem1017.read(iprot);
 -                    struct.uniqueConstraints.add(_elem1017);
++                    _elem1025 = new SQLUniqueConstraint();
++                    _elem1025.read(iprot);
++                    struct.uniqueConstraints.add(_elem1025);
                    }
                    iprot.readListEnd();
                  }
@@@ -55990,14 -56130,14 +56122,14 @@@
              case 5: // NOT_NULL_CONSTRAINTS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1003 = iprot.readListBegin();
-                   struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1003.size);
-                   SQLNotNullConstraint _elem1004;
-                   for (int _i1005 = 0; _i1005 < _list1003.size; ++_i1005)
 -                  org.apache.thrift.protocol.TList _list1019 = iprot.readListBegin();
 -                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1019.size);
 -                  SQLNotNullConstraint _elem1020;
 -                  for (int _i1021 = 0; _i1021 < _list1019.size; ++_i1021)
++                  org.apache.thrift.protocol.TList _list1027 = iprot.readListBegin();
++                  struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1027.size);
++                  SQLNotNullConstraint _elem1028;
++                  for (int _i1029 = 0; _i1029 < _list1027.size; ++_i1029)
                    {
-                     _elem1004 = new SQLNotNullConstraint();
-                     _elem1004.read(iprot);
-                     struct.notNullConstraints.add(_elem1004);
 -                    _elem1020 = new SQLNotNullConstraint();
 -                    _elem1020.read(iprot);
 -                    struct.notNullConstraints.add(_elem1020);
++                    _elem1028 = new SQLNotNullConstraint();
++                    _elem1028.read(iprot);
++                    struct.notNullConstraints.add(_elem1028);
                    }
                    iprot.readListEnd();
                  }
@@@ -56009,14 -56149,14 +56141,14 @@@
              case 6: // DEFAULT_CONSTRAINTS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1006 = iprot.readListBegin();
-                   struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1006.size);
-                   SQLDefaultConstraint _elem1007;
-                   for (int _i1008 = 0; _i1008 < _list1006.size; ++_i1008)
 -                  org.apache.thrift.protocol.TList _list1022 = iprot.readListBegin();
 -                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1022.size);
 -                  SQLDefaultConstraint _elem1023;
 -                  for (int _i1024 = 0; _i1024 < _list1022.size; ++_i1024)
++                  org.apache.thrift.protocol.TList _list1030 = iprot.readListBegin();
++                  struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1030.size);
++                  SQLDefaultConstraint _elem1031;
++                  for (int _i1032 = 0; _i1032 < _list1030.size; ++_i1032)
                    {
-                     _elem1007 = new SQLDefaultConstraint();
-                     _elem1007.read(iprot);
-                     struct.defaultConstraints.add(_elem1007);
 -                    _elem1023 = new SQLDefaultConstraint();
 -                    _elem1023.read(iprot);
 -                    struct.defaultConstraints.add(_elem1023);
++                    _elem1031 = new SQLDefaultConstraint();
++                    _elem1031.read(iprot);
++                    struct.defaultConstraints.add(_elem1031);
                    }
                    iprot.readListEnd();
                  }
@@@ -56028,14 -56168,14 +56160,14 @@@
              case 7: // CHECK_CONSTRAINTS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1009 = iprot.readListBegin();
-                   struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1009.size);
-                   SQLCheckConstraint _elem1010;
-                   for (int _i1011 = 0; _i1011 < _list1009.size; ++_i1011)
 -                  org.apache.thrift.protocol.TList _list1025 = iprot.readListBegin();
 -                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1025.size);
 -                  SQLCheckConstraint _elem1026;
 -                  for (int _i1027 = 0; _i1027 < _list1025.size; ++_i1027)
++                  org.apache.thrift.protocol.TList _list1033 = iprot.readListBegin();
++                  struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1033.size);
++                  SQLCheckConstraint _elem1034;
++                  for (int _i1035 = 0; _i1035 < _list1033.size; ++_i1035)
                    {
-                     _elem1010 = new SQLCheckConstraint();
-                     _elem1010.read(iprot);
-                     struct.checkConstraints.add(_elem1010);
 -                    _elem1026 = new SQLCheckConstraint();
 -                    _elem1026.read(iprot);
 -                    struct.checkConstraints.add(_elem1026);
++                    _elem1034 = new SQLCheckConstraint();
++                    _elem1034.read(iprot);
++                    struct.checkConstraints.add(_elem1034);
                    }
                    iprot.readListEnd();
                  }
@@@ -56066,9 -56206,9 +56198,9 @@@
            oprot.writeFieldBegin(PRIMARY_KEYS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.primaryKeys.size()));
-             for (SQLPrimaryKey _iter1012 : struct.primaryKeys)
 -            for (SQLPrimaryKey _iter1028 : struct.primaryKeys)
++            for (SQLPrimaryKey _iter1036 : struct.primaryKeys)
              {
-               _iter1012.write(oprot);
 -              _iter1028.write(oprot);
++              _iter1036.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -56078,9 -56218,9 +56210,9 @@@
            oprot.writeFieldBegin(FOREIGN_KEYS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.foreignKeys.size()));
-             for (SQLForeignKey _iter1013 : struct.foreignKeys)
 -            for (SQLForeignKey _iter1029 : struct.foreignKeys)
++            for (SQLForeignKey _iter1037 : struct.foreignKeys)
              {
-               _iter1013.write(oprot);
 -              _iter1029.write(oprot);
++              _iter1037.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -56090,9 -56230,9 +56222,9 @@@
            oprot.writeFieldBegin(UNIQUE_CONSTRAINTS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.uniqueConstraints.size()));
-             for (SQLUniqueConstraint _iter1014 : struct.uniqueConstraints)
 -            for (SQLUniqueConstraint _iter1030 : struct.uniqueConstraints)
++            for (SQLUniqueConstraint _iter1038 : struct.uniqueConstraints)
              {
-               _iter1014.write(oprot);
 -              _iter1030.write(oprot);
++              _iter1038.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -56102,9 -56242,9 +56234,9 @@@
            oprot.writeFieldBegin(NOT_NULL_CONSTRAINTS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.notNullConstraints.size()));
-             for (SQLNotNullConstraint _iter1015 : struct.notNullConstraints)
 -            for (SQLNotNullConstraint _iter1031 : struct.notNullConstraints)
++            for (SQLNotNullConstraint _iter1039 : struct.notNullConstraints)
              {
-               _iter1015.write(oprot);
 -              _iter1031.write(oprot);
++              _iter1039.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -56114,9 -56254,9 +56246,9 @@@
            oprot.writeFieldBegin(DEFAULT_CONSTRAINTS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.defaultConstraints.size()));
-             for (SQLDefaultConstraint _iter1016 : struct.defaultConstraints)
 -            for (SQLDefaultConstraint _iter1032 : struct.defaultConstraints)
++            for (SQLDefaultConstraint _iter1040 : struct.defaultConstraints)
              {
-               _iter1016.write(oprot);
 -              _iter1032.write(oprot);
++              _iter1040.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -56126,9 -56266,9 +56258,9 @@@
            oprot.writeFieldBegin(CHECK_CONSTRAINTS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.checkConstraints.size()));
-             for (SQLCheckConstraint _iter1017 : struct.checkConstraints)
 -            for (SQLCheckConstraint _iter1033 : struct.checkConstraints)
++            for (SQLCheckConstraint _iter1041 : struct.checkConstraints)
              {
-               _iter1017.write(oprot);
 -              _iter1033.write(oprot);
++              _iter1041.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -56180,54 -56320,54 +56312,54 @@@
          if (struct.isSetPrimaryKeys()) {
            {
              oprot.writeI32(struct.primaryKeys.size());
-             for (SQLPrimaryKey _iter1018 : struct.primaryKeys)
 -            for (SQLPrimaryKey _iter1034 : struct.primaryKeys)
++            for (SQLPrimaryKey _iter1042 : struct.primaryKeys)
              {
-               _iter1018.write(oprot);
 -              _iter1034.write(oprot);
++              _iter1042.write(oprot);
              }
            }
          }
          if (struct.isSetForeignKeys()) {
            {
              oprot.writeI32(struct.foreignKeys.size());
-             for (SQLForeignKey _iter1019 : struct.foreignKeys)
 -            for (SQLForeignKey _iter1035 : struct.foreignKeys)
++            for (SQLForeignKey _iter1043 : struct.foreignKeys)
              {
-               _iter1019.write(oprot);
 -              _iter1035.write(oprot);
++              _iter1043.write(oprot);
              }
            }
          }
          if (struct.isSetUniqueConstraints()) {
            {
              oprot.writeI32(struct.uniqueConstraints.size());
-             for (SQLUniqueConstraint _iter1020 : struct.uniqueConstraints)
 -            for (SQLUniqueConstraint _iter1036 : struct.uniqueConstraints)
++            for (SQLUniqueConstraint _iter1044 : struct.uniqueConstraints)
              {
-               _iter1020.write(oprot);
 -              _iter1036.write(oprot);
++              _iter1044.write(oprot);
              }
            }
          }
          if (struct.isSetNotNullConstraints()) {
            {
              oprot.writeI32(struct.notNullConstraints.size());
-             for (SQLNotNullConstraint _iter1021 : struct.notNullConstraints)
 -            for (SQLNotNullConstraint _iter1037 : struct.notNullConstraints)
++            for (SQLNotNullConstraint _iter1045 : struct.notNullConstraints)
              {
-               _iter1021.write(oprot);
 -              _iter1037.write(oprot);
++              _iter1045.write(oprot);
              }
            }
          }
          if (struct.isSetDefaultConstraints()) {
            {
              oprot.writeI32(struct.defaultConstraints.size());
-             for (SQLDefaultConstraint _iter1022 : struct.defaultConstraints)
 -            for (SQLDefaultConstraint _iter1038 : struct.defaultConstraints)
++            for (SQLDefaultConstraint _iter1046 : struct.defaultConstraints)
              {
-               _iter1022.write(oprot);
 -              _iter1038.write(oprot);
++              _iter1046.write(oprot);
              }
            }
          }
          if (struct.isSetCheckConstraints()) {
            {
              oprot.writeI32(struct.checkConstraints.size());
-             for (SQLCheckConstraint _iter1023 : struct.checkConstraints)
 -            for (SQLCheckConstraint _iter1039 : struct.checkConstraints)
++            for (SQLCheckConstraint _iter1047 : struct.checkConstraints)
              {
-               _iter1023.write(oprot);
 -              _iter1039.write(oprot);
++              _iter1047.write(oprot);
              }
            }
          }
@@@ -56244,84 -56384,84 +56376,84 @@@
          }
          if (incoming.get(1)) {
            {
-             org.apache.thrift.protocol.TList _list1024 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1024.size);
-             SQLPrimaryKey _elem1025;
-             for (int _i1026 = 0; _i1026 < _list1024.size; ++_i1026)
 -            org.apache.thrift.protocol.TList _list1040 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1040.size);
 -            SQLPrimaryKey _elem1041;
 -            for (int _i1042 = 0; _i1042 < _list1040.size; ++_i1042)
++            org.apache.thrift.protocol.TList _list1048 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.primaryKeys = new ArrayList<SQLPrimaryKey>(_list1048.size);
++            SQLPrimaryKey _elem1049;
++            for (int _i1050 = 0; _i1050 < _list1048.size; ++_i1050)
              {
-               _elem1025 = new SQLPrimaryKey();
-               _elem1025.read(iprot);
-               struct.primaryKeys.add(_elem1025);
 -              _elem1041 = new SQLPrimaryKey();
 -              _elem1041.read(iprot);
 -              struct.primaryKeys.add(_elem1041);
++              _elem1049 = new SQLPrimaryKey();
++              _elem1049.read(iprot);
++              struct.primaryKeys.add(_elem1049);
              }
            }
            struct.setPrimaryKeysIsSet(true);
          }
          if (incoming.get(2)) {
            {
-             org.apache.thrift.protocol.TList _list1027 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1027.size);
-             SQLForeignKey _elem1028;
-             for (int _i1029 = 0; _i1029 < _list1027.size; ++_i1029)
 -            org.apache.thrift.protocol.TList _list1043 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1043.size);
 -            SQLForeignKey _elem1044;
 -            for (int _i1045 = 0; _i1045 < _list1043.size; ++_i1045)
++            org.apache.thrift.protocol.TList _list1051 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.foreignKeys = new ArrayList<SQLForeignKey>(_list1051.size);
++            SQLForeignKey _elem1052;
++            for (int _i1053 = 0; _i1053 < _list1051.size; ++_i1053)
              {
-               _elem1028 = new SQLForeignKey();
-               _elem1028.read(iprot);
-               struct.foreignKeys.add(_elem1028);
 -              _elem1044 = new SQLForeignKey();
 -              _elem1044.read(iprot);
 -              struct.foreignKeys.add(_elem1044);
++              _elem1052 = new SQLForeignKey();
++              _elem1052.read(iprot);
++              struct.foreignKeys.add(_elem1052);
              }
            }
            struct.setForeignKeysIsSet(true);
          }
          if (incoming.get(3)) {
            {
-             org.apache.thrift.protocol.TList _list1030 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1030.size);
-             SQLUniqueConstraint _elem1031;
-             for (int _i1032 = 0; _i1032 < _list1030.size; ++_i1032)
 -            org.apache.thrift.protocol.TList _list1046 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1046.size);
 -            SQLUniqueConstraint _elem1047;
 -            for (int _i1048 = 0; _i1048 < _list1046.size; ++_i1048)
++            org.apache.thrift.protocol.TList _list1054 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.uniqueConstraints = new ArrayList<SQLUniqueConstraint>(_list1054.size);
++            SQLUniqueConstraint _elem1055;
++            for (int _i1056 = 0; _i1056 < _list1054.size; ++_i1056)
              {
-               _elem1031 = new SQLUniqueConstraint();
-               _elem1031.read(iprot);
-               struct.uniqueConstraints.add(_elem1031);
 -              _elem1047 = new SQLUniqueConstraint();
 -              _elem1047.read(iprot);
 -              struct.uniqueConstraints.add(_elem1047);
++              _elem1055 = new SQLUniqueConstraint();
++              _elem1055.read(iprot);
++              struct.uniqueConstraints.add(_elem1055);
              }
            }
            struct.setUniqueConstraintsIsSet(true);
          }
          if (incoming.get(4)) {
            {
-             org.apache.thrift.protocol.TList _list1033 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1033.size);
-             SQLNotNullConstraint _elem1034;
-             for (int _i1035 = 0; _i1035 < _list1033.size; ++_i1035)
 -            org.apache.thrift.protocol.TList _list1049 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1049.size);
 -            SQLNotNullConstraint _elem1050;
 -            for (int _i1051 = 0; _i1051 < _list1049.size; ++_i1051)
++            org.apache.thrift.protocol.TList _list1057 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.notNullConstraints = new ArrayList<SQLNotNullConstraint>(_list1057.size);
++            SQLNotNullConstraint _elem1058;
++            for (int _i1059 = 0; _i1059 < _list1057.size; ++_i1059)
              {
-               _elem1034 = new SQLNotNullConstraint();
-               _elem1034.read(iprot);
-               struct.notNullConstraints.add(_elem1034);
 -              _elem1050 = new SQLNotNullConstraint();
 -              _elem1050.read(iprot);
 -              struct.notNullConstraints.add(_elem1050);
++              _elem1058 = new SQLNotNullConstraint();
++              _elem1058.read(iprot);
++              struct.notNullConstraints.add(_elem1058);
              }
            }
            struct.setNotNullConstraintsIsSet(true);
          }
          if (incoming.get(5)) {
            {
-             org.apache.thrift.protocol.TList _list1036 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1036.size);
-             SQLDefaultConstraint _elem1037;
-             for (int _i1038 = 0; _i1038 < _list1036.size; ++_i1038)
 -            org.apache.thrift.protocol.TList _list1052 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1052.size);
 -            SQLDefaultConstraint _elem1053;
 -            for (int _i1054 = 0; _i1054 < _list1052.size; ++_i1054)
++            org.apache.thrift.protocol.TList _list1060 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.defaultConstraints = new ArrayList<SQLDefaultConstraint>(_list1060.size);
++            SQLDefaultConstraint _elem1061;
++            for (int _i1062 = 0; _i1062 < _list1060.size; ++_i1062)
              {
-               _elem1037 = new SQLDefaultConstraint();
-               _elem1037.read(iprot);
-               struct.defaultConstraints.add(_elem1037);
 -              _elem1053 = new SQLDefaultConstraint();
 -              _elem1053.read(iprot);
 -              struct.defaultConstraints.add(_elem1053);
++              _elem1061 = new SQLDefaultConstraint();
++              _elem1061.read(iprot);
++              struct.defaultConstraints.add(_elem1061);
              }
            }
            struct.setDefaultConstraintsIsSet(true);
          }
          if (incoming.get(6)) {
            {
-             org.apache.thrift.protocol.TList _list1039 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1039.size);
-             SQLCheckConstraint _elem1040;
-             for (int _i1041 = 0; _i1041 < _list1039.size; ++_i1041)
 -            org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1055.size);
 -            SQLCheckConstraint _elem1056;
 -            for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
++            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.checkConstraints = new ArrayList<SQLCheckConstraint>(_list1063.size);
++            SQLCheckConstraint _elem1064;
++            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
              {
-               _elem1040 = new SQLCheckConstraint();
-               _elem1040.read(iprot);
-               struct.checkConstraints.add(_elem1040);
 -              _elem1056 = new SQLCheckConstraint();
 -              _elem1056.read(iprot);
 -              struct.checkConstraints.add(_elem1056);
++              _elem1064 = new SQLCheckConstraint();
++              _elem1064.read(iprot);
++              struct.checkConstraints.add(_elem1064);
              }
            }
            struct.setCheckConstraintsIsSet(true);
@@@ -65471,13 -65611,13 +65603,13 @@@
              case 3: // PART_NAMES
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1042 = iprot.readListBegin();
-                   struct.partNames = new ArrayList<String>(_list1042.size);
-                   String _elem1043;
-                   for (int _i1044 = 0; _i1044 < _list1042.size; ++_i1044)
 -                  org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
 -                  struct.partNames = new ArrayList<String>(_list1058.size);
 -                  String _elem1059;
 -                  for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
++                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
++                  struct.partNames = new ArrayList<String>(_list1066.size);
++                  String _elem1067;
++                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
                    {
-                     _elem1043 = iprot.readString();
-                     struct.partNames.add(_elem1043);
 -                    _elem1059 = iprot.readString();
 -                    struct.partNames.add(_elem1059);
++                    _elem1067 = iprot.readString();
++                    struct.partNames.add(_elem1067);
                    }
                    iprot.readListEnd();
                  }
@@@ -65513,9 -65653,9 +65645,9 @@@
            oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-             for (String _iter1045 : struct.partNames)
 -            for (String _iter1061 : struct.partNames)
++            for (String _iter1069 : struct.partNames)
              {
-               oprot.writeString(_iter1045);
 -              oprot.writeString(_iter1061);
++              oprot.writeString(_iter1069);
              }
              oprot.writeListEnd();
            }
@@@ -65558,9 -65698,9 +65690,9 @@@
          if (struct.isSetPartNames()) {
            {
              oprot.writeI32(struct.partNames.size());
-             for (String _iter1046 : struct.partNames)
 -            for (String _iter1062 : struct.partNames)
++            for (String _iter1070 : struct.partNames)
              {
-               oprot.writeString(_iter1046);
 -              oprot.writeString(_iter1062);
++              oprot.writeString(_iter1070);
              }
            }
          }
@@@ -65580,13 -65720,13 +65712,13 @@@
          }
          if (incoming.get(2)) {
            {
-             org.apache.thrift.protocol.TList _list1047 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.partNames = new ArrayList<String>(_list1047.size);
-             String _elem1048;
-             for (int _i1049 = 0; _i1049 < _list1047.size; ++_i1049)
 -            org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.partNames = new ArrayList<String>(_list1063.size);
 -            String _elem1064;
 -            for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
++            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.partNames = new ArrayList<String>(_list1071.size);
++            String _elem1072;
++            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
              {
-               _elem1048 = iprot.readString();
-               struct.partNames.add(_elem1048);
 -              _elem1064 = iprot.readString();
 -              struct.partNames.add(_elem1064);
++              _elem1072 = iprot.readString();
++              struct.partNames.add(_elem1072);
              }
            }
            struct.setPartNamesIsSet(true);
@@@ -66811,13 -66951,13 +66943,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1050 = iprot.readListBegin();
-                   struct.success = new ArrayList<String>(_list1050.size);
-                   String _elem1051;
-                   for (int _i1052 = 0; _i1052 < _list1050.size; ++_i1052)
 -                  org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list1066.size);
 -                  String _elem1067;
 -                  for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
++                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list1074.size);
++                  String _elem1075;
++                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
                    {
-                     _elem1051 = iprot.readString();
-                     struct.success.add(_elem1051);
 -                    _elem1067 = iprot.readString();
 -                    struct.success.add(_elem1067);
++                    _elem1075 = iprot.readString();
++                    struct.success.add(_elem1075);
                    }
                    iprot.readListEnd();
                  }
@@@ -66852,9 -66992,9 +66984,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-             for (String _iter1053 : struct.success)
 -            for (String _iter1069 : struct.success)
++            for (String _iter1077 : struct.success)
              {
-               oprot.writeString(_iter1053);
 -              oprot.writeString(_iter1069);
++              oprot.writeString(_iter1077);
              }
              oprot.writeListEnd();
            }
@@@ -66893,9 -67033,9 +67025,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (String _iter1054 : struct.success)
 -            for (String _iter1070 : struct.success)
++            for (String _iter1078 : struct.success)
              {
-               oprot.writeString(_iter1054);
 -              oprot.writeString(_iter1070);
++              oprot.writeString(_iter1078);
              }
            }
          }
@@@ -66910,13 -67050,13 +67042,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list1055 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.success = new ArrayList<String>(_list1055.size);
-             String _elem1056;
-             for (int _i1057 = 0; _i1057 < _list1055.size; ++_i1057)
 -            org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list1071.size);
 -            String _elem1072;
 -            for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
++            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list1079.size);
++            String _elem1080;
++            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
              {
-               _elem1056 = iprot.readString();
-               struct.success.add(_elem1056);
 -              _elem1072 = iprot.readString();
 -              struct.success.add(_elem1072);
++              _elem1080 = iprot.readString();
++              struct.success.add(_elem1080);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -67890,13 -68030,13 +68022,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1058 = iprot.readListBegin();
-                   struct.success = new ArrayList<String>(_list1058.size);
-                   String _elem1059;
-                   for (int _i1060 = 0; _i1060 < _list1058.size; ++_i1060)
 -                  org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list1074.size);
 -                  String _elem1075;
 -                  for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
++                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list1082.size);
++                  String _elem1083;
++                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
                    {
-                     _elem1059 = iprot.readString();
-                     struct.success.add(_elem1059);
 -                    _elem1075 = iprot.readString();
 -                    struct.success.add(_elem1075);
++                    _elem1083 = iprot.readString();
++                    struct.success.add(_elem1083);
                    }
                    iprot.readListEnd();
                  }
@@@ -67931,9 -68071,9 +68063,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-             for (String _iter1061 : struct.success)
 -            for (String _iter1077 : struct.success)
++            for (String _iter1085 : struct.success)
              {
-               oprot.writeString(_iter1061);
 -              oprot.writeString(_iter1077);
++              oprot.writeString(_iter1085);
              }
              oprot.writeListEnd();
            }
@@@ -67972,9 -68112,9 +68104,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (String _iter1062 : struct.success)
 -            for (String _iter1078 : struct.success)
++            for (String _iter1086 : struct.success)
              {
-               oprot.writeString(_iter1062);
 -              oprot.writeString(_iter1078);
++              oprot.writeString(_iter1086);
              }
            }
          }
@@@ -67989,13 -68129,13 +68121,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list1063 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.success = new ArrayList<String>(_list1063.size);
-             String _elem1064;
-             for (int _i1065 = 0; _i1065 < _list1063.size; ++_i1065)
 -            org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list1079.size);
 -            String _elem1080;
 -            for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
++            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list1087.size);
++            String _elem1088;
++            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
              {
-               _elem1064 = iprot.readString();
-               struct.success.add(_elem1064);
 -              _elem1080 = iprot.readString();
 -              struct.success.add(_elem1080);
++              _elem1088 = iprot.readString();
++              struct.success.add(_elem1088);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -68761,13 -68901,13 +68893,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1066 = iprot.readListBegin();
-                   struct.success = new ArrayList<String>(_list1066.size);
-                   String _elem1067;
-                   for (int _i1068 = 0; _i1068 < _list1066.size; ++_i1068)
 -                  org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list1082.size);
 -                  String _elem1083;
 -                  for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
++                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list1090.size);
++                  String _elem1091;
++                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
                    {
-                     _elem1067 = iprot.readString();
-                     struct.success.add(_elem1067);
 -                    _elem1083 = iprot.readString();
 -                    struct.success.add(_elem1083);
++                    _elem1091 = iprot.readString();
++                    struct.success.add(_elem1091);
                    }
                    iprot.readListEnd();
                  }
@@@ -68802,9 -68942,9 +68934,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-             for (String _iter1069 : struct.success)
 -            for (String _iter1085 : struct.success)
++            for (String _iter1093 : struct.success)
              {
-               oprot.writeString(_iter1069);
 -              oprot.writeString(_iter1085);
++              oprot.writeString(_iter1093);
              }
              oprot.writeListEnd();
            }
@@@ -68843,9 -68983,9 +68975,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (String _iter1070 : struct.success)
 -            for (String _iter1086 : struct.success)
++            for (String _iter1094 : struct.success)
              {
-               oprot.writeString(_iter1070);
 -              oprot.writeString(_iter1086);
++              oprot.writeString(_iter1094);
              }
            }
          }
@@@ -68860,13 -69000,13 +68992,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list1071 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.success = new ArrayList<String>(_list1071.size);
-             String _elem1072;
-             for (int _i1073 = 0; _i1073 < _list1071.size; ++_i1073)
 -            org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list1087.size);
 -            String _elem1088;
 -            for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
++            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list1095.size);
++            String _elem1096;
++            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
              {
-               _elem1072 = iprot.readString();
-               struct.success.add(_elem1072);
 -              _elem1088 = iprot.readString();
 -              struct.success.add(_elem1088);
++              _elem1096 = iprot.readString();
++              struct.success.add(_elem1096);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -69371,13 -69511,13 +69503,13 @@@
              case 3: // TBL_TYPES
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1074 = iprot.readListBegin();
-                   struct.tbl_types = new ArrayList<String>(_list1074.size);
-                   String _elem1075;
-                   for (int _i1076 = 0; _i1076 < _list1074.size; ++_i1076)
 -                  org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
 -                  struct.tbl_types = new ArrayList<String>(_list1090.size);
 -                  String _elem1091;
 -                  for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
++                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
++                  struct.tbl_types = new ArrayList<String>(_list1098.size);
++                  String _elem1099;
++                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
                    {
-                     _elem1075 = iprot.readString();
-                     struct.tbl_types.add(_elem1075);
 -                    _elem1091 = iprot.readString();
 -                    struct.tbl_types.add(_elem1091);
++                    _elem1099 = iprot.readString();
++                    struct.tbl_types.add(_elem1099);
                    }
                    iprot.readListEnd();
                  }
@@@ -69413,9 -69553,9 +69545,9 @@@
            oprot.writeFieldBegin(TBL_TYPES_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_types.size()));
-             for (String _iter1077 : struct.tbl_types)
 -            for (String _iter1093 : struct.tbl_types)
++            for (String _iter1101 : struct.tbl_types)
              {
-               oprot.writeString(_iter1077);
 -              oprot.writeString(_iter1093);
++              oprot.writeString(_iter1101);
              }
              oprot.writeListEnd();
            }
@@@ -69458,9 -69598,9 +69590,9 @@@
          if (struct.isSetTbl_types()) {
            {
              oprot.writeI32(struct.tbl_types.size());
-             for (String _iter1078 : struct.tbl_types)
 -            for (String _iter1094 : struct.tbl_types)
++            for (String _iter1102 : struct.tbl_types)
              {
-               oprot.writeString(_iter1078);
 -              oprot.writeString(_iter1094);
++              oprot.writeString(_iter1102);
              }
            }
          }
@@@ -69480,13 -69620,13 +69612,13 @@@
          }
          if (incoming.get(2)) {
            {
-             org.apache.thrift.protocol.TList _list1079 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.tbl_types = new ArrayList<String>(_list1079.size);
-             String _elem1080;
-             for (int _i1081 = 0; _i1081 < _list1079.size; ++_i1081)
 -            org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.tbl_types = new ArrayList<String>(_list1095.size);
 -            String _elem1096;
 -            for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
++            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.tbl_types = new ArrayList<String>(_list1103.size);
++            String _elem1104;
++            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
              {
-               _elem1080 = iprot.readString();
-               struct.tbl_types.add(_elem1080);
 -              _elem1096 = iprot.readString();
 -              struct.tbl_types.add(_elem1096);
++              _elem1104 = iprot.readString();
++              struct.tbl_types.add(_elem1104);
              }
            }
            struct.setTbl_typesIsSet(true);
@@@ -69892,14 -70032,14 +70024,14 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1082 = iprot.readListBegin();
-                   struct.success = new ArrayList<TableMeta>(_list1082.size);
-                   TableMeta _elem1083;
-                   for (int _i1084 = 0; _i1084 < _list1082.size; ++_i1084)
 -                  org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
 -                  struct.success = new ArrayList<TableMeta>(_list1098.size);
 -                  TableMeta _elem1099;
 -                  for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
++                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
++                  struct.success = new ArrayList<TableMeta>(_list1106.size);
++                  TableMeta _elem1107;
++                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
                    {
-                     _elem1083 = new TableMeta();
-                     _elem1083.read(iprot);
-                     struct.success.add(_elem1083);
 -                    _elem1099 = new TableMeta();
 -                    _elem1099.read(iprot);
 -                    struct.success.add(_elem1099);
++                    _elem1107 = new TableMeta();
++                    _elem1107.read(iprot);
++                    struct.success.add(_elem1107);
                    }
                    iprot.readListEnd();
                  }
@@@ -69934,9 -70074,9 +70066,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.success.size()));
-             for (TableMeta _iter1085 : struct.success)
 -            for (TableMeta _iter1101 : struct.success)
++            for (TableMeta _iter1109 : struct.success)
              {
-               _iter1085.write(oprot);
 -              _iter1101.write(oprot);
++              _iter1109.write(oprot);
              }
              oprot.writeListEnd();
            }
@@@ -69975,9 -70115,9 +70107,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (TableMeta _iter1086 : struct.success)
 -            for (TableMeta _iter1102 : struct.success)
++            for (TableMeta _iter1110 : struct.success)
              {
-               _iter1086.write(oprot);
 -              _iter1102.write(oprot);
++              _iter1110.write(oprot);
              }
            }
          }
@@@ -69992,14 -70132,14 +70124,14 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list1087 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-             struct.success = new ArrayList<TableMeta>(_list1087.size);
-             TableMeta _elem1088;
-             for (int _i1089 = 0; _i1089 < _list1087.size; ++_i1089)
 -            org.apache.thrift.protocol.TList _list1103 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
 -            struct.success = new ArrayList<TableMeta>(_list1103.size);
 -            TableMeta _elem1104;
 -            for (int _i1105 = 0; _i1105 < _list1103.size; ++_i1105)
++            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
++            struct.success = new ArrayList<TableMeta>(_list1111.size);
++            TableMeta _elem1112;
++            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
              {
-               _elem1088 = new TableMeta();
-               _elem1088.read(iprot);
-               struct.success.add(_elem1088);
 -              _elem1104 = new TableMeta();
 -              _elem1104.read(iprot);
 -              struct.success.add(_elem1104);
++              _elem1112 = new TableMeta();
++              _elem1112.read(iprot);
++              struct.success.add(_elem1112);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -70765,13 -70905,13 +70897,13 @@@
              case 0: // SUCCESS
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1090 = iprot.readListBegin();
-                   struct.success = new ArrayList<String>(_list1090.size);
-                   String _elem1091;
-                   for (int _i1092 = 0; _i1092 < _list1090.size; ++_i1092)
 -                  org.apache.thrift.protocol.TList _list1106 = iprot.readListBegin();
 -                  struct.success = new ArrayList<String>(_list1106.size);
 -                  String _elem1107;
 -                  for (int _i1108 = 0; _i1108 < _list1106.size; ++_i1108)
++                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
++                  struct.success = new ArrayList<String>(_list1114.size);
++                  String _elem1115;
++                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
                    {
-                     _elem1091 = iprot.readString();
-                     struct.success.add(_elem1091);
 -                    _elem1107 = iprot.readString();
 -                    struct.success.add(_elem1107);
++                    _elem1115 = iprot.readString();
++                    struct.success.add(_elem1115);
                    }
                    iprot.readListEnd();
                  }
@@@ -70806,9 -70946,9 +70938,9 @@@
            oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-             for (String _iter1093 : struct.success)
 -            for (String _iter1109 : struct.success)
++            for (String _iter1117 : struct.success)
              {
-               oprot.writeString(_iter1093);
 -              oprot.writeString(_iter1109);
++              oprot.writeString(_iter1117);
              }
              oprot.writeListEnd();
            }
@@@ -70847,9 -70987,9 +70979,9 @@@
          if (struct.isSetSuccess()) {
            {
              oprot.writeI32(struct.success.size());
-             for (String _iter1094 : struct.success)
 -            for (String _iter1110 : struct.success)
++            for (String _iter1118 : struct.success)
              {
-               oprot.writeString(_iter1094);
 -              oprot.writeString(_iter1110);
++              oprot.writeString(_iter1118);
              }
            }
          }
@@@ -70864,13 -71004,13 +70996,13 @@@
          BitSet incoming = iprot.readBitSet(2);
          if (incoming.get(0)) {
            {
-             org.apache.thrift.protocol.TList _list1095 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-             struct.success = new ArrayList<String>(_list1095.size);
-             String _elem1096;
-             for (int _i1097 = 0; _i1097 < _list1095.size; ++_i1097)
 -            org.apache.thrift.protocol.TList _list1111 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
 -            struct.success = new ArrayList<String>(_list1111.size);
 -            String _elem1112;
 -            for (int _i1113 = 0; _i1113 < _list1111.size; ++_i1113)
++            org.apache.thrift.protocol.TList _list1119 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
++            struct.success = new ArrayList<String>(_list1119.size);
++            String _elem1120;
++            for (int _i1121 = 0; _i1121 < _list1119.size; ++_i1121)
              {
-               _elem1096 = iprot.readString();
-               struct.success.add(_elem1096);
 -              _elem1112 = iprot.readString();
 -              struct.success.add(_elem1112);
++              _elem1120 = iprot.readString();
++              struct.success.add(_elem1120);
              }
            }
            struct.setSuccessIsSet(true);
@@@ -72323,13 -72463,13 +72455,13 @@@
              case 2: // TBL_NAMES
                if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                  {
-                   org.apache.thrift.protocol.TList _list1098 = iprot.readListBegin();
-                   struct.tbl_names = new ArrayList<String>(_list1098.size);
-                   String _elem1099;
-                   for (int _i1100 = 0; _i1100 < _list1098.size; ++_i1100)
 -                  org.apache.thrift.protocol.TList _list1114 = iprot.readListBegin();
 -                  struct.tbl_names = new ArrayList<String>(_list1114.size);
 -                  String _elem1115;
 -                  for (int _i1116 = 0; _i1116 < _list1114.size; ++_i1116)
++                  org.apache.thrift.protocol.TList _list1122 = iprot.readListBegin();
++                  struct.tbl_names = new ArrayList<String>(_list1122.size);
++                  String _elem1123;
++                  for (int _i1124 = 0; _i1124 < _list1122.size; ++_i1124)
                    {
-                     _elem1099 = iprot.readString();
-                     struct.tbl_names.add(_elem1099);
 -                    _elem1115 = iprot.readString();
 -                    struct.tbl_names.add(_elem1115);
++                    _elem1123 = iprot.readString();
++                    struct.tbl_names.add(_elem1123);
                    }
                    iprot.readListEnd();
                  }
@@@ -72360,9 -72500,9 +72492,9 @@@
            oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
            {
              oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tbl_names.size()));
-             for (String _iter1101 : struct.tbl_names)
 -            for (String _iter1117 : struct.tbl_names)
++            for (String _iter1125 : struct.tbl_names)
              {
-               oprot.writeString(_iter1101);
 -              oprot.writeString(_iter1117);
++              oprot.writeString(_iter1125);
              }
              oprot.writeListEnd();
            }
@@@ -72399,9 -72539,9 +72531,9 @@@
          if (struct.isSetTbl_names()) {
            {
              oprot.writeI32(struct.tbl_names.size());
-

<TRUNCATED>

[39/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 9b02951,031e72b..6e0e598
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@@ -22074,213 -22043,6 +22463,213 @@@ class GetRuntimeStatsRequest
      value = (value * 31) ^ hash(self.maxWeight)
      value = (value * 31) ^ hash(self.maxCreateTime)
      return value
 +
 +  def __repr__(self):
 +    L = ['%s=%r' % (key, value)
 +      for key, value in self.__dict__.iteritems()]
 +    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
 +
 +  def __eq__(self, other):
 +    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
 +
 +  def __ne__(self, other):
 +    return not (self == other)
 +
 +class AlterPartitionsRequest:
 +  """
 +  Attributes:
 +   - dbName
 +   - tableName
 +   - partitions
 +   - environmentContext
 +   - txnId
 +   - writeId
 +   - validWriteIdList
 +  """
 +
 +  thrift_spec = (
 +    None, # 0
 +    (1, TType.STRING, 'dbName', None, None, ), # 1
 +    (2, TType.STRING, 'tableName', None, None, ), # 2
 +    (3, TType.LIST, 'partitions', (TType.STRUCT,(Partition, Partition.thrift_spec)), None, ), # 3
 +    (4, TType.STRUCT, 'environmentContext', (EnvironmentContext, EnvironmentContext.thrift_spec), None, ), # 4
 +    (5, TType.I64, 'txnId', None, -1, ), # 5
 +    (6, TType.I64, 'writeId', None, -1, ), # 6
 +    (7, TType.STRING, 'validWriteIdList', None, None, ), # 7
 +  )
 +
 +  def __init__(self, dbName=None, tableName=None, partitions=None, environmentContext=None, txnId=thrift_spec[5][4], writeId=thrift_spec[6][4], validWriteIdList=None,):
 +    self.dbName = dbName
 +    self.tableName = tableName
 +    self.partitions = partitions
 +    self.environmentContext = environmentContext
 +    self.txnId = txnId
 +    self.writeId = writeId
 +    self.validWriteIdList = validWriteIdList
 +
 +  def read(self, iprot):
 +    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
 +      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
 +      return
 +    iprot.readStructBegin()
 +    while True:
 +      (fname, ftype, fid) = iprot.readFieldBegin()
 +      if ftype == TType.STOP:
 +        break
 +      if fid == 1:
 +        if ftype == TType.STRING:
 +          self.dbName = iprot.readString()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 2:
 +        if ftype == TType.STRING:
 +          self.tableName = iprot.readString()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 3:
 +        if ftype == TType.LIST:
 +          self.partitions = []
-           (_etype819, _size816) = iprot.readListBegin()
-           for _i820 in xrange(_size816):
-             _elem821 = Partition()
-             _elem821.read(iprot)
-             self.partitions.append(_elem821)
++          (_etype840, _size837) = iprot.readListBegin()
++          for _i841 in xrange(_size837):
++            _elem842 = Partition()
++            _elem842.read(iprot)
++            self.partitions.append(_elem842)
 +          iprot.readListEnd()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 4:
 +        if ftype == TType.STRUCT:
 +          self.environmentContext = EnvironmentContext()
 +          self.environmentContext.read(iprot)
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 5:
 +        if ftype == TType.I64:
 +          self.txnId = iprot.readI64()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 6:
 +        if ftype == TType.I64:
 +          self.writeId = iprot.readI64()
 +        else:
 +          iprot.skip(ftype)
 +      elif fid == 7:
 +        if ftype == TType.STRING:
 +          self.validWriteIdList = iprot.readString()
 +        else:
 +          iprot.skip(ftype)
 +      else:
 +        iprot.skip(ftype)
 +      iprot.readFieldEnd()
 +    iprot.readStructEnd()
 +
 +  def write(self, oprot):
 +    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
 +      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
 +      return
 +    oprot.writeStructBegin('AlterPartitionsRequest')
 +    if self.dbName is not None:
 +      oprot.writeFieldBegin('dbName', TType.STRING, 1)
 +      oprot.writeString(self.dbName)
 +      oprot.writeFieldEnd()
 +    if self.tableName is not None:
 +      oprot.writeFieldBegin('tableName', TType.STRING, 2)
 +      oprot.writeString(self.tableName)
 +      oprot.writeFieldEnd()
 +    if self.partitions is not None:
 +      oprot.writeFieldBegin('partitions', TType.LIST, 3)
 +      oprot.writeListBegin(TType.STRUCT, len(self.partitions))
-       for iter822 in self.partitions:
-         iter822.write(oprot)
++      for iter843 in self.partitions:
++        iter843.write(oprot)
 +      oprot.writeListEnd()
 +      oprot.writeFieldEnd()
 +    if self.environmentContext is not None:
 +      oprot.writeFieldBegin('environmentContext', TType.STRUCT, 4)
 +      self.environmentContext.write(oprot)
 +      oprot.writeFieldEnd()
 +    if self.txnId is not None:
 +      oprot.writeFieldBegin('txnId', TType.I64, 5)
 +      oprot.writeI64(self.txnId)
 +      oprot.writeFieldEnd()
 +    if self.writeId is not None:
 +      oprot.writeFieldBegin('writeId', TType.I64, 6)
 +      oprot.writeI64(self.writeId)
 +      oprot.writeFieldEnd()
 +    if self.validWriteIdList is not None:
 +      oprot.writeFieldBegin('validWriteIdList', TType.STRING, 7)
 +      oprot.writeString(self.validWriteIdList)
 +      oprot.writeFieldEnd()
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    if self.dbName is None:
 +      raise TProtocol.TProtocolException(message='Required field dbName is unset!')
 +    if self.tableName is None:
 +      raise TProtocol.TProtocolException(message='Required field tableName is unset!')
 +    if self.partitions is None:
 +      raise TProtocol.TProtocolException(message='Required field partitions is unset!')
 +    if self.environmentContext is None:
 +      raise TProtocol.TProtocolException(message='Required field environmentContext is unset!')
 +    return
 +
 +
 +  def __hash__(self):
 +    value = 17
 +    value = (value * 31) ^ hash(self.dbName)
 +    value = (value * 31) ^ hash(self.tableName)
 +    value = (value * 31) ^ hash(self.partitions)
 +    value = (value * 31) ^ hash(self.environmentContext)
 +    value = (value * 31) ^ hash(self.txnId)
 +    value = (value * 31) ^ hash(self.writeId)
 +    value = (value * 31) ^ hash(self.validWriteIdList)
 +    return value
 +
 +  def __repr__(self):
 +    L = ['%s=%r' % (key, value)
 +      for key, value in self.__dict__.iteritems()]
 +    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
 +
 +  def __eq__(self, other):
 +    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
 +
 +  def __ne__(self, other):
 +    return not (self == other)
 +
 +class AlterPartitionsResponse:
 +
 +  thrift_spec = (
 +  )
 +
 +  def read(self, iprot):
 +    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
 +      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
 +      return
 +    iprot.readStructBegin()
 +    while True:
 +      (fname, ftype, fid) = iprot.readFieldBegin()
 +      if ftype == TType.STOP:
 +        break
 +      else:
 +        iprot.skip(ftype)
 +      iprot.readFieldEnd()
 +    iprot.readStructEnd()
 +
 +  def write(self, oprot):
 +    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
 +      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
 +      return
 +    oprot.writeStructBegin('AlterPartitionsResponse')
 +    oprot.writeFieldStop()
 +    oprot.writeStructEnd()
 +
 +  def validate(self):
 +    return
 +
 +
 +  def __hash__(self):
 +    value = 17
 +    return value
  
    def __repr__(self):
      L = ['%s=%r' % (key, value)

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --cc standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index 5b767bc,71f5034..c9c6b30
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@@ -338,5 -340,3 +340,4 @@@ ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR
  -- These lines need to be last.  Insert any changes above.
  UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
  SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;
 +
- ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR2(10) NULL;

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --cc standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index 70771cd,2454479..8729883
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@@ -73,18 -85,7 +73,19 @@@ import org.apache.hadoop.hive.metastore
  import org.apache.hadoop.hive.metastore.api.UnknownTableException;
  import org.apache.hadoop.hive.metastore.api.WMMapping;
  import org.apache.hadoop.hive.metastore.api.WMPool;
 +import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 +import org.apache.hadoop.hive.metastore.api.ISchemaName;
 +import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 +
 +import java.nio.ByteBuffer;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +
 +import org.apache.hadoop.conf.Configurable;
 +import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
  import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
  import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
  import org.apache.thrift.TException;

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClientPreCatalog.java
----------------------------------------------------------------------


[41/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index 38895e3,a83017b..ebf2448
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@@ -15438,14 -15496,14 +15494,14 @@@ class ThriftHiveMetastore_get_databases
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size827 = 0;
-             $_etype830 = 0;
-             $xfer += $input->readListBegin($_etype830, $_size827);
-             for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
 -            $_size841 = 0;
 -            $_etype844 = 0;
 -            $xfer += $input->readListBegin($_etype844, $_size841);
 -            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
++            $_size848 = 0;
++            $_etype851 = 0;
++            $xfer += $input->readListBegin($_etype851, $_size848);
++            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
              {
-               $elem832 = null;
-               $xfer += $input->readString($elem832);
-               $this->success []= $elem832;
 -              $elem846 = null;
 -              $xfer += $input->readString($elem846);
 -              $this->success []= $elem846;
++              $elem853 = null;
++              $xfer += $input->readString($elem853);
++              $this->success []= $elem853;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -15481,9 -15539,9 +15537,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter833)
 -          foreach ($this->success as $iter847)
++          foreach ($this->success as $iter854)
            {
-             $xfer += $output->writeString($iter833);
 -            $xfer += $output->writeString($iter847);
++            $xfer += $output->writeString($iter854);
            }
          }
          $output->writeListEnd();
@@@ -15614,14 -15672,14 +15670,14 @@@ class ThriftHiveMetastore_get_all_datab
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size834 = 0;
-             $_etype837 = 0;
-             $xfer += $input->readListBegin($_etype837, $_size834);
-             for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
 -            $_size848 = 0;
 -            $_etype851 = 0;
 -            $xfer += $input->readListBegin($_etype851, $_size848);
 -            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
++            $_size855 = 0;
++            $_etype858 = 0;
++            $xfer += $input->readListBegin($_etype858, $_size855);
++            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
              {
-               $elem839 = null;
-               $xfer += $input->readString($elem839);
-               $this->success []= $elem839;
 -              $elem853 = null;
 -              $xfer += $input->readString($elem853);
 -              $this->success []= $elem853;
++              $elem860 = null;
++              $xfer += $input->readString($elem860);
++              $this->success []= $elem860;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -15657,9 -15715,9 +15713,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter840)
 -          foreach ($this->success as $iter854)
++          foreach ($this->success as $iter861)
            {
-             $xfer += $output->writeString($iter840);
 -            $xfer += $output->writeString($iter854);
++            $xfer += $output->writeString($iter861);
            }
          }
          $output->writeListEnd();
@@@ -16660,18 -16718,18 +16716,18 @@@ class ThriftHiveMetastore_get_type_all_
          case 0:
            if ($ftype == TType::MAP) {
              $this->success = array();
-             $_size841 = 0;
-             $_ktype842 = 0;
-             $_vtype843 = 0;
-             $xfer += $input->readMapBegin($_ktype842, $_vtype843, $_size841);
-             for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
 -            $_size855 = 0;
 -            $_ktype856 = 0;
 -            $_vtype857 = 0;
 -            $xfer += $input->readMapBegin($_ktype856, $_vtype857, $_size855);
 -            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
++            $_size862 = 0;
++            $_ktype863 = 0;
++            $_vtype864 = 0;
++            $xfer += $input->readMapBegin($_ktype863, $_vtype864, $_size862);
++            for ($_i866 = 0; $_i866 < $_size862; ++$_i866)
              {
-               $key846 = '';
-               $val847 = new \metastore\Type();
-               $xfer += $input->readString($key846);
-               $val847 = new \metastore\Type();
-               $xfer += $val847->read($input);
-               $this->success[$key846] = $val847;
 -              $key860 = '';
 -              $val861 = new \metastore\Type();
 -              $xfer += $input->readString($key860);
 -              $val861 = new \metastore\Type();
 -              $xfer += $val861->read($input);
 -              $this->success[$key860] = $val861;
++              $key867 = '';
++              $val868 = new \metastore\Type();
++              $xfer += $input->readString($key867);
++              $val868 = new \metastore\Type();
++              $xfer += $val868->read($input);
++              $this->success[$key867] = $val868;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -16707,10 -16765,10 +16763,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $kiter848 => $viter849)
 -          foreach ($this->success as $kiter862 => $viter863)
++          foreach ($this->success as $kiter869 => $viter870)
            {
-             $xfer += $output->writeString($kiter848);
-             $xfer += $viter849->write($output);
 -            $xfer += $output->writeString($kiter862);
 -            $xfer += $viter863->write($output);
++            $xfer += $output->writeString($kiter869);
++            $xfer += $viter870->write($output);
            }
          }
          $output->writeMapEnd();
@@@ -16914,15 -16972,15 +16970,15 @@@ class ThriftHiveMetastore_get_fields_re
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size850 = 0;
-             $_etype853 = 0;
-             $xfer += $input->readListBegin($_etype853, $_size850);
-             for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
 -            $_size864 = 0;
 -            $_etype867 = 0;
 -            $xfer += $input->readListBegin($_etype867, $_size864);
 -            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
++            $_size871 = 0;
++            $_etype874 = 0;
++            $xfer += $input->readListBegin($_etype874, $_size871);
++            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
              {
-               $elem855 = null;
-               $elem855 = new \metastore\FieldSchema();
-               $xfer += $elem855->read($input);
-               $this->success []= $elem855;
 -              $elem869 = null;
 -              $elem869 = new \metastore\FieldSchema();
 -              $xfer += $elem869->read($input);
 -              $this->success []= $elem869;
++              $elem876 = null;
++              $elem876 = new \metastore\FieldSchema();
++              $xfer += $elem876->read($input);
++              $this->success []= $elem876;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -16974,9 -17032,9 +17030,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter856)
 -          foreach ($this->success as $iter870)
++          foreach ($this->success as $iter877)
            {
-             $xfer += $iter856->write($output);
 -            $xfer += $iter870->write($output);
++            $xfer += $iter877->write($output);
            }
          }
          $output->writeListEnd();
@@@ -17218,15 -17276,15 +17274,15 @@@ class ThriftHiveMetastore_get_fields_wi
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size857 = 0;
-             $_etype860 = 0;
-             $xfer += $input->readListBegin($_etype860, $_size857);
-             for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
 -            $_size871 = 0;
 -            $_etype874 = 0;
 -            $xfer += $input->readListBegin($_etype874, $_size871);
 -            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
++            $_size878 = 0;
++            $_etype881 = 0;
++            $xfer += $input->readListBegin($_etype881, $_size878);
++            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
              {
-               $elem862 = null;
-               $elem862 = new \metastore\FieldSchema();
-               $xfer += $elem862->read($input);
-               $this->success []= $elem862;
 -              $elem876 = null;
 -              $elem876 = new \metastore\FieldSchema();
 -              $xfer += $elem876->read($input);
 -              $this->success []= $elem876;
++              $elem883 = null;
++              $elem883 = new \metastore\FieldSchema();
++              $xfer += $elem883->read($input);
++              $this->success []= $elem883;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -17278,9 -17336,9 +17334,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter863)
 -          foreach ($this->success as $iter877)
++          foreach ($this->success as $iter884)
            {
-             $xfer += $iter863->write($output);
 -            $xfer += $iter877->write($output);
++            $xfer += $iter884->write($output);
            }
          }
          $output->writeListEnd();
@@@ -17494,15 -17552,15 +17550,15 @@@ class ThriftHiveMetastore_get_schema_re
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size864 = 0;
-             $_etype867 = 0;
-             $xfer += $input->readListBegin($_etype867, $_size864);
-             for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
 -            $_size878 = 0;
 -            $_etype881 = 0;
 -            $xfer += $input->readListBegin($_etype881, $_size878);
 -            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
++            $_size885 = 0;
++            $_etype888 = 0;
++            $xfer += $input->readListBegin($_etype888, $_size885);
++            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
              {
-               $elem869 = null;
-               $elem869 = new \metastore\FieldSchema();
-               $xfer += $elem869->read($input);
-               $this->success []= $elem869;
 -              $elem883 = null;
 -              $elem883 = new \metastore\FieldSchema();
 -              $xfer += $elem883->read($input);
 -              $this->success []= $elem883;
++              $elem890 = null;
++              $elem890 = new \metastore\FieldSchema();
++              $xfer += $elem890->read($input);
++              $this->success []= $elem890;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -17554,9 -17612,9 +17610,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter870)
 -          foreach ($this->success as $iter884)
++          foreach ($this->success as $iter891)
            {
-             $xfer += $iter870->write($output);
 -            $xfer += $iter884->write($output);
++            $xfer += $iter891->write($output);
            }
          }
          $output->writeListEnd();
@@@ -17798,15 -17856,15 +17854,15 @@@ class ThriftHiveMetastore_get_schema_wi
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size871 = 0;
-             $_etype874 = 0;
-             $xfer += $input->readListBegin($_etype874, $_size871);
-             for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
 -            $_size885 = 0;
 -            $_etype888 = 0;
 -            $xfer += $input->readListBegin($_etype888, $_size885);
 -            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
++            $_size892 = 0;
++            $_etype895 = 0;
++            $xfer += $input->readListBegin($_etype895, $_size892);
++            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
              {
-               $elem876 = null;
-               $elem876 = new \metastore\FieldSchema();
-               $xfer += $elem876->read($input);
-               $this->success []= $elem876;
 -              $elem890 = null;
 -              $elem890 = new \metastore\FieldSchema();
 -              $xfer += $elem890->read($input);
 -              $this->success []= $elem890;
++              $elem897 = null;
++              $elem897 = new \metastore\FieldSchema();
++              $xfer += $elem897->read($input);
++              $this->success []= $elem897;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -17858,9 -17916,9 +17914,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter877)
 -          foreach ($this->success as $iter891)
++          foreach ($this->success as $iter898)
            {
-             $xfer += $iter877->write($output);
 -            $xfer += $iter891->write($output);
++            $xfer += $iter898->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18532,15 -18590,15 +18588,15 @@@ class ThriftHiveMetastore_create_table_
          case 2:
            if ($ftype == TType::LST) {
              $this->primaryKeys = array();
-             $_size878 = 0;
-             $_etype881 = 0;
-             $xfer += $input->readListBegin($_etype881, $_size878);
-             for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
 -            $_size892 = 0;
 -            $_etype895 = 0;
 -            $xfer += $input->readListBegin($_etype895, $_size892);
 -            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
++            $_size899 = 0;
++            $_etype902 = 0;
++            $xfer += $input->readListBegin($_etype902, $_size899);
++            for ($_i903 = 0; $_i903 < $_size899; ++$_i903)
              {
-               $elem883 = null;
-               $elem883 = new \metastore\SQLPrimaryKey();
-               $xfer += $elem883->read($input);
-               $this->primaryKeys []= $elem883;
 -              $elem897 = null;
 -              $elem897 = new \metastore\SQLPrimaryKey();
 -              $xfer += $elem897->read($input);
 -              $this->primaryKeys []= $elem897;
++              $elem904 = null;
++              $elem904 = new \metastore\SQLPrimaryKey();
++              $xfer += $elem904->read($input);
++              $this->primaryKeys []= $elem904;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18550,15 -18608,15 +18606,15 @@@
          case 3:
            if ($ftype == TType::LST) {
              $this->foreignKeys = array();
-             $_size884 = 0;
-             $_etype887 = 0;
-             $xfer += $input->readListBegin($_etype887, $_size884);
-             for ($_i888 = 0; $_i888 < $_size884; ++$_i888)
 -            $_size898 = 0;
 -            $_etype901 = 0;
 -            $xfer += $input->readListBegin($_etype901, $_size898);
 -            for ($_i902 = 0; $_i902 < $_size898; ++$_i902)
++            $_size905 = 0;
++            $_etype908 = 0;
++            $xfer += $input->readListBegin($_etype908, $_size905);
++            for ($_i909 = 0; $_i909 < $_size905; ++$_i909)
              {
-               $elem889 = null;
-               $elem889 = new \metastore\SQLForeignKey();
-               $xfer += $elem889->read($input);
-               $this->foreignKeys []= $elem889;
 -              $elem903 = null;
 -              $elem903 = new \metastore\SQLForeignKey();
 -              $xfer += $elem903->read($input);
 -              $this->foreignKeys []= $elem903;
++              $elem910 = null;
++              $elem910 = new \metastore\SQLForeignKey();
++              $xfer += $elem910->read($input);
++              $this->foreignKeys []= $elem910;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18568,15 -18626,15 +18624,15 @@@
          case 4:
            if ($ftype == TType::LST) {
              $this->uniqueConstraints = array();
-             $_size890 = 0;
-             $_etype893 = 0;
-             $xfer += $input->readListBegin($_etype893, $_size890);
-             for ($_i894 = 0; $_i894 < $_size890; ++$_i894)
 -            $_size904 = 0;
 -            $_etype907 = 0;
 -            $xfer += $input->readListBegin($_etype907, $_size904);
 -            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
++            $_size911 = 0;
++            $_etype914 = 0;
++            $xfer += $input->readListBegin($_etype914, $_size911);
++            for ($_i915 = 0; $_i915 < $_size911; ++$_i915)
              {
-               $elem895 = null;
-               $elem895 = new \metastore\SQLUniqueConstraint();
-               $xfer += $elem895->read($input);
-               $this->uniqueConstraints []= $elem895;
 -              $elem909 = null;
 -              $elem909 = new \metastore\SQLUniqueConstraint();
 -              $xfer += $elem909->read($input);
 -              $this->uniqueConstraints []= $elem909;
++              $elem916 = null;
++              $elem916 = new \metastore\SQLUniqueConstraint();
++              $xfer += $elem916->read($input);
++              $this->uniqueConstraints []= $elem916;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18586,15 -18644,15 +18642,15 @@@
          case 5:
            if ($ftype == TType::LST) {
              $this->notNullConstraints = array();
-             $_size896 = 0;
-             $_etype899 = 0;
-             $xfer += $input->readListBegin($_etype899, $_size896);
-             for ($_i900 = 0; $_i900 < $_size896; ++$_i900)
 -            $_size910 = 0;
 -            $_etype913 = 0;
 -            $xfer += $input->readListBegin($_etype913, $_size910);
 -            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
++            $_size917 = 0;
++            $_etype920 = 0;
++            $xfer += $input->readListBegin($_etype920, $_size917);
++            for ($_i921 = 0; $_i921 < $_size917; ++$_i921)
              {
-               $elem901 = null;
-               $elem901 = new \metastore\SQLNotNullConstraint();
-               $xfer += $elem901->read($input);
-               $this->notNullConstraints []= $elem901;
 -              $elem915 = null;
 -              $elem915 = new \metastore\SQLNotNullConstraint();
 -              $xfer += $elem915->read($input);
 -              $this->notNullConstraints []= $elem915;
++              $elem922 = null;
++              $elem922 = new \metastore\SQLNotNullConstraint();
++              $xfer += $elem922->read($input);
++              $this->notNullConstraints []= $elem922;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18604,15 -18662,15 +18660,15 @@@
          case 6:
            if ($ftype == TType::LST) {
              $this->defaultConstraints = array();
-             $_size902 = 0;
-             $_etype905 = 0;
-             $xfer += $input->readListBegin($_etype905, $_size902);
-             for ($_i906 = 0; $_i906 < $_size902; ++$_i906)
 -            $_size916 = 0;
 -            $_etype919 = 0;
 -            $xfer += $input->readListBegin($_etype919, $_size916);
 -            for ($_i920 = 0; $_i920 < $_size916; ++$_i920)
++            $_size923 = 0;
++            $_etype926 = 0;
++            $xfer += $input->readListBegin($_etype926, $_size923);
++            for ($_i927 = 0; $_i927 < $_size923; ++$_i927)
              {
-               $elem907 = null;
-               $elem907 = new \metastore\SQLDefaultConstraint();
-               $xfer += $elem907->read($input);
-               $this->defaultConstraints []= $elem907;
 -              $elem921 = null;
 -              $elem921 = new \metastore\SQLDefaultConstraint();
 -              $xfer += $elem921->read($input);
 -              $this->defaultConstraints []= $elem921;
++              $elem928 = null;
++              $elem928 = new \metastore\SQLDefaultConstraint();
++              $xfer += $elem928->read($input);
++              $this->defaultConstraints []= $elem928;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18622,15 -18680,15 +18678,15 @@@
          case 7:
            if ($ftype == TType::LST) {
              $this->checkConstraints = array();
-             $_size908 = 0;
-             $_etype911 = 0;
-             $xfer += $input->readListBegin($_etype911, $_size908);
-             for ($_i912 = 0; $_i912 < $_size908; ++$_i912)
 -            $_size922 = 0;
 -            $_etype925 = 0;
 -            $xfer += $input->readListBegin($_etype925, $_size922);
 -            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
++            $_size929 = 0;
++            $_etype932 = 0;
++            $xfer += $input->readListBegin($_etype932, $_size929);
++            for ($_i933 = 0; $_i933 < $_size929; ++$_i933)
              {
-               $elem913 = null;
-               $elem913 = new \metastore\SQLCheckConstraint();
-               $xfer += $elem913->read($input);
-               $this->checkConstraints []= $elem913;
 -              $elem927 = null;
 -              $elem927 = new \metastore\SQLCheckConstraint();
 -              $xfer += $elem927->read($input);
 -              $this->checkConstraints []= $elem927;
++              $elem934 = null;
++              $elem934 = new \metastore\SQLCheckConstraint();
++              $xfer += $elem934->read($input);
++              $this->checkConstraints []= $elem934;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -18666,9 -18724,9 +18722,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
          {
-           foreach ($this->primaryKeys as $iter914)
 -          foreach ($this->primaryKeys as $iter928)
++          foreach ($this->primaryKeys as $iter935)
            {
-             $xfer += $iter914->write($output);
 -            $xfer += $iter928->write($output);
++            $xfer += $iter935->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18683,9 -18741,9 +18739,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
          {
-           foreach ($this->foreignKeys as $iter915)
 -          foreach ($this->foreignKeys as $iter929)
++          foreach ($this->foreignKeys as $iter936)
            {
-             $xfer += $iter915->write($output);
 -            $xfer += $iter929->write($output);
++            $xfer += $iter936->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18700,9 -18758,9 +18756,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
          {
-           foreach ($this->uniqueConstraints as $iter916)
 -          foreach ($this->uniqueConstraints as $iter930)
++          foreach ($this->uniqueConstraints as $iter937)
            {
-             $xfer += $iter916->write($output);
 -            $xfer += $iter930->write($output);
++            $xfer += $iter937->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18717,9 -18775,9 +18773,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
          {
-           foreach ($this->notNullConstraints as $iter917)
 -          foreach ($this->notNullConstraints as $iter931)
++          foreach ($this->notNullConstraints as $iter938)
            {
-             $xfer += $iter917->write($output);
 -            $xfer += $iter931->write($output);
++            $xfer += $iter938->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18734,9 -18792,9 +18790,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
          {
-           foreach ($this->defaultConstraints as $iter918)
 -          foreach ($this->defaultConstraints as $iter932)
++          foreach ($this->defaultConstraints as $iter939)
            {
-             $xfer += $iter918->write($output);
 -            $xfer += $iter932->write($output);
++            $xfer += $iter939->write($output);
            }
          }
          $output->writeListEnd();
@@@ -18751,9 -18809,9 +18807,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
          {
-           foreach ($this->checkConstraints as $iter919)
 -          foreach ($this->checkConstraints as $iter933)
++          foreach ($this->checkConstraints as $iter940)
            {
-             $xfer += $iter919->write($output);
 -            $xfer += $iter933->write($output);
++            $xfer += $iter940->write($output);
            }
          }
          $output->writeListEnd();
@@@ -20753,14 -20811,14 +20809,14 @@@ class ThriftHiveMetastore_truncate_tabl
          case 3:
            if ($ftype == TType::LST) {
              $this->partNames = array();
-             $_size920 = 0;
-             $_etype923 = 0;
-             $xfer += $input->readListBegin($_etype923, $_size920);
-             for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
 -            $_size934 = 0;
 -            $_etype937 = 0;
 -            $xfer += $input->readListBegin($_etype937, $_size934);
 -            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
++            $_size941 = 0;
++            $_etype944 = 0;
++            $xfer += $input->readListBegin($_etype944, $_size941);
++            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
              {
-               $elem925 = null;
-               $xfer += $input->readString($elem925);
-               $this->partNames []= $elem925;
 -              $elem939 = null;
 -              $xfer += $input->readString($elem939);
 -              $this->partNames []= $elem939;
++              $elem946 = null;
++              $xfer += $input->readString($elem946);
++              $this->partNames []= $elem946;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -20798,9 -20856,9 +20854,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->partNames));
          {
-           foreach ($this->partNames as $iter926)
 -          foreach ($this->partNames as $iter940)
++          foreach ($this->partNames as $iter947)
            {
-             $xfer += $output->writeString($iter926);
 -            $xfer += $output->writeString($iter940);
++            $xfer += $output->writeString($iter947);
            }
          }
          $output->writeListEnd();
@@@ -21051,14 -21109,14 +21107,14 @@@ class ThriftHiveMetastore_get_tables_re
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size927 = 0;
-             $_etype930 = 0;
-             $xfer += $input->readListBegin($_etype930, $_size927);
-             for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
 -            $_size941 = 0;
 -            $_etype944 = 0;
 -            $xfer += $input->readListBegin($_etype944, $_size941);
 -            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
++            $_size948 = 0;
++            $_etype951 = 0;
++            $xfer += $input->readListBegin($_etype951, $_size948);
++            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
              {
-               $elem932 = null;
-               $xfer += $input->readString($elem932);
-               $this->success []= $elem932;
 -              $elem946 = null;
 -              $xfer += $input->readString($elem946);
 -              $this->success []= $elem946;
++              $elem953 = null;
++              $xfer += $input->readString($elem953);
++              $this->success []= $elem953;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -21094,9 -21152,9 +21150,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter933)
 -          foreach ($this->success as $iter947)
++          foreach ($this->success as $iter954)
            {
-             $xfer += $output->writeString($iter933);
 -            $xfer += $output->writeString($iter947);
++            $xfer += $output->writeString($iter954);
            }
          }
          $output->writeListEnd();
@@@ -21298,14 -21356,14 +21354,14 @@@ class ThriftHiveMetastore_get_tables_by
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size934 = 0;
-             $_etype937 = 0;
-             $xfer += $input->readListBegin($_etype937, $_size934);
-             for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
 -            $_size948 = 0;
 -            $_etype951 = 0;
 -            $xfer += $input->readListBegin($_etype951, $_size948);
 -            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
++            $_size955 = 0;
++            $_etype958 = 0;
++            $xfer += $input->readListBegin($_etype958, $_size955);
++            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
              {
-               $elem939 = null;
-               $xfer += $input->readString($elem939);
-               $this->success []= $elem939;
 -              $elem953 = null;
 -              $xfer += $input->readString($elem953);
 -              $this->success []= $elem953;
++              $elem960 = null;
++              $xfer += $input->readString($elem960);
++              $this->success []= $elem960;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -21341,9 -21399,9 +21397,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter940)
 -          foreach ($this->success as $iter954)
++          foreach ($this->success as $iter961)
            {
-             $xfer += $output->writeString($iter940);
 -            $xfer += $output->writeString($iter954);
++            $xfer += $output->writeString($iter961);
            }
          }
          $output->writeListEnd();
@@@ -21499,14 -21557,14 +21555,14 @@@ class ThriftHiveMetastore_get_materiali
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size941 = 0;
-             $_etype944 = 0;
-             $xfer += $input->readListBegin($_etype944, $_size941);
-             for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
 -            $_size955 = 0;
 -            $_etype958 = 0;
 -            $xfer += $input->readListBegin($_etype958, $_size955);
 -            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
++            $_size962 = 0;
++            $_etype965 = 0;
++            $xfer += $input->readListBegin($_etype965, $_size962);
++            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
              {
-               $elem946 = null;
-               $xfer += $input->readString($elem946);
-               $this->success []= $elem946;
 -              $elem960 = null;
 -              $xfer += $input->readString($elem960);
 -              $this->success []= $elem960;
++              $elem967 = null;
++              $xfer += $input->readString($elem967);
++              $this->success []= $elem967;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -21542,9 -21600,9 +21598,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter947)
 -          foreach ($this->success as $iter961)
++          foreach ($this->success as $iter968)
            {
-             $xfer += $output->writeString($iter947);
 -            $xfer += $output->writeString($iter961);
++            $xfer += $output->writeString($iter968);
            }
          }
          $output->writeListEnd();
@@@ -21649,14 -21707,14 +21705,14 @@@ class ThriftHiveMetastore_get_table_met
          case 3:
            if ($ftype == TType::LST) {
              $this->tbl_types = array();
-             $_size948 = 0;
-             $_etype951 = 0;
-             $xfer += $input->readListBegin($_etype951, $_size948);
-             for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
 -            $_size962 = 0;
 -            $_etype965 = 0;
 -            $xfer += $input->readListBegin($_etype965, $_size962);
 -            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
++            $_size969 = 0;
++            $_etype972 = 0;
++            $xfer += $input->readListBegin($_etype972, $_size969);
++            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
              {
-               $elem953 = null;
-               $xfer += $input->readString($elem953);
-               $this->tbl_types []= $elem953;
 -              $elem967 = null;
 -              $xfer += $input->readString($elem967);
 -              $this->tbl_types []= $elem967;
++              $elem974 = null;
++              $xfer += $input->readString($elem974);
++              $this->tbl_types []= $elem974;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -21694,9 -21752,9 +21750,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->tbl_types));
          {
-           foreach ($this->tbl_types as $iter954)
 -          foreach ($this->tbl_types as $iter968)
++          foreach ($this->tbl_types as $iter975)
            {
-             $xfer += $output->writeString($iter954);
 -            $xfer += $output->writeString($iter968);
++            $xfer += $output->writeString($iter975);
            }
          }
          $output->writeListEnd();
@@@ -21773,15 -21831,15 +21829,15 @@@ class ThriftHiveMetastore_get_table_met
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size955 = 0;
-             $_etype958 = 0;
-             $xfer += $input->readListBegin($_etype958, $_size955);
-             for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
 -            $_size969 = 0;
 -            $_etype972 = 0;
 -            $xfer += $input->readListBegin($_etype972, $_size969);
 -            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
++            $_size976 = 0;
++            $_etype979 = 0;
++            $xfer += $input->readListBegin($_etype979, $_size976);
++            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
              {
-               $elem960 = null;
-               $elem960 = new \metastore\TableMeta();
-               $xfer += $elem960->read($input);
-               $this->success []= $elem960;
 -              $elem974 = null;
 -              $elem974 = new \metastore\TableMeta();
 -              $xfer += $elem974->read($input);
 -              $this->success []= $elem974;
++              $elem981 = null;
++              $elem981 = new \metastore\TableMeta();
++              $xfer += $elem981->read($input);
++              $this->success []= $elem981;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -21817,9 -21875,9 +21873,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter961)
 -          foreach ($this->success as $iter975)
++          foreach ($this->success as $iter982)
            {
-             $xfer += $iter961->write($output);
 -            $xfer += $iter975->write($output);
++            $xfer += $iter982->write($output);
            }
          }
          $output->writeListEnd();
@@@ -21975,14 -22033,14 +22031,14 @@@ class ThriftHiveMetastore_get_all_table
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size962 = 0;
-             $_etype965 = 0;
-             $xfer += $input->readListBegin($_etype965, $_size962);
-             for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
 -            $_size976 = 0;
 -            $_etype979 = 0;
 -            $xfer += $input->readListBegin($_etype979, $_size976);
 -            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
++            $_size983 = 0;
++            $_etype986 = 0;
++            $xfer += $input->readListBegin($_etype986, $_size983);
++            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
              {
-               $elem967 = null;
-               $xfer += $input->readString($elem967);
-               $this->success []= $elem967;
 -              $elem981 = null;
 -              $xfer += $input->readString($elem981);
 -              $this->success []= $elem981;
++              $elem988 = null;
++              $xfer += $input->readString($elem988);
++              $this->success []= $elem988;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -22018,9 -22076,9 +22074,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter968)
 -          foreach ($this->success as $iter982)
++          foreach ($this->success as $iter989)
            {
-             $xfer += $output->writeString($iter968);
 -            $xfer += $output->writeString($iter982);
++            $xfer += $output->writeString($iter989);
            }
          }
          $output->writeListEnd();
@@@ -22335,14 -22393,14 +22391,14 @@@ class ThriftHiveMetastore_get_table_obj
          case 2:
            if ($ftype == TType::LST) {
              $this->tbl_names = array();
-             $_size969 = 0;
-             $_etype972 = 0;
-             $xfer += $input->readListBegin($_etype972, $_size969);
-             for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
 -            $_size983 = 0;
 -            $_etype986 = 0;
 -            $xfer += $input->readListBegin($_etype986, $_size983);
 -            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
++            $_size990 = 0;
++            $_etype993 = 0;
++            $xfer += $input->readListBegin($_etype993, $_size990);
++            for ($_i994 = 0; $_i994 < $_size990; ++$_i994)
              {
-               $elem974 = null;
-               $xfer += $input->readString($elem974);
-               $this->tbl_names []= $elem974;
 -              $elem988 = null;
 -              $xfer += $input->readString($elem988);
 -              $this->tbl_names []= $elem988;
++              $elem995 = null;
++              $xfer += $input->readString($elem995);
++              $this->tbl_names []= $elem995;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -22375,9 -22433,9 +22431,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->tbl_names));
          {
-           foreach ($this->tbl_names as $iter975)
 -          foreach ($this->tbl_names as $iter989)
++          foreach ($this->tbl_names as $iter996)
            {
-             $xfer += $output->writeString($iter975);
 -            $xfer += $output->writeString($iter989);
++            $xfer += $output->writeString($iter996);
            }
          }
          $output->writeListEnd();
@@@ -22442,15 -22500,15 +22498,15 @@@ class ThriftHiveMetastore_get_table_obj
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size976 = 0;
-             $_etype979 = 0;
-             $xfer += $input->readListBegin($_etype979, $_size976);
-             for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
 -            $_size990 = 0;
 -            $_etype993 = 0;
 -            $xfer += $input->readListBegin($_etype993, $_size990);
 -            for ($_i994 = 0; $_i994 < $_size990; ++$_i994)
++            $_size997 = 0;
++            $_etype1000 = 0;
++            $xfer += $input->readListBegin($_etype1000, $_size997);
++            for ($_i1001 = 0; $_i1001 < $_size997; ++$_i1001)
              {
-               $elem981 = null;
-               $elem981 = new \metastore\Table();
-               $xfer += $elem981->read($input);
-               $this->success []= $elem981;
 -              $elem995 = null;
 -              $elem995 = new \metastore\Table();
 -              $xfer += $elem995->read($input);
 -              $this->success []= $elem995;
++              $elem1002 = null;
++              $elem1002 = new \metastore\Table();
++              $xfer += $elem1002->read($input);
++              $this->success []= $elem1002;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -22478,9 -22536,9 +22534,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter982)
 -          foreach ($this->success as $iter996)
++          foreach ($this->success as $iter1003)
            {
-             $xfer += $iter982->write($output);
 -            $xfer += $iter996->write($output);
++            $xfer += $iter1003->write($output);
            }
          }
          $output->writeListEnd();
@@@ -23007,14 -23065,14 +23063,14 @@@ class ThriftHiveMetastore_get_materiali
          case 2:
            if ($ftype == TType::LST) {
              $this->tbl_names = array();
-             $_size983 = 0;
-             $_etype986 = 0;
-             $xfer += $input->readListBegin($_etype986, $_size983);
-             for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
 -            $_size997 = 0;
 -            $_etype1000 = 0;
 -            $xfer += $input->readListBegin($_etype1000, $_size997);
 -            for ($_i1001 = 0; $_i1001 < $_size997; ++$_i1001)
++            $_size1004 = 0;
++            $_etype1007 = 0;
++            $xfer += $input->readListBegin($_etype1007, $_size1004);
++            for ($_i1008 = 0; $_i1008 < $_size1004; ++$_i1008)
              {
-               $elem988 = null;
-               $xfer += $input->readString($elem988);
-               $this->tbl_names []= $elem988;
 -              $elem1002 = null;
 -              $xfer += $input->readString($elem1002);
 -              $this->tbl_names []= $elem1002;
++              $elem1009 = null;
++              $xfer += $input->readString($elem1009);
++              $this->tbl_names []= $elem1009;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -23047,9 -23105,9 +23103,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->tbl_names));
          {
-           foreach ($this->tbl_names as $iter989)
 -          foreach ($this->tbl_names as $iter1003)
++          foreach ($this->tbl_names as $iter1010)
            {
-             $xfer += $output->writeString($iter989);
 -            $xfer += $output->writeString($iter1003);
++            $xfer += $output->writeString($iter1010);
            }
          }
          $output->writeListEnd();
@@@ -23154,18 -23212,18 +23210,18 @@@ class ThriftHiveMetastore_get_materiali
          case 0:
            if ($ftype == TType::MAP) {
              $this->success = array();
-             $_size990 = 0;
-             $_ktype991 = 0;
-             $_vtype992 = 0;
-             $xfer += $input->readMapBegin($_ktype991, $_vtype992, $_size990);
-             for ($_i994 = 0; $_i994 < $_size990; ++$_i994)
 -            $_size1004 = 0;
 -            $_ktype1005 = 0;
 -            $_vtype1006 = 0;
 -            $xfer += $input->readMapBegin($_ktype1005, $_vtype1006, $_size1004);
 -            for ($_i1008 = 0; $_i1008 < $_size1004; ++$_i1008)
++            $_size1011 = 0;
++            $_ktype1012 = 0;
++            $_vtype1013 = 0;
++            $xfer += $input->readMapBegin($_ktype1012, $_vtype1013, $_size1011);
++            for ($_i1015 = 0; $_i1015 < $_size1011; ++$_i1015)
              {
-               $key995 = '';
-               $val996 = new \metastore\Materialization();
-               $xfer += $input->readString($key995);
-               $val996 = new \metastore\Materialization();
-               $xfer += $val996->read($input);
-               $this->success[$key995] = $val996;
 -              $key1009 = '';
 -              $val1010 = new \metastore\Materialization();
 -              $xfer += $input->readString($key1009);
 -              $val1010 = new \metastore\Materialization();
 -              $xfer += $val1010->read($input);
 -              $this->success[$key1009] = $val1010;
++              $key1016 = '';
++              $val1017 = new \metastore\Materialization();
++              $xfer += $input->readString($key1016);
++              $val1017 = new \metastore\Materialization();
++              $xfer += $val1017->read($input);
++              $this->success[$key1016] = $val1017;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -23217,10 -23275,10 +23273,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $kiter997 => $viter998)
 -          foreach ($this->success as $kiter1011 => $viter1012)
++          foreach ($this->success as $kiter1018 => $viter1019)
            {
-             $xfer += $output->writeString($kiter997);
-             $xfer += $viter998->write($output);
 -            $xfer += $output->writeString($kiter1011);
 -            $xfer += $viter1012->write($output);
++            $xfer += $output->writeString($kiter1018);
++            $xfer += $viter1019->write($output);
            }
          }
          $output->writeMapEnd();
@@@ -23732,14 -23790,14 +23788,14 @@@ class ThriftHiveMetastore_get_table_nam
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size999 = 0;
-             $_etype1002 = 0;
-             $xfer += $input->readListBegin($_etype1002, $_size999);
-             for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
 -            $_size1013 = 0;
 -            $_etype1016 = 0;
 -            $xfer += $input->readListBegin($_etype1016, $_size1013);
 -            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
++            $_size1020 = 0;
++            $_etype1023 = 0;
++            $xfer += $input->readListBegin($_etype1023, $_size1020);
++            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
              {
-               $elem1004 = null;
-               $xfer += $input->readString($elem1004);
-               $this->success []= $elem1004;
 -              $elem1018 = null;
 -              $xfer += $input->readString($elem1018);
 -              $this->success []= $elem1018;
++              $elem1025 = null;
++              $xfer += $input->readString($elem1025);
++              $this->success []= $elem1025;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -23791,9 -23849,9 +23847,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter1005)
 -          foreach ($this->success as $iter1019)
++          foreach ($this->success as $iter1026)
            {
-             $xfer += $output->writeString($iter1005);
 -            $xfer += $output->writeString($iter1019);
++            $xfer += $output->writeString($iter1026);
            }
          }
          $output->writeListEnd();
@@@ -25106,15 -25164,15 +25162,15 @@@ class ThriftHiveMetastore_add_partition
          case 1:
            if ($ftype == TType::LST) {
              $this->new_parts = array();
-             $_size1006 = 0;
-             $_etype1009 = 0;
-             $xfer += $input->readListBegin($_etype1009, $_size1006);
-             for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
 -            $_size1020 = 0;
 -            $_etype1023 = 0;
 -            $xfer += $input->readListBegin($_etype1023, $_size1020);
 -            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
++            $_size1027 = 0;
++            $_etype1030 = 0;
++            $xfer += $input->readListBegin($_etype1030, $_size1027);
++            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
              {
-               $elem1011 = null;
-               $elem1011 = new \metastore\Partition();
-               $xfer += $elem1011->read($input);
-               $this->new_parts []= $elem1011;
 -              $elem1025 = null;
 -              $elem1025 = new \metastore\Partition();
 -              $xfer += $elem1025->read($input);
 -              $this->new_parts []= $elem1025;
++              $elem1032 = null;
++              $elem1032 = new \metastore\Partition();
++              $xfer += $elem1032->read($input);
++              $this->new_parts []= $elem1032;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25142,9 -25200,9 +25198,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->new_parts));
          {
-           foreach ($this->new_parts as $iter1012)
 -          foreach ($this->new_parts as $iter1026)
++          foreach ($this->new_parts as $iter1033)
            {
-             $xfer += $iter1012->write($output);
 -            $xfer += $iter1026->write($output);
++            $xfer += $iter1033->write($output);
            }
          }
          $output->writeListEnd();
@@@ -25359,15 -25417,15 +25415,15 @@@ class ThriftHiveMetastore_add_partition
          case 1:
            if ($ftype == TType::LST) {
              $this->new_parts = array();
-             $_size1013 = 0;
-             $_etype1016 = 0;
-             $xfer += $input->readListBegin($_etype1016, $_size1013);
-             for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
 -            $_size1027 = 0;
 -            $_etype1030 = 0;
 -            $xfer += $input->readListBegin($_etype1030, $_size1027);
 -            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
++            $_size1034 = 0;
++            $_etype1037 = 0;
++            $xfer += $input->readListBegin($_etype1037, $_size1034);
++            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
              {
-               $elem1018 = null;
-               $elem1018 = new \metastore\PartitionSpec();
-               $xfer += $elem1018->read($input);
-               $this->new_parts []= $elem1018;
 -              $elem1032 = null;
 -              $elem1032 = new \metastore\PartitionSpec();
 -              $xfer += $elem1032->read($input);
 -              $this->new_parts []= $elem1032;
++              $elem1039 = null;
++              $elem1039 = new \metastore\PartitionSpec();
++              $xfer += $elem1039->read($input);
++              $this->new_parts []= $elem1039;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25395,9 -25453,9 +25451,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->new_parts));
          {
-           foreach ($this->new_parts as $iter1019)
 -          foreach ($this->new_parts as $iter1033)
++          foreach ($this->new_parts as $iter1040)
            {
-             $xfer += $iter1019->write($output);
 -            $xfer += $iter1033->write($output);
++            $xfer += $iter1040->write($output);
            }
          }
          $output->writeListEnd();
@@@ -25647,14 -25705,14 +25703,14 @@@ class ThriftHiveMetastore_append_partit
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1020 = 0;
-             $_etype1023 = 0;
-             $xfer += $input->readListBegin($_etype1023, $_size1020);
-             for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
 -            $_size1034 = 0;
 -            $_etype1037 = 0;
 -            $xfer += $input->readListBegin($_etype1037, $_size1034);
 -            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
++            $_size1041 = 0;
++            $_etype1044 = 0;
++            $xfer += $input->readListBegin($_etype1044, $_size1041);
++            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
              {
-               $elem1025 = null;
-               $xfer += $input->readString($elem1025);
-               $this->part_vals []= $elem1025;
 -              $elem1039 = null;
 -              $xfer += $input->readString($elem1039);
 -              $this->part_vals []= $elem1039;
++              $elem1046 = null;
++              $xfer += $input->readString($elem1046);
++              $this->part_vals []= $elem1046;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -25692,9 -25750,9 +25748,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1026)
 -          foreach ($this->part_vals as $iter1040)
++          foreach ($this->part_vals as $iter1047)
            {
-             $xfer += $output->writeString($iter1026);
 -            $xfer += $output->writeString($iter1040);
++            $xfer += $output->writeString($iter1047);
            }
          }
          $output->writeListEnd();
@@@ -26196,14 -26254,14 +26252,14 @@@ class ThriftHiveMetastore_append_partit
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1027 = 0;
-             $_etype1030 = 0;
-             $xfer += $input->readListBegin($_etype1030, $_size1027);
-             for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
 -            $_size1041 = 0;
 -            $_etype1044 = 0;
 -            $xfer += $input->readListBegin($_etype1044, $_size1041);
 -            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
++            $_size1048 = 0;
++            $_etype1051 = 0;
++            $xfer += $input->readListBegin($_etype1051, $_size1048);
++            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
              {
-               $elem1032 = null;
-               $xfer += $input->readString($elem1032);
-               $this->part_vals []= $elem1032;
 -              $elem1046 = null;
 -              $xfer += $input->readString($elem1046);
 -              $this->part_vals []= $elem1046;
++              $elem1053 = null;
++              $xfer += $input->readString($elem1053);
++              $this->part_vals []= $elem1053;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -26249,9 -26307,9 +26305,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1033)
 -          foreach ($this->part_vals as $iter1047)
++          foreach ($this->part_vals as $iter1054)
            {
-             $xfer += $output->writeString($iter1033);
 -            $xfer += $output->writeString($iter1047);
++            $xfer += $output->writeString($iter1054);
            }
          }
          $output->writeListEnd();
@@@ -27105,14 -27163,14 +27161,14 @@@ class ThriftHiveMetastore_drop_partitio
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1034 = 0;
-             $_etype1037 = 0;
-             $xfer += $input->readListBegin($_etype1037, $_size1034);
-             for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
 -            $_size1048 = 0;
 -            $_etype1051 = 0;
 -            $xfer += $input->readListBegin($_etype1051, $_size1048);
 -            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
++            $_size1055 = 0;
++            $_etype1058 = 0;
++            $xfer += $input->readListBegin($_etype1058, $_size1055);
++            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
              {
-               $elem1039 = null;
-               $xfer += $input->readString($elem1039);
-               $this->part_vals []= $elem1039;
 -              $elem1053 = null;
 -              $xfer += $input->readString($elem1053);
 -              $this->part_vals []= $elem1053;
++              $elem1060 = null;
++              $xfer += $input->readString($elem1060);
++              $this->part_vals []= $elem1060;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -27157,9 -27215,9 +27213,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1040)
 -          foreach ($this->part_vals as $iter1054)
++          foreach ($this->part_vals as $iter1061)
            {
-             $xfer += $output->writeString($iter1040);
 -            $xfer += $output->writeString($iter1054);
++            $xfer += $output->writeString($iter1061);
            }
          }
          $output->writeListEnd();
@@@ -27412,14 -27470,14 +27468,14 @@@ class ThriftHiveMetastore_drop_partitio
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1041 = 0;
-             $_etype1044 = 0;
-             $xfer += $input->readListBegin($_etype1044, $_size1041);
-             for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
 -            $_size1055 = 0;
 -            $_etype1058 = 0;
 -            $xfer += $input->readListBegin($_etype1058, $_size1055);
 -            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
++            $_size1062 = 0;
++            $_etype1065 = 0;
++            $xfer += $input->readListBegin($_etype1065, $_size1062);
++            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
              {
-               $elem1046 = null;
-               $xfer += $input->readString($elem1046);
-               $this->part_vals []= $elem1046;
 -              $elem1060 = null;
 -              $xfer += $input->readString($elem1060);
 -              $this->part_vals []= $elem1060;
++              $elem1067 = null;
++              $xfer += $input->readString($elem1067);
++              $this->part_vals []= $elem1067;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -27472,9 -27530,9 +27528,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1047)
 -          foreach ($this->part_vals as $iter1061)
++          foreach ($this->part_vals as $iter1068)
            {
-             $xfer += $output->writeString($iter1047);
 -            $xfer += $output->writeString($iter1061);
++            $xfer += $output->writeString($iter1068);
            }
          }
          $output->writeListEnd();
@@@ -28488,14 -28546,14 +28544,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1048 = 0;
-             $_etype1051 = 0;
-             $xfer += $input->readListBegin($_etype1051, $_size1048);
-             for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
 -            $_size1062 = 0;
 -            $_etype1065 = 0;
 -            $xfer += $input->readListBegin($_etype1065, $_size1062);
 -            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
++            $_size1069 = 0;
++            $_etype1072 = 0;
++            $xfer += $input->readListBegin($_etype1072, $_size1069);
++            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
              {
-               $elem1053 = null;
-               $xfer += $input->readString($elem1053);
-               $this->part_vals []= $elem1053;
 -              $elem1067 = null;
 -              $xfer += $input->readString($elem1067);
 -              $this->part_vals []= $elem1067;
++              $elem1074 = null;
++              $xfer += $input->readString($elem1074);
++              $this->part_vals []= $elem1074;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -28533,9 -28591,9 +28589,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1054)
 -          foreach ($this->part_vals as $iter1068)
++          foreach ($this->part_vals as $iter1075)
            {
-             $xfer += $output->writeString($iter1054);
 -            $xfer += $output->writeString($iter1068);
++            $xfer += $output->writeString($iter1075);
            }
          }
          $output->writeListEnd();
@@@ -28777,17 -28835,17 +28833,17 @@@ class ThriftHiveMetastore_exchange_part
          case 1:
            if ($ftype == TType::MAP) {
              $this->partitionSpecs = array();
-             $_size1055 = 0;
-             $_ktype1056 = 0;
-             $_vtype1057 = 0;
-             $xfer += $input->readMapBegin($_ktype1056, $_vtype1057, $_size1055);
-             for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
 -            $_size1069 = 0;
 -            $_ktype1070 = 0;
 -            $_vtype1071 = 0;
 -            $xfer += $input->readMapBegin($_ktype1070, $_vtype1071, $_size1069);
 -            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
++            $_size1076 = 0;
++            $_ktype1077 = 0;
++            $_vtype1078 = 0;
++            $xfer += $input->readMapBegin($_ktype1077, $_vtype1078, $_size1076);
++            for ($_i1080 = 0; $_i1080 < $_size1076; ++$_i1080)
              {
-               $key1060 = '';
-               $val1061 = '';
-               $xfer += $input->readString($key1060);
-               $xfer += $input->readString($val1061);
-               $this->partitionSpecs[$key1060] = $val1061;
 -              $key1074 = '';
 -              $val1075 = '';
 -              $xfer += $input->readString($key1074);
 -              $xfer += $input->readString($val1075);
 -              $this->partitionSpecs[$key1074] = $val1075;
++              $key1081 = '';
++              $val1082 = '';
++              $xfer += $input->readString($key1081);
++              $xfer += $input->readString($val1082);
++              $this->partitionSpecs[$key1081] = $val1082;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -28843,10 -28901,10 +28899,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
          {
-           foreach ($this->partitionSpecs as $kiter1062 => $viter1063)
 -          foreach ($this->partitionSpecs as $kiter1076 => $viter1077)
++          foreach ($this->partitionSpecs as $kiter1083 => $viter1084)
            {
-             $xfer += $output->writeString($kiter1062);
-             $xfer += $output->writeString($viter1063);
 -            $xfer += $output->writeString($kiter1076);
 -            $xfer += $output->writeString($viter1077);
++            $xfer += $output->writeString($kiter1083);
++            $xfer += $output->writeString($viter1084);
            }
          }
          $output->writeMapEnd();
@@@ -29158,17 -29216,17 +29214,17 @@@ class ThriftHiveMetastore_exchange_part
          case 1:
            if ($ftype == TType::MAP) {
              $this->partitionSpecs = array();
-             $_size1064 = 0;
-             $_ktype1065 = 0;
-             $_vtype1066 = 0;
-             $xfer += $input->readMapBegin($_ktype1065, $_vtype1066, $_size1064);
-             for ($_i1068 = 0; $_i1068 < $_size1064; ++$_i1068)
 -            $_size1078 = 0;
 -            $_ktype1079 = 0;
 -            $_vtype1080 = 0;
 -            $xfer += $input->readMapBegin($_ktype1079, $_vtype1080, $_size1078);
 -            for ($_i1082 = 0; $_i1082 < $_size1078; ++$_i1082)
++            $_size1085 = 0;
++            $_ktype1086 = 0;
++            $_vtype1087 = 0;
++            $xfer += $input->readMapBegin($_ktype1086, $_vtype1087, $_size1085);
++            for ($_i1089 = 0; $_i1089 < $_size1085; ++$_i1089)
              {
-               $key1069 = '';
-               $val1070 = '';
-               $xfer += $input->readString($key1069);
-               $xfer += $input->readString($val1070);
-               $this->partitionSpecs[$key1069] = $val1070;
 -              $key1083 = '';
 -              $val1084 = '';
 -              $xfer += $input->readString($key1083);
 -              $xfer += $input->readString($val1084);
 -              $this->partitionSpecs[$key1083] = $val1084;
++              $key1090 = '';
++              $val1091 = '';
++              $xfer += $input->readString($key1090);
++              $xfer += $input->readString($val1091);
++              $this->partitionSpecs[$key1090] = $val1091;
              }
              $xfer += $input->readMapEnd();
            } else {
@@@ -29224,10 -29282,10 +29280,10 @@@
        {
          $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
          {
-           foreach ($this->partitionSpecs as $kiter1071 => $viter1072)
 -          foreach ($this->partitionSpecs as $kiter1085 => $viter1086)
++          foreach ($this->partitionSpecs as $kiter1092 => $viter1093)
            {
-             $xfer += $output->writeString($kiter1071);
-             $xfer += $output->writeString($viter1072);
 -            $xfer += $output->writeString($kiter1085);
 -            $xfer += $output->writeString($viter1086);
++            $xfer += $output->writeString($kiter1092);
++            $xfer += $output->writeString($viter1093);
            }
          }
          $output->writeMapEnd();
@@@ -29360,15 -29418,15 +29416,15 @@@ class ThriftHiveMetastore_exchange_part
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1073 = 0;
-             $_etype1076 = 0;
-             $xfer += $input->readListBegin($_etype1076, $_size1073);
-             for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
 -            $_size1087 = 0;
 -            $_etype1090 = 0;
 -            $xfer += $input->readListBegin($_etype1090, $_size1087);
 -            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
++            $_size1094 = 0;
++            $_etype1097 = 0;
++            $xfer += $input->readListBegin($_etype1097, $_size1094);
++            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
              {
-               $elem1078 = null;
-               $elem1078 = new \metastore\Partition();
-               $xfer += $elem1078->read($input);
-               $this->success []= $elem1078;
 -              $elem1092 = null;
 -              $elem1092 = new \metastore\Partition();
 -              $xfer += $elem1092->read($input);
 -              $this->success []= $elem1092;
++              $elem1099 = null;
++              $elem1099 = new \metastore\Partition();
++              $xfer += $elem1099->read($input);
++              $this->success []= $elem1099;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -29428,9 -29486,9 +29484,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter1079)
 -          foreach ($this->success as $iter1093)
++          foreach ($this->success as $iter1100)
            {
-             $xfer += $iter1079->write($output);
 -            $xfer += $iter1093->write($output);
++            $xfer += $iter1100->write($output);
            }
          }
          $output->writeListEnd();
@@@ -29576,14 -29634,14 +29632,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1080 = 0;
-             $_etype1083 = 0;
-             $xfer += $input->readListBegin($_etype1083, $_size1080);
-             for ($_i1084 = 0; $_i1084 < $_size1080; ++$_i1084)
 -            $_size1094 = 0;
 -            $_etype1097 = 0;
 -            $xfer += $input->readListBegin($_etype1097, $_size1094);
 -            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
++            $_size1101 = 0;
++            $_etype1104 = 0;
++            $xfer += $input->readListBegin($_etype1104, $_size1101);
++            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
              {
-               $elem1085 = null;
-               $xfer += $input->readString($elem1085);
-               $this->part_vals []= $elem1085;
 -              $elem1099 = null;
 -              $xfer += $input->readString($elem1099);
 -              $this->part_vals []= $elem1099;
++              $elem1106 = null;
++              $xfer += $input->readString($elem1106);
++              $this->part_vals []= $elem1106;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -29600,14 -29658,14 +29656,14 @@@
          case 5:
            if ($ftype == TType::LST) {
              $this->group_names = array();
-             $_size1086 = 0;
-             $_etype1089 = 0;
-             $xfer += $input->readListBegin($_etype1089, $_size1086);
-             for ($_i1090 = 0; $_i1090 < $_size1086; ++$_i1090)
 -            $_size1100 = 0;
 -            $_etype1103 = 0;
 -            $xfer += $input->readListBegin($_etype1103, $_size1100);
 -            for ($_i1104 = 0; $_i1104 < $_size1100; ++$_i1104)
++            $_size1107 = 0;
++            $_etype1110 = 0;
++            $xfer += $input->readListBegin($_etype1110, $_size1107);
++            for ($_i1111 = 0; $_i1111 < $_size1107; ++$_i1111)
              {
-               $elem1091 = null;
-               $xfer += $input->readString($elem1091);
-               $this->group_names []= $elem1091;
 -              $elem1105 = null;
 -              $xfer += $input->readString($elem1105);
 -              $this->group_names []= $elem1105;
++              $elem1112 = null;
++              $xfer += $input->readString($elem1112);
++              $this->group_names []= $elem1112;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -29645,9 -29703,9 +29701,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1092)
 -          foreach ($this->part_vals as $iter1106)
++          foreach ($this->part_vals as $iter1113)
            {
-             $xfer += $output->writeString($iter1092);
 -            $xfer += $output->writeString($iter1106);
++            $xfer += $output->writeString($iter1113);
            }
          }
          $output->writeListEnd();
@@@ -29667,9 -29725,9 +29723,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
-           foreach ($this->group_names as $iter1093)
 -          foreach ($this->group_names as $iter1107)
++          foreach ($this->group_names as $iter1114)
            {
-             $xfer += $output->writeString($iter1093);
 -            $xfer += $output->writeString($iter1107);
++            $xfer += $output->writeString($iter1114);
            }
          }
          $output->writeListEnd();
@@@ -30260,15 -30318,15 +30316,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1094 = 0;
-             $_etype1097 = 0;
-             $xfer += $input->readListBegin($_etype1097, $_size1094);
-             for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
 -            $_size1108 = 0;
 -            $_etype1111 = 0;
 -            $xfer += $input->readListBegin($_etype1111, $_size1108);
 -            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
++            $_size1115 = 0;
++            $_etype1118 = 0;
++            $xfer += $input->readListBegin($_etype1118, $_size1115);
++            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
              {
-               $elem1099 = null;
-               $elem1099 = new \metastore\Partition();
-               $xfer += $elem1099->read($input);
-               $this->success []= $elem1099;
 -              $elem1113 = null;
 -              $elem1113 = new \metastore\Partition();
 -              $xfer += $elem1113->read($input);
 -              $this->success []= $elem1113;
++              $elem1120 = null;
++              $elem1120 = new \metastore\Partition();
++              $xfer += $elem1120->read($input);
++              $this->success []= $elem1120;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -30312,9 -30370,9 +30368,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter1100)
 -          foreach ($this->success as $iter1114)
++          foreach ($this->success as $iter1121)
            {
-             $xfer += $iter1100->write($output);
 -            $xfer += $iter1114->write($output);
++            $xfer += $iter1121->write($output);
            }
          }
          $output->writeListEnd();
@@@ -30460,14 -30518,14 +30516,14 @@@ class ThriftHiveMetastore_get_partition
          case 5:
            if ($ftype == TType::LST) {
              $this->group_names = array();
-             $_size1101 = 0;
-             $_etype1104 = 0;
-             $xfer += $input->readListBegin($_etype1104, $_size1101);
-             for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
 -            $_size1115 = 0;
 -            $_etype1118 = 0;
 -            $xfer += $input->readListBegin($_etype1118, $_size1115);
 -            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
++            $_size1122 = 0;
++            $_etype1125 = 0;
++            $xfer += $input->readListBegin($_etype1125, $_size1122);
++            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
              {
-               $elem1106 = null;
-               $xfer += $input->readString($elem1106);
-               $this->group_names []= $elem1106;
 -              $elem1120 = null;
 -              $xfer += $input->readString($elem1120);
 -              $this->group_names []= $elem1120;
++              $elem1127 = null;
++              $xfer += $input->readString($elem1127);
++              $this->group_names []= $elem1127;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -30515,9 -30573,9 +30571,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
-           foreach ($this->group_names as $iter1107)
 -          foreach ($this->group_names as $iter1121)
++          foreach ($this->group_names as $iter1128)
            {
-             $xfer += $output->writeString($iter1107);
 -            $xfer += $output->writeString($iter1121);
++            $xfer += $output->writeString($iter1128);
            }
          }
          $output->writeListEnd();
@@@ -30606,15 -30664,15 +30662,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1108 = 0;
-             $_etype1111 = 0;
-             $xfer += $input->readListBegin($_etype1111, $_size1108);
-             for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
 -            $_size1122 = 0;
 -            $_etype1125 = 0;
 -            $xfer += $input->readListBegin($_etype1125, $_size1122);
 -            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
++            $_size1129 = 0;
++            $_etype1132 = 0;
++            $xfer += $input->readListBegin($_etype1132, $_size1129);
++            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
              {
-               $elem1113 = null;
-               $elem1113 = new \metastore\Partition();
-               $xfer += $elem1113->read($input);
-               $this->success []= $elem1113;
 -              $elem1127 = null;
 -              $elem1127 = new \metastore\Partition();
 -              $xfer += $elem1127->read($input);
 -              $this->success []= $elem1127;
++              $elem1134 = null;
++              $elem1134 = new \metastore\Partition();
++              $xfer += $elem1134->read($input);
++              $this->success []= $elem1134;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -30658,9 -30716,9 +30714,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter1114)
 -          foreach ($this->success as $iter1128)
++          foreach ($this->success as $iter1135)
            {
-             $xfer += $iter1114->write($output);
 -            $xfer += $iter1128->write($output);
++            $xfer += $iter1135->write($output);
            }
          }
          $output->writeListEnd();
@@@ -30880,15 -30938,15 +30936,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1115 = 0;
-             $_etype1118 = 0;
-             $xfer += $input->readListBegin($_etype1118, $_size1115);
-             for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
 -            $_size1129 = 0;
 -            $_etype1132 = 0;
 -            $xfer += $input->readListBegin($_etype1132, $_size1129);
 -            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
++            $_size1136 = 0;
++            $_etype1139 = 0;
++            $xfer += $input->readListBegin($_etype1139, $_size1136);
++            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
              {
-               $elem1120 = null;
-               $elem1120 = new \metastore\PartitionSpec();
-               $xfer += $elem1120->read($input);
-               $this->success []= $elem1120;
 -              $elem1134 = null;
 -              $elem1134 = new \metastore\PartitionSpec();
 -              $xfer += $elem1134->read($input);
 -              $this->success []= $elem1134;
++              $elem1141 = null;
++              $elem1141 = new \metastore\PartitionSpec();
++              $xfer += $elem1141->read($input);
++              $this->success []= $elem1141;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -30932,9 -30990,9 +30988,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter1121)
 -          foreach ($this->success as $iter1135)
++          foreach ($this->success as $iter1142)
            {
-             $xfer += $iter1121->write($output);
 -            $xfer += $iter1135->write($output);
++            $xfer += $iter1142->write($output);
            }
          }
          $output->writeListEnd();
@@@ -31153,14 -31211,14 +31209,14 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1122 = 0;
-             $_etype1125 = 0;
-             $xfer += $input->readListBegin($_etype1125, $_size1122);
-             for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
 -            $_size1136 = 0;
 -            $_etype1139 = 0;
 -            $xfer += $input->readListBegin($_etype1139, $_size1136);
 -            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
++            $_size1143 = 0;
++            $_etype1146 = 0;
++            $xfer += $input->readListBegin($_etype1146, $_size1143);
++            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
              {
-               $elem1127 = null;
-               $xfer += $input->readString($elem1127);
-               $this->success []= $elem1127;
 -              $elem1141 = null;
 -              $xfer += $input->readString($elem1141);
 -              $this->success []= $elem1141;
++              $elem1148 = null;
++              $xfer += $input->readString($elem1148);
++              $this->success []= $elem1148;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -31204,9 -31262,9 +31260,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->success));
          {
-           foreach ($this->success as $iter1128)
 -          foreach ($this->success as $iter1142)
++          foreach ($this->success as $iter1149)
            {
-             $xfer += $output->writeString($iter1128);
 -            $xfer += $output->writeString($iter1142);
++            $xfer += $output->writeString($iter1149);
            }
          }
          $output->writeListEnd();
@@@ -31537,14 -31595,14 +31593,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1129 = 0;
-             $_etype1132 = 0;
-             $xfer += $input->readListBegin($_etype1132, $_size1129);
-             for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
 -            $_size1143 = 0;
 -            $_etype1146 = 0;
 -            $xfer += $input->readListBegin($_etype1146, $_size1143);
 -            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
++            $_size1150 = 0;
++            $_etype1153 = 0;
++            $xfer += $input->readListBegin($_etype1153, $_size1150);
++            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
              {
-               $elem1134 = null;
-               $xfer += $input->readString($elem1134);
-               $this->part_vals []= $elem1134;
 -              $elem1148 = null;
 -              $xfer += $input->readString($elem1148);
 -              $this->part_vals []= $elem1148;
++              $elem1155 = null;
++              $xfer += $input->readString($elem1155);
++              $this->part_vals []= $elem1155;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -31589,9 -31647,9 +31645,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1135)
 -          foreach ($this->part_vals as $iter1149)
++          foreach ($this->part_vals as $iter1156)
            {
-             $xfer += $output->writeString($iter1135);
 -            $xfer += $output->writeString($iter1149);
++            $xfer += $output->writeString($iter1156);
            }
          }
          $output->writeListEnd();
@@@ -31685,15 -31743,15 +31741,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1136 = 0;
-             $_etype1139 = 0;
-             $xfer += $input->readListBegin($_etype1139, $_size1136);
-             for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
 -            $_size1150 = 0;
 -            $_etype1153 = 0;
 -            $xfer += $input->readListBegin($_etype1153, $_size1150);
 -            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
++            $_size1157 = 0;
++            $_etype1160 = 0;
++            $xfer += $input->readListBegin($_etype1160, $_size1157);
++            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
              {
-               $elem1141 = null;
-               $elem1141 = new \metastore\Partition();
-               $xfer += $elem1141->read($input);
-               $this->success []= $elem1141;
 -              $elem1155 = null;
 -              $elem1155 = new \metastore\Partition();
 -              $xfer += $elem1155->read($input);
 -              $this->success []= $elem1155;
++              $elem1162 = null;
++              $elem1162 = new \metastore\Partition();
++              $xfer += $elem1162->read($input);
++              $this->success []= $elem1162;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -31737,9 -31795,9 +31793,9 @@@
        {
          $output->writeListBegin(TType::STRUCT, count($this->success));
          {
-           foreach ($this->success as $iter1142)
 -          foreach ($this->success as $iter1156)
++          foreach ($this->success as $iter1163)
            {
-             $xfer += $iter1142->write($output);
 -            $xfer += $iter1156->write($output);
++            $xfer += $iter1163->write($output);
            }
          }
          $output->writeListEnd();
@@@ -31886,14 -31944,14 +31942,14 @@@ class ThriftHiveMetastore_get_partition
          case 3:
            if ($ftype == TType::LST) {
              $this->part_vals = array();
-             $_size1143 = 0;
-             $_etype1146 = 0;
-             $xfer += $input->readListBegin($_etype1146, $_size1143);
-             for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
 -            $_size1157 = 0;
 -            $_etype1160 = 0;
 -            $xfer += $input->readListBegin($_etype1160, $_size1157);
 -            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
++            $_size1164 = 0;
++            $_etype1167 = 0;
++            $xfer += $input->readListBegin($_etype1167, $_size1164);
++            for ($_i1168 = 0; $_i1168 < $_size1164; ++$_i1168)
              {
-               $elem1148 = null;
-               $xfer += $input->readString($elem1148);
-               $this->part_vals []= $elem1148;
 -              $elem1162 = null;
 -              $xfer += $input->readString($elem1162);
 -              $this->part_vals []= $elem1162;
++              $elem1169 = null;
++              $xfer += $input->readString($elem1169);
++              $this->part_vals []= $elem1169;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -31917,14 -31975,14 +31973,14 @@@
          case 6:
            if ($ftype == TType::LST) {
              $this->group_names = array();
-             $_size1149 = 0;
-             $_etype1152 = 0;
-             $xfer += $input->readListBegin($_etype1152, $_size1149);
-             for ($_i1153 = 0; $_i1153 < $_size1149; ++$_i1153)
 -            $_size1163 = 0;
 -            $_etype1166 = 0;
 -            $xfer += $input->readListBegin($_etype1166, $_size1163);
 -            for ($_i1167 = 0; $_i1167 < $_size1163; ++$_i1167)
++            $_size1170 = 0;
++            $_etype1173 = 0;
++            $xfer += $input->readListBegin($_etype1173, $_size1170);
++            for ($_i1174 = 0; $_i1174 < $_size1170; ++$_i1174)
              {
-               $elem1154 = null;
-               $xfer += $input->readString($elem1154);
-               $this->group_names []= $elem1154;
 -              $elem1168 = null;
 -              $xfer += $input->readString($elem1168);
 -              $this->group_names []= $elem1168;
++              $elem1175 = null;
++              $xfer += $input->readString($elem1175);
++              $this->group_names []= $elem1175;
              }
              $xfer += $input->readListEnd();
            } else {
@@@ -31962,9 -32020,9 +32018,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->part_vals));
          {
-           foreach ($this->part_vals as $iter1155)
 -          foreach ($this->part_vals as $iter1169)
++          foreach ($this->part_vals as $iter1176)
            {
-             $xfer += $output->writeString($iter1155);
 -            $xfer += $output->writeString($iter1169);
++            $xfer += $output->writeString($iter1176);
            }
          }
          $output->writeListEnd();
@@@ -31989,9 -32047,9 +32045,9 @@@
        {
          $output->writeListBegin(TType::STRING, count($this->group_names));
          {
-           foreach ($this->group_names as $iter1156)
 -          foreach ($this->group_names as $iter1170)
++          foreach ($this->group_names as $iter1177)
            {
-             $xfer += $output->writeString($iter1156);
 -            $xfer += $output->writeString($iter1170);
++            $xfer += $output->writeString($iter1177);
            }
          }
          $output->writeListEnd();
@@@ -32080,15 -32138,15 +32136,15 @@@ class ThriftHiveMetastore_get_partition
          case 0:
            if ($ftype == TType::LST) {
              $this->success = array();
-             $_size1157 = 0;
-             $_etype1160 = 0;
-             $xfer += $input->readListBegin($_etype1160, $_size1157);
-             for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
 -            $_size1171 = 0;
 -            $_etype1174 = 0;
 -            $xfer += $input->readListBegin($_etype1174, $_size1171);
 -            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
++            $_size1178 = 0;
++            $_etype1181 = 0;
++            $xfer += $input->readListBegin($_etype1181, $_size1178);
++            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
              {
-               $elem1162 = null;
-               $elem1162 = new \metastore\Partition();
-               $xfer += $elem1162->read($input);
-               $this->success []= $elem1162;
 -              $elem1176 = null;
 -              $elem1176 = new \metastore\Partition();
 -              $xfer += $elem1176->read($input);
 -     

<TRUNCATED>

[24/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
new file mode 100644
index 0000000..5758820
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogRequest.java
@@ -0,0 +1,949 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WriteNotificationLogRequest implements org.apache.thrift.TBase<WriteNotificationLogRequest, WriteNotificationLogRequest._Fields>, java.io.Serializable, Cloneable, Comparable<WriteNotificationLogRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WriteNotificationLogRequest");
+
+  private static final org.apache.thrift.protocol.TField TXN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnId", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)2);
+  private static final org.apache.thrift.protocol.TField DB_FIELD_DESC = new org.apache.thrift.protocol.TField("db", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField FILE_INFO_FIELD_DESC = new org.apache.thrift.protocol.TField("fileInfo", org.apache.thrift.protocol.TType.STRUCT, (short)5);
+  private static final org.apache.thrift.protocol.TField PARTITION_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionVals", org.apache.thrift.protocol.TType.LIST, (short)6);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WriteNotificationLogRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WriteNotificationLogRequestTupleSchemeFactory());
+  }
+
+  private long txnId; // required
+  private long writeId; // required
+  private String db; // required
+  private String table; // required
+  private InsertEventRequestData fileInfo; // required
+  private List<String> partitionVals; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    TXN_ID((short)1, "txnId"),
+    WRITE_ID((short)2, "writeId"),
+    DB((short)3, "db"),
+    TABLE((short)4, "table"),
+    FILE_INFO((short)5, "fileInfo"),
+    PARTITION_VALS((short)6, "partitionVals");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // TXN_ID
+          return TXN_ID;
+        case 2: // WRITE_ID
+          return WRITE_ID;
+        case 3: // DB
+          return DB;
+        case 4: // TABLE
+          return TABLE;
+        case 5: // FILE_INFO
+          return FILE_INFO;
+        case 6: // PARTITION_VALS
+          return PARTITION_VALS;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __TXNID_ISSET_ID = 0;
+  private static final int __WRITEID_ISSET_ID = 1;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.PARTITION_VALS};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("txnId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("writeId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.DB, new org.apache.thrift.meta_data.FieldMetaData("db", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FILE_INFO, new org.apache.thrift.meta_data.FieldMetaData("fileInfo", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, InsertEventRequestData.class)));
+    tmpMap.put(_Fields.PARTITION_VALS, new org.apache.thrift.meta_data.FieldMetaData("partitionVals", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriteNotificationLogRequest.class, metaDataMap);
+  }
+
+  public WriteNotificationLogRequest() {
+  }
+
+  public WriteNotificationLogRequest(
+    long txnId,
+    long writeId,
+    String db,
+    String table,
+    InsertEventRequestData fileInfo)
+  {
+    this();
+    this.txnId = txnId;
+    setTxnIdIsSet(true);
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+    this.db = db;
+    this.table = table;
+    this.fileInfo = fileInfo;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WriteNotificationLogRequest(WriteNotificationLogRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.txnId = other.txnId;
+    this.writeId = other.writeId;
+    if (other.isSetDb()) {
+      this.db = other.db;
+    }
+    if (other.isSetTable()) {
+      this.table = other.table;
+    }
+    if (other.isSetFileInfo()) {
+      this.fileInfo = new InsertEventRequestData(other.fileInfo);
+    }
+    if (other.isSetPartitionVals()) {
+      List<String> __this__partitionVals = new ArrayList<String>(other.partitionVals);
+      this.partitionVals = __this__partitionVals;
+    }
+  }
+
+  public WriteNotificationLogRequest deepCopy() {
+    return new WriteNotificationLogRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    setTxnIdIsSet(false);
+    this.txnId = 0;
+    setWriteIdIsSet(false);
+    this.writeId = 0;
+    this.db = null;
+    this.table = null;
+    this.fileInfo = null;
+    this.partitionVals = null;
+  }
+
+  public long getTxnId() {
+    return this.txnId;
+  }
+
+  public void setTxnId(long txnId) {
+    this.txnId = txnId;
+    setTxnIdIsSet(true);
+  }
+
+  public void unsetTxnId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TXNID_ISSET_ID);
+  }
+
+  /** Returns true if field txnId is set (has been assigned a value) and false otherwise */
+  public boolean isSetTxnId() {
+    return EncodingUtils.testBit(__isset_bitfield, __TXNID_ISSET_ID);
+  }
+
+  public void setTxnIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TXNID_ISSET_ID, value);
+  }
+
+  public long getWriteId() {
+    return this.writeId;
+  }
+
+  public void setWriteId(long writeId) {
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+  }
+
+  public void unsetWriteId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  /** Returns true if field writeId is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteId() {
+    return EncodingUtils.testBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  public void setWriteIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEID_ISSET_ID, value);
+  }
+
+  public String getDb() {
+    return this.db;
+  }
+
+  public void setDb(String db) {
+    this.db = db;
+  }
+
+  public void unsetDb() {
+    this.db = null;
+  }
+
+  /** Returns true if field db is set (has been assigned a value) and false otherwise */
+  public boolean isSetDb() {
+    return this.db != null;
+  }
+
+  public void setDbIsSet(boolean value) {
+    if (!value) {
+      this.db = null;
+    }
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public void unsetTable() {
+    this.table = null;
+  }
+
+  /** Returns true if field table is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable() {
+    return this.table != null;
+  }
+
+  public void setTableIsSet(boolean value) {
+    if (!value) {
+      this.table = null;
+    }
+  }
+
+  public InsertEventRequestData getFileInfo() {
+    return this.fileInfo;
+  }
+
+  public void setFileInfo(InsertEventRequestData fileInfo) {
+    this.fileInfo = fileInfo;
+  }
+
+  public void unsetFileInfo() {
+    this.fileInfo = null;
+  }
+
+  /** Returns true if field fileInfo is set (has been assigned a value) and false otherwise */
+  public boolean isSetFileInfo() {
+    return this.fileInfo != null;
+  }
+
+  public void setFileInfoIsSet(boolean value) {
+    if (!value) {
+      this.fileInfo = null;
+    }
+  }
+
+  public int getPartitionValsSize() {
+    return (this.partitionVals == null) ? 0 : this.partitionVals.size();
+  }
+
+  public java.util.Iterator<String> getPartitionValsIterator() {
+    return (this.partitionVals == null) ? null : this.partitionVals.iterator();
+  }
+
+  public void addToPartitionVals(String elem) {
+    if (this.partitionVals == null) {
+      this.partitionVals = new ArrayList<String>();
+    }
+    this.partitionVals.add(elem);
+  }
+
+  public List<String> getPartitionVals() {
+    return this.partitionVals;
+  }
+
+  public void setPartitionVals(List<String> partitionVals) {
+    this.partitionVals = partitionVals;
+  }
+
+  public void unsetPartitionVals() {
+    this.partitionVals = null;
+  }
+
+  /** Returns true if field partitionVals is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartitionVals() {
+    return this.partitionVals != null;
+  }
+
+  public void setPartitionValsIsSet(boolean value) {
+    if (!value) {
+      this.partitionVals = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case TXN_ID:
+      if (value == null) {
+        unsetTxnId();
+      } else {
+        setTxnId((Long)value);
+      }
+      break;
+
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
+    case DB:
+      if (value == null) {
+        unsetDb();
+      } else {
+        setDb((String)value);
+      }
+      break;
+
+    case TABLE:
+      if (value == null) {
+        unsetTable();
+      } else {
+        setTable((String)value);
+      }
+      break;
+
+    case FILE_INFO:
+      if (value == null) {
+        unsetFileInfo();
+      } else {
+        setFileInfo((InsertEventRequestData)value);
+      }
+      break;
+
+    case PARTITION_VALS:
+      if (value == null) {
+        unsetPartitionVals();
+      } else {
+        setPartitionVals((List<String>)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case TXN_ID:
+      return getTxnId();
+
+    case WRITE_ID:
+      return getWriteId();
+
+    case DB:
+      return getDb();
+
+    case TABLE:
+      return getTable();
+
+    case FILE_INFO:
+      return getFileInfo();
+
+    case PARTITION_VALS:
+      return getPartitionVals();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case TXN_ID:
+      return isSetTxnId();
+    case WRITE_ID:
+      return isSetWriteId();
+    case DB:
+      return isSetDb();
+    case TABLE:
+      return isSetTable();
+    case FILE_INFO:
+      return isSetFileInfo();
+    case PARTITION_VALS:
+      return isSetPartitionVals();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WriteNotificationLogRequest)
+      return this.equals((WriteNotificationLogRequest)that);
+    return false;
+  }
+
+  public boolean equals(WriteNotificationLogRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_txnId = true;
+    boolean that_present_txnId = true;
+    if (this_present_txnId || that_present_txnId) {
+      if (!(this_present_txnId && that_present_txnId))
+        return false;
+      if (this.txnId != that.txnId)
+        return false;
+    }
+
+    boolean this_present_writeId = true;
+    boolean that_present_writeId = true;
+    if (this_present_writeId || that_present_writeId) {
+      if (!(this_present_writeId && that_present_writeId))
+        return false;
+      if (this.writeId != that.writeId)
+        return false;
+    }
+
+    boolean this_present_db = true && this.isSetDb();
+    boolean that_present_db = true && that.isSetDb();
+    if (this_present_db || that_present_db) {
+      if (!(this_present_db && that_present_db))
+        return false;
+      if (!this.db.equals(that.db))
+        return false;
+    }
+
+    boolean this_present_table = true && this.isSetTable();
+    boolean that_present_table = true && that.isSetTable();
+    if (this_present_table || that_present_table) {
+      if (!(this_present_table && that_present_table))
+        return false;
+      if (!this.table.equals(that.table))
+        return false;
+    }
+
+    boolean this_present_fileInfo = true && this.isSetFileInfo();
+    boolean that_present_fileInfo = true && that.isSetFileInfo();
+    if (this_present_fileInfo || that_present_fileInfo) {
+      if (!(this_present_fileInfo && that_present_fileInfo))
+        return false;
+      if (!this.fileInfo.equals(that.fileInfo))
+        return false;
+    }
+
+    boolean this_present_partitionVals = true && this.isSetPartitionVals();
+    boolean that_present_partitionVals = true && that.isSetPartitionVals();
+    if (this_present_partitionVals || that_present_partitionVals) {
+      if (!(this_present_partitionVals && that_present_partitionVals))
+        return false;
+      if (!this.partitionVals.equals(that.partitionVals))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_txnId = true;
+    list.add(present_txnId);
+    if (present_txnId)
+      list.add(txnId);
+
+    boolean present_writeId = true;
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
+    boolean present_db = true && (isSetDb());
+    list.add(present_db);
+    if (present_db)
+      list.add(db);
+
+    boolean present_table = true && (isSetTable());
+    list.add(present_table);
+    if (present_table)
+      list.add(table);
+
+    boolean present_fileInfo = true && (isSetFileInfo());
+    list.add(present_fileInfo);
+    if (present_fileInfo)
+      list.add(fileInfo);
+
+    boolean present_partitionVals = true && (isSetPartitionVals());
+    list.add(present_partitionVals);
+    if (present_partitionVals)
+      list.add(partitionVals);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WriteNotificationLogRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetTxnId()).compareTo(other.isSetTxnId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTxnId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.txnId, other.txnId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetWriteId()).compareTo(other.isSetWriteId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeId, other.writeId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDb()).compareTo(other.isSetDb());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDb()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db, other.db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFileInfo()).compareTo(other.isSetFileInfo());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFileInfo()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fileInfo, other.fileInfo);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPartitionVals()).compareTo(other.isSetPartitionVals());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartitionVals()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionVals, other.partitionVals);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WriteNotificationLogRequest(");
+    boolean first = true;
+
+    sb.append("txnId:");
+    sb.append(this.txnId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("writeId:");
+    sb.append(this.writeId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("db:");
+    if (this.db == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.db);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("table:");
+    if (this.table == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("fileInfo:");
+    if (this.fileInfo == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.fileInfo);
+    }
+    first = false;
+    if (isSetPartitionVals()) {
+      if (!first) sb.append(", ");
+      sb.append("partitionVals:");
+      if (this.partitionVals == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partitionVals);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetTxnId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'txnId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetWriteId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'writeId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetDb()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'db' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTable()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'table' is unset! Struct:" + toString());
+    }
+
+    if (!isSetFileInfo()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'fileInfo' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+    if (fileInfo != null) {
+      fileInfo.validate();
+    }
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WriteNotificationLogRequestStandardSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogRequestStandardScheme getScheme() {
+      return new WriteNotificationLogRequestStandardScheme();
+    }
+  }
+
+  private static class WriteNotificationLogRequestStandardScheme extends StandardScheme<WriteNotificationLogRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // TXN_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.txnId = iprot.readI64();
+              struct.setTxnIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // WRITE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.writeId = iprot.readI64();
+              struct.setWriteIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.db = iprot.readString();
+              struct.setDbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table = iprot.readString();
+              struct.setTableIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // FILE_INFO
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.fileInfo = new InsertEventRequestData();
+              struct.fileInfo.read(iprot);
+              struct.setFileInfoIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // PARTITION_VALS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list764 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list764.size);
+                String _elem765;
+                for (int _i766 = 0; _i766 < _list764.size; ++_i766)
+                {
+                  _elem765 = iprot.readString();
+                  struct.partitionVals.add(_elem765);
+                }
+                iprot.readListEnd();
+              }
+              struct.setPartitionValsIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(TXN_ID_FIELD_DESC);
+      oprot.writeI64(struct.txnId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+      oprot.writeI64(struct.writeId);
+      oprot.writeFieldEnd();
+      if (struct.db != null) {
+        oprot.writeFieldBegin(DB_FIELD_DESC);
+        oprot.writeString(struct.db);
+        oprot.writeFieldEnd();
+      }
+      if (struct.table != null) {
+        oprot.writeFieldBegin(TABLE_FIELD_DESC);
+        oprot.writeString(struct.table);
+        oprot.writeFieldEnd();
+      }
+      if (struct.fileInfo != null) {
+        oprot.writeFieldBegin(FILE_INFO_FIELD_DESC);
+        struct.fileInfo.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      if (struct.partitionVals != null) {
+        if (struct.isSetPartitionVals()) {
+          oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
+            for (String _iter767 : struct.partitionVals)
+            {
+              oprot.writeString(_iter767);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WriteNotificationLogRequestTupleSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogRequestTupleScheme getScheme() {
+      return new WriteNotificationLogRequestTupleScheme();
+    }
+  }
+
+  private static class WriteNotificationLogRequestTupleScheme extends TupleScheme<WriteNotificationLogRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.txnId);
+      oprot.writeI64(struct.writeId);
+      oprot.writeString(struct.db);
+      oprot.writeString(struct.table);
+      struct.fileInfo.write(oprot);
+      BitSet optionals = new BitSet();
+      if (struct.isSetPartitionVals()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetPartitionVals()) {
+        {
+          oprot.writeI32(struct.partitionVals.size());
+          for (String _iter768 : struct.partitionVals)
+          {
+            oprot.writeString(_iter768);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.txnId = iprot.readI64();
+      struct.setTxnIdIsSet(true);
+      struct.writeId = iprot.readI64();
+      struct.setWriteIdIsSet(true);
+      struct.db = iprot.readString();
+      struct.setDbIsSet(true);
+      struct.table = iprot.readString();
+      struct.setTableIsSet(true);
+      struct.fileInfo = new InsertEventRequestData();
+      struct.fileInfo.read(iprot);
+      struct.setFileInfoIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        {
+          org.apache.thrift.protocol.TList _list769 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list769.size);
+          String _elem770;
+          for (int _i771 = 0; _i771 < _list769.size; ++_i771)
+          {
+            _elem770 = iprot.readString();
+            struct.partitionVals.add(_elem770);
+          }
+        }
+        struct.setPartitionValsIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java
new file mode 100644
index 0000000..fab4da2
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteNotificationLogResponse.java
@@ -0,0 +1,283 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WriteNotificationLogResponse implements org.apache.thrift.TBase<WriteNotificationLogResponse, WriteNotificationLogResponse._Fields>, java.io.Serializable, Cloneable, Comparable<WriteNotificationLogResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WriteNotificationLogResponse");
+
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WriteNotificationLogResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WriteNotificationLogResponseTupleSchemeFactory());
+  }
+
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+;
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriteNotificationLogResponse.class, metaDataMap);
+  }
+
+  public WriteNotificationLogResponse() {
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WriteNotificationLogResponse(WriteNotificationLogResponse other) {
+  }
+
+  public WriteNotificationLogResponse deepCopy() {
+    return new WriteNotificationLogResponse(this);
+  }
+
+  @Override
+  public void clear() {
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WriteNotificationLogResponse)
+      return this.equals((WriteNotificationLogResponse)that);
+    return false;
+  }
+
+  public boolean equals(WriteNotificationLogResponse that) {
+    if (that == null)
+      return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WriteNotificationLogResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WriteNotificationLogResponse(");
+    boolean first = true;
+
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WriteNotificationLogResponseStandardSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogResponseStandardScheme getScheme() {
+      return new WriteNotificationLogResponseStandardScheme();
+    }
+  }
+
+  private static class WriteNotificationLogResponseStandardScheme extends StandardScheme<WriteNotificationLogResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WriteNotificationLogResponseTupleSchemeFactory implements SchemeFactory {
+    public WriteNotificationLogResponseTupleScheme getScheme() {
+      return new WriteNotificationLogResponseTupleScheme();
+    }
+  }
+
+  private static class WriteNotificationLogResponseTupleScheme extends TupleScheme<WriteNotificationLogResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WriteNotificationLogResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+    }
+  }
+
+}
+


[34/46] hive git commit: HIVE-19792: Upgrade orc to 1.5.2 and enable decimal_64 schema evolution tests (Prasanth Jayachandran reviewed by Matt McCline)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d6fddac8/ql/src/test/results/clientpositive/tez/orc_merge12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_merge12.q.out b/ql/src/test/results/clientpositive/tez/orc_merge12.q.out
index acb2fb9..558aeab 100644
--- a/ql/src/test/results/clientpositive/tez/orc_merge12.q.out
+++ b/ql/src/test/results/clientpositive/tez/orc_merge12.q.out
@@ -161,8 +161,8 @@ Stripe Statistics:
     Column 6: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 7: count: 12288 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 442368
     Column 8: count: 12288 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 884736
-    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 11: count: 9174 hasNull: true true: 6138
     Column 12: count: 9173 hasNull: true true: 3983
     Column 13: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -173,8 +173,8 @@ Stripe Statistics:
     Column 18: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 19: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 20: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 23: count: 9174 hasNull: true true: 6138
     Column 24: count: 9173 hasNull: true true: 3983
     Column 25: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -185,8 +185,8 @@ Stripe Statistics:
     Column 30: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 31: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 32: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 35: count: 9174 hasNull: true true: 6138
     Column 36: count: 9173 hasNull: true true: 3983
   Stripe 2:
@@ -199,8 +199,8 @@ Stripe Statistics:
     Column 6: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 7: count: 12288 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 442368
     Column 8: count: 12288 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 884736
-    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 9: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 10: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 11: count: 9174 hasNull: true true: 6138
     Column 12: count: 9173 hasNull: true true: 3983
     Column 13: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -211,8 +211,8 @@ Stripe Statistics:
     Column 18: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 19: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 20: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 21: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 22: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 23: count: 9174 hasNull: true true: 6138
     Column 24: count: 9173 hasNull: true true: 3983
     Column 25: count: 9173 hasNull: true min: -64 max: 62 sum: -39856
@@ -223,8 +223,8 @@ Stripe Statistics:
     Column 30: count: 9174 hasNull: true min: -16379.0 max: 9763215.5639 sum: 5.62236530305E7
     Column 31: count: 9174 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 127881
     Column 32: count: 9173 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 149134
-    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+    Column 33: count: 9173 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+    Column 34: count: 9174 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
     Column 35: count: 9174 hasNull: true true: 6138
     Column 36: count: 9173 hasNull: true true: 3983
 
@@ -238,8 +238,8 @@ File Statistics:
   Column 6: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
   Column 7: count: 24576 hasNull: false min: 00020767-dd8f-4f4d-bd68-4b7be64b8e44 max: fffa3516-e219-4027-b0d3-72bb2e676c52 sum: 884736
   Column 8: count: 24576 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 1769472
-  Column 9: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-  Column 10: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+  Column 9: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+  Column 10: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
   Column 11: count: 18348 hasNull: true true: 12276
   Column 12: count: 18346 hasNull: true true: 7966
   Column 13: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
@@ -250,8 +250,8 @@ File Statistics:
   Column 18: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
   Column 19: count: 18348 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 255762
   Column 20: count: 18346 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 298268
-  Column 21: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-  Column 22: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+  Column 21: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+  Column 22: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
   Column 23: count: 18348 hasNull: true true: 12276
   Column 24: count: 18346 hasNull: true true: 7966
   Column 25: count: 18346 hasNull: true min: -64 max: 62 sum: -79712
@@ -262,8 +262,8 @@ File Statistics:
   Column 30: count: 18348 hasNull: true min: -16379.0 max: 9763215.5639 sum: 1.12447306061E8
   Column 31: count: 18348 hasNull: true min: 0042l0d5rPD6sMlJ7Ue0q max: yy2GiGM sum: 255762
   Column 32: count: 18346 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 298268
-  Column 33: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
-  Column 34: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808
+  Column 33: count: 18346 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
+  Column 34: count: 18348 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808
   Column 35: count: 18348 hasNull: true true: 12276
   Column 36: count: 18346 hasNull: true true: 7966
 
@@ -457,11 +457,11 @@ Stripes:
       Entry 0: count: 10000 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 720000 positions: 0,0,0,0,0
       Entry 1: count: 2288 hasNull: false min: 00124556-8383-44c4-a28b-7a413de74ccc4137606f-2cf7-43fb-beff-b6d374fd15ec max: ffde3bce-bb56-4fa9-81d7-146ca2eab946225c18e0-0002-4d07-9853-12c92c0f5637 sum: 164736 positions: 384237,64640,0,76,272
     Row group indices for column 9:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 10:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 11:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -493,11 +493,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 21:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 22:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 23:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -529,11 +529,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 33:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 34:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 35:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -729,11 +729,11 @@ Stripes:
       Entry 0: count: 10000 hasNull: false min: 000976f7-7075-4f3f-a564-5a375fafcc101416a2b7-7f64-41b7-851f-97d15405037e max: fffd0642-5f01-48cd-8d97-3428faee49e9b39f2b4c-efdc-4e5f-9ab5-4aa5394cb156 sum: 720000 positions: 0,0,0,0,0
       Entry 1: count: 2288 hasNull: false min: 00124556-8383-44c4-a28b-7a413de74ccc4137606f-2cf7-43fb-beff-b6d374fd15ec max: ffde3bce-bb56-4fa9-81d7-146ca2eab946225c18e0-0002-4d07-9853-12c92c0f5637 sum: 164736 positions: 384237,64640,0,76,272
     Row group indices for column 9:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 10:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 11:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -765,11 +765,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 21:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 22:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 23:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4
@@ -801,11 +801,11 @@ Stripes:
       Entry 0: count: 6889 hasNull: true min: 0034fkcXMQI3 max: yyt0S8WorA sum: 109415 positions: 0,0,0,0,0,0,0
       Entry 1: count: 2284 hasNull: true min: 004J8y max: yjDBo sum: 39719 positions: 0,168,8,0,0,9196,262
     Row group indices for column 33:
-      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:43.64 max UTC: 1969-12-31 08:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
+      Entry 0: count: 7909 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1264 hasNull: true min: 1969-12-31 15:59:43.64 max: 1969-12-31 16:00:30.808 positions: 0,182,100,0,0,22588,218,0,11248,258
     Row group indices for column 34:
-      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
-      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 min UTC: 1969-12-31 07:59:30.929 max UTC: 1969-12-31 08:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
+      Entry 0: count: 7924 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,0,0,0,0,0,0,0,0,0
+      Entry 1: count: 1250 hasNull: true min: 1969-12-31 15:59:30.929 max: 1969-12-31 16:00:30.808 positions: 0,126,97,0,0,20399,273,0,10229,272
     Row group indices for column 35:
       Entry 0: count: 7140 hasNull: true true: 5115 positions: 0,0,0,0,0,0,0,0
       Entry 1: count: 2034 hasNull: true true: 1023 positions: 0,126,98,0,0,520,126,4


[46/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703


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

Branch: refs/heads/master-txnstats
Commit: 4db8b1c66c459d6c0487565c3c176055a910da2a
Parents: d7bbc20 2ca70b9
Author: sergey <se...@apache.org>
Authored: Tue Jul 3 10:47:09 2018 -0700
Committer: sergey <se...@apache.org>
Committed: Tue Jul 3 10:47:09 2018 -0700

----------------------------------------------------------------------
 .../hive/accumulo/serde/AccumuloSerDe.java      |    6 +-
 .../java/org/apache/hive/beeline/BeeLine.java   |    2 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |    2 +-
 .../org/apache/hive/beeline/HiveSchemaTool.java | 1563 ------
 .../hive/beeline/schematool/HiveSchemaTool.java |  415 ++
 .../schematool/HiveSchemaToolCommandLine.java   |  286 ++
 .../beeline/schematool/HiveSchemaToolTask.java  |   32 +
 .../HiveSchemaToolTaskAlterCatalog.java         |   90 +
 .../HiveSchemaToolTaskCreateCatalog.java        |  132 +
 .../schematool/HiveSchemaToolTaskInfo.java      |   43 +
 .../schematool/HiveSchemaToolTaskInit.java      |   73 +
 .../HiveSchemaToolTaskMoveDatabase.java         |   96 +
 .../schematool/HiveSchemaToolTaskMoveTable.java |  142 +
 .../schematool/HiveSchemaToolTaskUpgrade.java   |  116 +
 .../schematool/HiveSchemaToolTaskValidate.java  |  631 +++
 .../apache/hive/beeline/TestHiveSchemaTool.java |   92 -
 .../beeline/schematool/TestHiveSchemaTool.java  |   92 +
 bin/ext/schemaTool.sh                           |    4 +-
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   11 +-
 .../listener/DbNotificationListener.java        |  209 +-
 .../listener/DummyRawStoreFailEvent.java        |   17 +-
 .../listener/TestDbNotificationListener.java    |    5 +
 .../hadoop/hive/ql/parse/TestExportImport.java  |   55 +
 .../hive/ql/parse/TestReplicationScenarios.java |   72 -
 .../TestReplicationScenariosAcidTables.java     |  602 ++-
 ...TestReplicationScenariosAcrossInstances.java |   92 +-
 .../hadoop/hive/ql/parse/WarehouseInstance.java |    5 +
 .../TestHiveAuthorizerCheckInvocation.java      |   94 +-
 .../org/apache/hive/beeline/TestSchemaTool.java |  801 ---
 .../hive/beeline/TestSchemaToolCatalogOps.java  |  417 --
 .../hive/beeline/schematool/TestSchemaTool.java |  826 ++++
 .../schematool/TestSchemaToolCatalogOps.java    |  478 ++
 .../hive/jdbc/TestTriggersWorkloadManager.java  |    2 +
 .../test/resources/testconfiguration.properties |    6 +-
 .../apache/hadoop/hive/ql/QOutProcessor.java    |    2 +
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    3 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |    4 +-
 .../hive/llap/io/api/impl/LlapRecordReader.java |   64 +
 pom.xml                                         |    2 +-
 .../metastore/SynchronizedMetaStoreClient.java  |    5 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |   10 +-
 .../apache/hadoop/hive/ql/DriverContext.java    |   29 +-
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |    4 +-
 .../apache/hadoop/hive/ql/exec/ExportTask.java  |    2 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |   19 +-
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    6 +-
 .../apache/hadoop/hive/ql/exec/Operator.java    |   11 +-
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |    5 +-
 .../apache/hadoop/hive/ql/exec/ReplTxnTask.java |   31 +-
 .../hive/ql/exec/SerializationUtilities.java    |   22 -
 .../hadoop/hive/ql/exec/TableScanOperator.java  |    1 +
 .../hadoop/hive/ql/exec/repl/ReplDumpTask.java  |    4 +-
 .../IncrementalLoadTasksBuilder.java            |   73 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |   15 +-
 .../ql/exec/spark/session/SparkSessionImpl.java |   10 +-
 .../exec/spark/status/LocalSparkJobMonitor.java |    4 +-
 .../spark/status/RemoteSparkJobMonitor.java     |    5 +-
 .../ql/exec/spark/status/RenderStrategy.java    |  246 +
 .../ql/exec/spark/status/SparkJobMonitor.java   |  157 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |   37 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   24 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   18 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |    7 +-
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   10 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  166 +-
 .../hadoop/hive/ql/metadata/HiveUtils.java      |   11 +-
 .../apache/hadoop/hive/ql/metadata/Table.java   |    7 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |    4 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   15 +-
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |   28 +-
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |    5 +-
 .../hadoop/hive/ql/parse/GenTezUtils.java       |    2 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   83 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |   28 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  128 +-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   32 +-
 .../hadoop/hive/ql/parse/repl/CopyUtils.java    |    2 +-
 .../hadoop/hive/ql/parse/repl/dump/Utils.java   |    8 +-
 .../repl/dump/events/CommitTxnHandler.java      |  125 +-
 .../ql/parse/repl/dump/events/EventHandler.java |   23 +-
 .../parse/repl/dump/events/InsertHandler.java   |    4 +
 .../parse/repl/load/UpdatedMetaDataTracker.java |  124 +-
 .../repl/load/message/AbortTxnHandler.java      |    7 +-
 .../repl/load/message/AllocWriteIdHandler.java  |    2 +-
 .../repl/load/message/CommitTxnHandler.java     |   78 +-
 .../parse/repl/load/message/MessageHandler.java |    8 +-
 .../parse/repl/load/message/OpenTxnHandler.java |    7 +-
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |   18 +-
 .../apache/hadoop/hive/ql/plan/MoveWork.java    |   12 +-
 .../apache/hadoop/hive/ql/plan/ReplTxnWork.java |   15 +
 .../hadoop/hive/ql/reexec/ReExecDriver.java     |    2 +-
 .../hive/ql/stats/StatsCollectionContext.java   |   12 +-
 .../hive/ql/stats/fs/FSStatsPublisher.java      |   15 +-
 .../apache/hadoop/hive/ql/TestTxnNoBuckets.java |   41 +
 .../exec/spark/TestSparkInvalidFileFormat.java  |   81 +
 .../hive/ql/exec/spark/TestSparkTask.java       |    1 +
 .../exec/spark/status/TestSparkJobMonitor.java  |   29 +-
 .../ql/parse/TestMacroSemanticAnalyzer.java     |    9 +-
 .../hadoop/hive/ql/parse/TestQBCompact.java     |   15 +-
 .../authorization/AuthorizationTestUtil.java    |    8 +-
 .../spark_submit_negative_executor_cores.q      |    5 +
 .../spark_submit_negative_executor_memory.q     |    5 +
 .../queries/clientpositive/autoColumnStats_9.q  |    3 +
 .../clientpositive/orc_schema_evolution_float.q |    2 -
 .../queries/clientpositive/repl_2_exim_basic.q  |    1 +
 .../schema_evol_orc_nonvec_part_all_primitive.q |    2 -
 ...evol_orc_nonvec_part_all_primitive_llap_io.q |    2 -
 .../schema_evol_orc_vec_part_all_primitive.q    |    2 -
 ...ma_evol_orc_vec_part_all_primitive_llap_io.q |    2 -
 .../clientpositive/type_change_test_int.q       |    3 -
 .../type_change_test_int_vectorized.q           |    2 -
 .../test/queries/clientpositive/unicode_data.q  |   15 +
 .../queries/clientpositive/union_fast_stats.q   |    6 +
 .../queries/clientpositive/union_rowcounts.q    |   51 +
 .../test/queries/clientpositive/union_stats.q   |   44 +-
 .../vector_llap_io_data_conversion.q            |   19 +
 .../exim_03_nonpart_noncompat_colschema.q.out   |    2 +-
 .../exim_04_nonpart_noncompat_colnumber.q.out   |    2 +-
 .../exim_05_nonpart_noncompat_coltype.q.out     |    2 +-
 .../exim_06_nonpart_noncompat_storage.q.out     |    2 +-
 .../exim_07_nonpart_noncompat_ifof.q.out        |    2 +-
 .../exim_08_nonpart_noncompat_serde.q.out       |    2 +-
 .../exim_09_nonpart_noncompat_serdeparam.q.out  |    2 +-
 .../exim_10_nonpart_noncompat_bucketing.q.out   |    2 +-
 .../exim_11_nonpart_noncompat_sorting.q.out     |    2 +-
 .../exim_12_nonnative_export.q.out              |    2 +-
 .../exim_13_nonnative_import.q.out              |    2 +-
 .../clientnegative/exim_14_nonpart_part.q.out   |    2 +-
 .../clientnegative/exim_15_part_nonpart.q.out   |    2 +-
 .../exim_16_part_noncompat_schema.q.out         |    2 +-
 .../exim_19_external_over_existing.q.out        |    2 +-
 .../exim_21_part_managed_external.q.out         |    2 +-
 .../spark_submit_negative_executor_cores.q.out  |    5 +
 .../spark_submit_negative_executor_memory.q.out |    5 +
 .../clientpositive/autoColumnStats_9.q.out      |   13 +-
 .../clientpositive/llap/multiMapJoin1.q.out     |  150 +-
 .../clientpositive/llap/orc_merge11.q.out       |   54 +-
 ...schema_evol_orc_vec_part_all_primitive.q.out |   19 +-
 ...vol_orc_vec_part_all_primitive_llap_io.q.out |   19 +-
 .../clientpositive/llap/union_fast_stats.q.out  |   58 +-
 .../clientpositive/llap/union_rowcounts.q.out   |  180 +
 .../clientpositive/llap/union_stats.q.out       |  145 +-
 .../llap/vector_llap_io_data_conversion.q.out   |  187 +
 .../results/clientpositive/orc_file_dump.q.out  |   24 +-
 .../results/clientpositive/orc_merge11.q.out    |   54 +-
 .../results/clientpositive/orc_merge12.q.out    |   84 +-
 .../clientpositive/tez/orc_merge12.q.out        |   84 +-
 .../results/clientpositive/unicode_data.q.out   |  123 +
 .../results/clientpositive/union_stats.q.out    |  243 +
 .../apache/hadoop/hive/serde2/Deserializer.java |    2 +
 .../org/apache/hive/service/ServiceUtils.java   |    5 +-
 .../cli/SparkProgressMonitorStatusMapper.java   |   52 +
 .../service/cli/thrift/ThriftCLIService.java    |    5 +-
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |    3 +
 .../hive/spark/client/AbstractSparkClient.java  |   25 +-
 .../spark/client/SparkSubmitSparkClient.java    |   22 +-
 .../apache/hive/spark/client/rpc/RpcServer.java |   21 +-
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2649 +++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  126 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 4003 ++++++++-------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  218 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../api/AllocateTableWriteIdsRequest.java       |   68 +-
 .../api/AllocateTableWriteIdsResponse.java      |   36 +-
 .../metastore/api/AlterPartitionsRequest.java   |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/CommitTxnRequest.java    |  168 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../hive/metastore/api/CreationMetadata.java    |   32 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |   32 +-
 .../metastore/api/GetValidWriteIdsResponse.java |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |  227 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../hive/metastore/api/Materialization.java     |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../api/ReplTblWriteIdStateRequest.java         |   32 +-
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableValidWriteIds.java  |   32 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 4564 +++++++++++-------
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../hive/metastore/api/WriteEventInfo.java      | 1012 ++++
 .../api/WriteNotificationLogRequest.java        |  949 ++++
 .../api/WriteNotificationLogResponse.java       |  283 ++
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1610 +++---
 .../src/gen/thrift/gen-php/metastore/Types.php  | 1628 +++++--
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1125 +++--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  947 ++--
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   86 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   54 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   86 +
 .../hive/metastore/HiveMetaStoreClient.java     |   10 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   16 +-
 .../hive/metastore/MetaStoreEventListener.java  |   12 +
 .../metastore/MetaStoreListenerNotifier.java    |    6 +
 .../hadoop/hive/metastore/ObjectStore.java      |   65 +-
 .../apache/hadoop/hive/metastore/RawStore.java  |   16 +-
 .../hive/metastore/ReplChangeManager.java       |   10 +-
 .../hive/metastore/cache/CachedStore.java       |   13 +-
 .../hive/metastore/events/AcidWriteEvent.java   |   91 +
 .../metastore/messaging/AcidWriteMessage.java   |   50 +
 .../metastore/messaging/CommitTxnMessage.java   |   23 +
 .../hive/metastore/messaging/EventMessage.java  |    3 +-
 .../messaging/MessageDeserializer.java          |    9 +
 .../metastore/messaging/MessageFactory.java     |   12 +
 .../messaging/json/JSONAcidWriteMessage.java    |  150 +
 .../messaging/json/JSONCommitTxnMessage.java    |   95 +
 .../messaging/json/JSONMessageDeserializer.java |    9 +
 .../messaging/json/JSONMessageFactory.java      |    8 +
 .../model/MTxnWriteNotificationLog.java         |  123 +
 .../hive/metastore/tools/SQLGenerator.java      |    9 +
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   28 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  187 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |   11 +
 .../hadoop/hive/metastore/utils/FileUtils.java  |   12 +-
 .../src/main/resources/package.jdo              |   35 +
 .../main/sql/derby/hive-schema-3.1.0.derby.sql  |   15 +
 .../main/sql/derby/hive-schema-4.0.0.derby.sql  |   15 +
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |    1 -
 .../sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql  |   16 +
 .../main/sql/mssql/hive-schema-3.1.0.mssql.sql  |   17 +
 .../main/sql/mssql/hive-schema-4.0.0.mssql.sql  |   17 +
 .../sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql  |   16 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |    1 -
 .../main/sql/mysql/hive-schema-3.1.0.mysql.sql  |   16 +
 .../main/sql/mysql/hive-schema-4.0.0.mysql.sql  |   16 +
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |    4 +-
 .../sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql  |   16 +
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |    1 -
 .../sql/oracle/hive-schema-3.1.0.oracle.sql     |   15 +
 .../sql/oracle/hive-schema-4.0.0.oracle.sql     |   15 +
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |    3 +-
 .../oracle/upgrade-3.0.0-to-3.1.0.oracle.sql    |   16 +
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |    2 -
 .../sql/postgres/hive-schema-3.1.0.postgres.sql |   15 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql |   15 +
 .../upgrade-3.0.0-to-3.1.0.postgres.sql         |   16 +
 .../src/main/thrift/hive_metastore.thrift       |   30 +-
 .../DummyRawStoreControlledCommit.java          |   13 +-
 .../DummyRawStoreForJdoConnection.java          |   12 +-
 .../HiveMetaStoreClientPreCatalog.java          |   10 +-
 .../hadoop/hive/metastore/TestObjectStore.java  |    8 +-
 .../hive/streaming/AbstractRecordWriter.java    |    2 +-
 testutils/ptest2/pom.xml                        |    2 +-
 262 files changed, 22305 insertions(+), 11434 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --cc ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index f126550,16ba82e..19dbc64
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@@ -2032,9 -1892,31 +2017,31 @@@ public class AcidUtils 
      return null;
    }
  
+   //Get the first level acid directory (if any) from a given path
+   public static String getFirstLevelAcidDirPath(Path dataPath, FileSystem fileSystem) throws IOException {
+     if (dataPath == null) {
+       return null;
+     }
+     String firstLevelAcidDir = getAcidSubDir(dataPath);
+     if (firstLevelAcidDir != null) {
+       return firstLevelAcidDir;
+     }
+ 
+     String acidDirPath = getFirstLevelAcidDirPath(dataPath.getParent(), fileSystem);
+     if (acidDirPath == null) {
+       return null;
+     }
+ 
+     // We need the path for directory so no need to append file name
+     if (fileSystem.isDirectory(dataPath)) {
+       return acidDirPath + Path.SEPARATOR + dataPath.getName();
+     }
+     return acidDirPath;
+   }
+ 
    public static boolean isAcidEnabled(HiveConf hiveConf) {
 -    String txnMgr = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER);
 -    boolean concurrency =  hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);
 +    String txnMgr = hiveConf.getVar(ConfVars.HIVE_TXN_MANAGER);
 +    boolean concurrency =  hiveConf.getBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY);
      String dbTxnMgr = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager";
      if (txnMgr.equals(dbTxnMgr) && concurrency) {
        return true;

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/HiveTxnManager.java
----------------------------------------------------------------------

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


[32/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
index 9e0ce82..5e113c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/incremental/IncrementalLoadTasksBuilder.java
@@ -179,7 +179,7 @@ public class IncrementalLoadTasksBuilder {
       Database database;
       try {
         database = Hive.get().getDatabase(dbName);
-        return isEventNotReplayed(database.getParameters(), dir, dumpType);
+        return database == null ? true : isEventNotReplayed(database.getParameters(), dir, dumpType);
       } catch (HiveException e) {
         //may be the db is getting created in this load
         log.debug("failed to get the database " + dbName);
@@ -255,50 +255,55 @@ public class IncrementalLoadTasksBuilder {
     return updateReplIdTask;
   }
 
-  private List<Task<? extends Serializable>> addUpdateReplStateTasks(boolean isDatabaseLoad,
-                                           UpdatedMetaDataTracker updatedMetadata,
-                                           List<Task<? extends Serializable>> importTasks) throws SemanticException {
-    String replState = updatedMetadata.getReplicationState();
-    String database = updatedMetadata.getDatabase();
-    String table = updatedMetadata.getTable();
-
-    // If no import tasks generated by the event or no table updated for table level load, then no
-    // need to update the repl state to any object.
-    if (importTasks.isEmpty() || (!isDatabaseLoad && (table == null))) {
-      log.debug("No objects need update of repl state: Either 0 import tasks or table level load");
+  private List<Task<? extends Serializable>> addUpdateReplStateTasks(
+          boolean isDatabaseLoad,
+          UpdatedMetaDataTracker updatedMetaDataTracker,
+          List<Task<? extends Serializable>> importTasks) throws SemanticException {
+    // If no import tasks generated by the event then no need to update the repl state to any object.
+    if (importTasks.isEmpty()) {
+      log.debug("No objects need update of repl state: 0 import tasks");
       return importTasks;
     }
 
     // Create a barrier task for dependency collection of import tasks
-    Task<? extends Serializable> barrierTask = TaskFactory.get(new DependencyCollectionWork());
-
-    // Link import tasks to the barrier task which will in-turn linked with repl state update tasks
-    for (Task<? extends Serializable> t : importTasks){
-      t.addDependentTask(barrierTask);
-      log.debug("Added {}:{} as a precursor of barrier task {}:{}",
-              t.getClass(), t.getId(), barrierTask.getClass(), barrierTask.getId());
-    }
-
+    Task<? extends Serializable> barrierTask = TaskFactory.get(new DependencyCollectionWork(), conf);
     List<Task<? extends Serializable>> tasks = new ArrayList<>();
     Task<? extends Serializable> updateReplIdTask;
 
-    // If any partition is updated, then update repl state in partition object
-    for (final Map<String, String> partSpec : updatedMetadata.getPartitions()) {
-      updateReplIdTask = tableUpdateReplStateTask(database, table, partSpec, replState, barrierTask);
-      tasks.add(updateReplIdTask);
+    for (UpdatedMetaDataTracker.UpdateMetaData updateMetaData : updatedMetaDataTracker.getUpdateMetaDataList()) {
+      String replState = updateMetaData.getReplState();
+      String dbName = updateMetaData.getDbName();
+      String tableName = updateMetaData.getTableName();
+      // If any partition is updated, then update repl state in partition object
+      for (final Map<String, String> partSpec : updateMetaData.getPartitionsList()) {
+        updateReplIdTask = tableUpdateReplStateTask(dbName, tableName, partSpec, replState, barrierTask);
+        tasks.add(updateReplIdTask);
+      }
+
+      if (tableName != null) {
+        // If any table/partition is updated, then update repl state in table object
+        updateReplIdTask = tableUpdateReplStateTask(dbName, tableName, null, replState, barrierTask);
+        tasks.add(updateReplIdTask);
+      }
+
+      // For table level load, need not update replication state for the database
+      if (isDatabaseLoad) {
+        // If any table/partition is updated, then update repl state in db object
+        updateReplIdTask = dbUpdateReplStateTask(dbName, replState, barrierTask);
+        tasks.add(updateReplIdTask);
+      }
     }
 
-    if (table != null) {
-      // If any table/partition is updated, then update repl state in table object
-      updateReplIdTask = tableUpdateReplStateTask(database, table, null, replState, barrierTask);
-      tasks.add(updateReplIdTask);
+    if (tasks.isEmpty()) {
+      log.debug("No objects need update of repl state: 0 update tracker tasks");
+      return importTasks;
     }
 
-    // For table level load, need not update replication state for the database
-    if (isDatabaseLoad) {
-      // If any table/partition is updated, then update repl state in db object
-      updateReplIdTask = dbUpdateReplStateTask(database, replState, barrierTask);
-      tasks.add(updateReplIdTask);
+    // Link import tasks to the barrier task which will in-turn linked with repl state update tasks
+    for (Task<? extends Serializable> t : importTasks){
+      t.addDependentTask(barrierTask);
+      log.debug("Added {}:{} as a precursor of barrier task {}:{}",
+              t.getClass(), t.getId(), barrierTask.getClass(), barrierTask.getId());
     }
 
     // At least one task would have been added to update the repl state

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 7fce67f..16ba82e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.hive.ql.exec.Utilities.COPY_KEYWORD;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -71,21 +70,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-import java.io.Serializable;
 import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import static org.apache.hadoop.hive.ql.exec.Utilities.COPY_KEYWORD;
-
 
 /**
  * Utilities that are shared by all of the ACID input and output formats. They
@@ -1907,6 +1892,28 @@ public class AcidUtils {
     return null;
   }
 
+  //Get the first level acid directory (if any) from a given path
+  public static String getFirstLevelAcidDirPath(Path dataPath, FileSystem fileSystem) throws IOException {
+    if (dataPath == null) {
+      return null;
+    }
+    String firstLevelAcidDir = getAcidSubDir(dataPath);
+    if (firstLevelAcidDir != null) {
+      return firstLevelAcidDir;
+    }
+
+    String acidDirPath = getFirstLevelAcidDirPath(dataPath.getParent(), fileSystem);
+    if (acidDirPath == null) {
+      return null;
+    }
+
+    // We need the path for directory so no need to append file name
+    if (fileSystem.isDirectory(dataPath)) {
+      return acidDirPath + Path.SEPARATOR + dataPath.getName();
+    }
+    return acidDirPath;
+  }
+
   public static boolean isAcidEnabled(HiveConf hiveConf) {
     String txnMgr = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_MANAGER);
     boolean concurrency =  hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index bcc0508..ec8527e 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -26,30 +26,32 @@ import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.StringInternUtils;
+import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hive.common.util.HiveStringUtils;
+import org.apache.hive.common.util.Ref;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.common.JavaUtils;
-import org.apache.hadoop.hive.common.StringInternUtils;
-import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
-import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
 import org.apache.hadoop.hive.llap.io.api.LlapIo;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.ql.exec.Operator;
-import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner;
@@ -62,8 +64,6 @@ import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.PartitionDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
-import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc;
-import org.apache.hadoop.hive.ql.plan.VectorPartitionDesc.VectorMapOperatorReadType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
 import org.apache.hadoop.hive.serde2.Deserializer;
@@ -78,10 +78,7 @@ import org.apache.hadoop.mapred.JobConfigurable;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hive.common.util.Ref;
 import org.apache.hive.common.util.ReflectionUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * HiveInputFormat is a parameterized InputFormat which looks at the path name
@@ -460,8 +457,9 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
       InputFormat inputFormat, Class<? extends InputFormat> inputFormatClass, int splits,
       TableDesc table, List<InputSplit> result)
           throws IOException {
+    String tableName = table.getTableName();
     ValidWriteIdList validWriteIdList = AcidUtils.getTableValidWriteIdList(
-        conf, table.getTableName());
+        conf, tableName == null ? null : HiveStringUtils.normalizeIdentifier(tableName));
     ValidWriteIdList validMmWriteIdList = getMmValidWriteIds(conf, table, validWriteIdList);
 
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 4fd1d4e..78980fa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
 import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
 import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Context;
@@ -638,14 +639,15 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
-  public void replCommitTxn(String replPolicy, long srcTxnId) throws LockException {
+  public void replCommitTxn(CommitTxnRequest rqst) throws LockException {
     try {
-      getMS().replCommitTxn(srcTxnId, replPolicy);
+      getMS().replCommitTxn(rqst);
     } catch (NoSuchTxnException e) {
-      LOG.error("Metastore could not find " + JavaUtils.txnIdToString(srcTxnId));
-      throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(srcTxnId));
+      LOG.error("Metastore could not find " + JavaUtils.txnIdToString(rqst.getTxnid()));
+      throw new LockException(e, ErrorMsg.TXN_NO_SUCH_TRANSACTION, JavaUtils.txnIdToString(rqst.getTxnid()));
     } catch (TxnAbortedException e) {
-      LockException le = new LockException(e, ErrorMsg.TXN_ABORTED, JavaUtils.txnIdToString(srcTxnId), e.getMessage());
+      LockException le = new LockException(e, ErrorMsg.TXN_ABORTED,
+              JavaUtils.txnIdToString(rqst.getTxnid()), e.getMessage());
       LOG.error(le.getMessage());
       throw le;
     } catch (TException e) {
@@ -1013,7 +1015,11 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
     assert isTxnOpen();
     return stmtId++;
   }
-
+  @Override
+  public int getCurrentStmtId() {
+    assert isTxnOpen();
+    return stmtId;
+  }
   @Override
   public long getTableWriteId(String dbName, String tableName) throws LockException {
     assert isTxnOpen();

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
index ab9d67e..1feddeb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DummyTxnManager.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -74,6 +75,10 @@ class DummyTxnManager extends HiveTxnManagerImpl {
     return 0;
   }
   @Override
+  public int getCurrentStmtId() {
+    return  0;
+  }
+  @Override
   public long getTableWriteId(String dbName, String tableName) throws LockException {
     return 0L;
   }
@@ -220,7 +225,7 @@ class DummyTxnManager extends HiveTxnManagerImpl {
   }
 
   @Override
-  public void replCommitTxn(String replPolicy, long srcTxnId) throws LockException {
+  public void replCommitTxn(CommitTxnRequest rqst) throws LockException {
     // No-op
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/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 5f68e08..9575552 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidTxnWriteIdList;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.ql.Context;
@@ -61,11 +62,11 @@ public interface HiveTxnManager {
 
   /**
    * Commit the transaction in target cluster.
-   * @param replPolicy Replication policy to uniquely identify the source cluster.
-   * @param srcTxnId The id of the transaction at the source cluster
+   *
+   * @param rqst Commit transaction request having information related to commit txn and write events.
    * @throws LockException in case of failure to commit the transaction.
    */
-  void replCommitTxn(String replPolicy, long srcTxnId) throws LockException;
+  void replCommitTxn(CommitTxnRequest rqst) throws LockException;
 
  /**
    * Abort the transaction in target cluster.
@@ -295,6 +296,9 @@ public interface HiveTxnManager {
    */
   int getStmtIdAndIncrement();
 
+  // Can be used by operation to set the stmt id when allocation is done somewhere else.
+  int getCurrentStmtId();
+
   /**
    * Acquire the materialization rebuild lock for a given view. We need to specify the fully
    * qualified name of the materialized view and the open transaction ID so we can identify

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/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 c3809d8..c2ffe02 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
@@ -159,6 +159,7 @@ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator;
@@ -1719,6 +1720,13 @@ public class Hive {
       List<Path> newFiles = Collections.synchronizedList(new ArrayList<Path>());
 
       perfLogger.PerfLogBegin("MoveTask", PerfLogger.FILE_MOVES);
+      
+      // If config is set, table is not temporary and partition being inserted exists, capture
+      // the list of files added. For not yet existing partitions (insert overwrite to new partition
+      // or dynamic partition inserts), the add partition event will capture the list of files added.
+      if (areEventsForDmlNeeded(tbl, oldPart)) {
+        newFiles = Collections.synchronizedList(new ArrayList<Path>());
+      }
 
       // Note: the stats for ACID tables do not have any coordination with either Hive ACID logic
       //       like txn commits, time outs, etc.; nor the lower level sync in metastore pertaining
@@ -1731,8 +1739,8 @@ public class Hive {
           Utilities.FILE_OP_LOGGER.trace("not moving " + loadPath + " to " + newPartPath + " (MM)");
         }
         assert !isAcidIUDoperation;
-        if (areEventsForDmlNeeded(tbl, oldPart)) {
-          newFiles = listFilesCreatedByQuery(loadPath, writeId, stmtId);
+        if (newFiles != null) {
+          listFilesCreatedByQuery(loadPath, writeId, stmtId, isMmTableWrite ? isInsertOverwrite : false, newFiles);
         }
         if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
           Utilities.FILE_OP_LOGGER.trace("maybe deleting stuff from " + oldPartPath
@@ -1781,8 +1789,15 @@ public class Hive {
       // or dynamic partition inserts), the add partition event will capture the list of files added.
       // Generate an insert event only if inserting into an existing partition
       // When inserting into a new partition, the add partition event takes care of insert event
-      if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary() && (null != oldPart)) {
-        fireInsertEvent(tbl, partSpec, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
+      if ((null != oldPart) && (null != newFiles)) {
+        if (isTxnTable) {
+          addWriteNotificationLog(tbl, partSpec, newFiles, writeId);
+        } else {
+          fireInsertEvent(tbl, partSpec, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
+        }
+      } else {
+        LOG.debug("No new files were created, and is not a replace, or we're inserting into a "
+                + "partition that does not exist yet. Skipping generating INSERT event.");
       }
 
       // column stats will be inaccurate
@@ -1852,6 +1867,12 @@ public class Hive {
           }
           throw e;
         }
+
+        // For acid table, add the acid_write event with file list at the time of load itself. But
+        // it should be done after partition is created.
+        if (isTxnTable && (null != newFiles)) {
+          addWriteNotificationLog(tbl, partSpec, newFiles, writeId);
+        }
       } else {
         setStatsPropAndAlterPartition(hasFollowingStatsTask, tbl, newTPart);
       }
@@ -1906,50 +1927,47 @@ public class Hive {
   }
 
   private boolean areEventsForDmlNeeded(Table tbl, Partition oldPart) {
-    return conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary() && oldPart != null;
+    // For Acid IUD, add partition is a meta data only operation. So need to add the new files added
+    // information into the TXN_WRITE_NOTIFICATION_LOG table.
+    return conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary() &&
+            ((null != oldPart) || AcidUtils.isTransactionalTable(tbl));
+  }
+
+  private void listFilesInsideAcidDirectory(Path acidDir, FileSystem srcFs, List<Path> newFiles) throws IOException {
+    // list out all the files/directory in the path
+    FileStatus[] acidFiles;
+    acidFiles = srcFs.listStatus(acidDir);
+    if (acidFiles == null) {
+      LOG.debug("No files added by this query in: " + acidDir);
+      return;
+    }
+    for (FileStatus acidFile : acidFiles) {
+      // need to list out only files, ignore folders.
+      if (!acidFile.isDirectory()) {
+        newFiles.add(acidFile.getPath());
+      } else {
+        listFilesInsideAcidDirectory(acidFile.getPath(), srcFs, newFiles);
+      }
+    }
   }
 
-  private List<Path> listFilesCreatedByQuery(Path loadPath, long writeId, int stmtId) throws HiveException {
-    List<Path> newFiles = new ArrayList<Path>();
-    final String filePrefix = AcidUtils.deltaSubdir(writeId, writeId, stmtId);
-    FileStatus[] srcs;
-    FileSystem srcFs;
+  private void listFilesCreatedByQuery(Path loadPath, long writeId, int stmtId,
+                                             boolean isInsertOverwrite, List<Path> newFiles) throws HiveException {
+    Path acidDir = new Path(loadPath, AcidUtils.baseOrDeltaSubdir(isInsertOverwrite, writeId, writeId, stmtId));
     try {
-      srcFs = loadPath.getFileSystem(conf);
-      srcs = srcFs.listStatus(loadPath);
+      FileSystem srcFs = loadPath.getFileSystem(conf);
+      if (srcFs.exists(acidDir) && srcFs.isDirectory(acidDir)){
+        // list out all the files in the path
+        listFilesInsideAcidDirectory(acidDir, srcFs, newFiles);
+      } else {
+        LOG.info("directory does not exist: " + acidDir);
+        return;
+      }
     } catch (IOException e) {
       LOG.error("Error listing files", e);
       throw new HiveException(e);
     }
-    if (srcs == null) {
-      LOG.info("No sources specified: " + loadPath);
-      return newFiles;
-    }
-    PathFilter subdirFilter = null;
-
-    // Note: just like the move path, we only do one level of recursion.
-    for (FileStatus src : srcs) {
-      if (src.isDirectory()) {
-        if (subdirFilter == null) {
-          subdirFilter = new PathFilter() {
-            @Override
-            public boolean accept(Path path) {
-              return path.getName().startsWith(filePrefix);
-            }
-          };
-        }
-        try {
-          for (FileStatus srcFile : srcFs.listStatus(src.getPath(), subdirFilter)) {
-            newFiles.add(srcFile.getPath());
-          }
-        } catch (IOException e) {
-          throw new HiveException(e);
-        }
-      } else if (src.getPath().getName().startsWith(filePrefix)) {
-        newFiles.add(src.getPath());
-      }
-    }
-    return newFiles;
+    return;
   }
 
   private void setStatsPropAndAlterPartition(boolean hasFollowingStatsTask, Table tbl,
@@ -2301,13 +2319,17 @@ private void constructOneLBLocationMap(FileStatus fSta,
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.PerfLogBegin("MoveTask", PerfLogger.LOAD_TABLE);
 
-    List<Path> newFiles = Collections.synchronizedList(new ArrayList<Path>());
+    List<Path> newFiles = null;
     Table tbl = getTable(tableName);
     assert tbl.getPath() != null : "null==getPath() for " + tbl.getTableName();
     boolean isTxnTable = AcidUtils.isTransactionalTable(tbl);
     boolean isMmTable = AcidUtils.isInsertOnlyTable(tbl);
     boolean isFullAcidTable = AcidUtils.isFullAcidTable(tbl);
 
+    if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary()) {
+      newFiles = Collections.synchronizedList(new ArrayList<Path>());
+    }
+
     // Note: this assumes both paths are qualified; which they are, currently.
     if ((isMmTable || isFullAcidTable) && loadPath.equals(tbl.getPath())) {
       /**
@@ -2319,7 +2341,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
         Utilities.FILE_OP_LOGGER.debug(
             "not moving " + loadPath + " to " + tbl.getPath() + " (MM)");
       }
-      newFiles = listFilesCreatedByQuery(loadPath, writeId, stmtId);
+
+      //new files list is required only for event notification.
+      if (newFiles != null) {
+        listFilesCreatedByQuery(loadPath, writeId, stmtId, isMmTable ? isInsertOverwrite : false, newFiles);
+      }
     } else {
       // Either a non-MM query, or a load into MM table from an external source.
       Path tblPath = tbl.getPath();
@@ -2390,10 +2416,10 @@ private void constructOneLBLocationMap(FileStatus fSta,
 
     alterTable(tbl, environmentContext);
 
-    if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML) && !tbl.isTemporary()) {
-      fireInsertEvent(tbl, null, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
+    if (AcidUtils.isTransactionalTable(tbl)) {
+      addWriteNotificationLog(tbl, null, newFiles, writeId);
     } else {
-      fireInsertEvent(tbl, null, (loadFileType == LoadFileType.REPLACE_ALL), null);
+      fireInsertEvent(tbl, null, (loadFileType == LoadFileType.REPLACE_ALL), newFiles);
     }
 
     perfLogger.PerfLogEnd("MoveTask", PerfLogger.LOAD_TABLE);
@@ -2647,6 +2673,48 @@ private void constructOneLBLocationMap(FileStatus fSta,
     tpart.getSd().setLocation(partPath);
   }
 
+  private void addWriteNotificationLog(Table tbl, Map<String, String> partitionSpec,
+                                       List<Path> newFiles, Long writeId) throws HiveException {
+    if (!conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML)) {
+      LOG.debug("write notification log is ignored as dml event logging is disabled");
+      return;
+    }
+
+    if (tbl.isTemporary()) {
+      LOG.debug("write notification log is ignored as " + tbl.getTableName() + " is temporary : " + writeId);
+      return;
+    }
+
+    if (newFiles == null || newFiles.isEmpty()) {
+      LOG.debug("write notification log is ignored as file list is empty");
+      return;
+    }
+
+    LOG.debug("adding write notification log for operation " + writeId + " table " + tbl.getCompleteName() +
+                        "partition " + partitionSpec + " list of files " + newFiles);
+
+    try {
+      FileSystem fileSystem = tbl.getDataLocation().getFileSystem(conf);
+      Long txnId = SessionState.get().getTxnMgr().getCurrentTxnId();
+
+      InsertEventRequestData insertData = new InsertEventRequestData();
+      insertData.setReplace(true);
+
+      WriteNotificationLogRequest rqst = new WriteNotificationLogRequest(txnId, writeId,
+              tbl.getDbName(), tbl.getTableName(), insertData);
+      addInsertFileInformation(newFiles, fileSystem, insertData);
+
+      if (partitionSpec != null && !partitionSpec.isEmpty()) {
+        for (FieldSchema fs : tbl.getPartitionKeys()) {
+          rqst.addToPartitionVals(partitionSpec.get(fs.getName()));
+        }
+      }
+      getSynchronizedMSC().addWriteNotificationLog(rqst);
+    } catch (IOException | TException e) {
+      throw new HiveException(e);
+    }
+  }
+
   private void fireInsertEvent(Table tbl, Map<String, String> partitionSpec, boolean replace, List<Path> newFiles)
       throws HiveException {
     if (conf.getBoolVar(ConfVars.FIRE_EVENTS_FOR_DML)) {
@@ -2723,6 +2791,7 @@ private void constructOneLBLocationMap(FileStatus fSta,
       InsertEventRequestData insertData) throws IOException {
     insertData.addToFilesAdded(p.toString());
     FileChecksum cksum = fileSystem.getFileChecksum(p);
+    String acidDirPath = AcidUtils.getFirstLevelAcidDirPath(p.getParent(), fileSystem);
     // File checksum is not implemented for local filesystem (RawLocalFileSystem)
     if (cksum != null) {
       String checksumString =
@@ -2732,6 +2801,11 @@ private void constructOneLBLocationMap(FileStatus fSta,
       // Add an empty checksum string for filesystems that don't generate one
       insertData.addToFilesAddedChecksum("");
     }
+
+    // acid dir will be present only for acid write operations.
+    if (acidDirPath != null) {
+      insertData.addToSubDirectoryList(acidDirPath);
+    }
   }
 
   public boolean dropPartition(String tblName, List<String> part_vals, boolean deleteData)
@@ -3690,7 +3764,6 @@ private void constructOneLBLocationMap(FileStatus fSta,
                   @Override
                   public Void call() throws HiveException {
                     SessionState.setCurrentSessionState(parentSession);
-                    final String group = srcStatus.getGroup();
                     try {
                       boolean success = false;
                       if (destFs instanceof DistributedFileSystem) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
index f1c4d98..e04a0f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.metadata;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -431,11 +432,19 @@ public final class HiveUtils {
 
   public static String getReplPolicy(String dbName, String tableName) {
     if ((dbName == null) || (dbName.isEmpty())) {
-      return null;
+      return "*.*";
     } else if ((tableName == null) || (tableName.isEmpty())) {
       return dbName.toLowerCase() + ".*";
     } else {
       return dbName.toLowerCase() + "." + tableName.toLowerCase();
     }
   }
+
+  public static Path getDumpPath(Path root, String dbName, String tableName) {
+    assert (dbName != null);
+    if ((tableName != null) && (!tableName.isEmpty())) {
+      return new Path(root, dbName + "." + tableName);
+    }
+    return new Path(root, dbName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
index d34de61..eb594f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ImportSemanticAnalyzer.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hive.ql.plan.ImportTableDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.DropTableDesc;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
+import org.apache.hadoop.hive.ql.plan.LoadMultiFilesDesc;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc.LoadFileType;
 import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -249,9 +250,11 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new HiveException(e);
     }
 
+    boolean inReplicationScope = false;
     if ((replicationSpec != null) && replicationSpec.isInReplicationScope()){
       tblDesc.setReplicationSpec(replicationSpec);
       StatsSetupConst.setBasicStatsState(tblDesc.getTblProps(), StatsSetupConst.FALSE);
+      inReplicationScope = true;
     }
 
     if (isExternalSet) {
@@ -275,7 +278,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     for (Partition partition : partitions) {
       // TODO: this should ideally not create AddPartitionDesc per partition
       AddPartitionDesc partsDesc = getBaseAddPartitionDescFromPartition(fromPath, dbname, tblDesc, partition);
-      if ((replicationSpec != null) && replicationSpec.isInReplicationScope()){
+      if (inReplicationScope){
         StatsSetupConst.setBasicStatsState(partsDesc.getPartition(0).getPartParams(), StatsSetupConst.FALSE);
       }
       partitionDescs.add(partsDesc);
@@ -335,13 +338,14 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       //if importing into existing transactional table or will create a new transactional table
       //(because Export was done from transactional table), need a writeId
       // Explain plan doesn't open a txn and hence no need to allocate write id.
-      if (x.getCtx().getExplainConfig() == null) {
+      // In replication flow, no need to allocate write id. It will be allocated using the alloc write id event.
+      if (x.getCtx().getExplainConfig() == null && !inReplicationScope) {
         writeId = txnMgr.getTableWriteId(tblDesc.getDatabaseName(), tblDesc.getTableName());
         stmtId = txnMgr.getStmtIdAndIncrement();
       }
     }
 
-    if (!replicationSpec.isInReplicationScope()) {
+    if (!inReplicationScope) {
       createRegularImportTasks(
           tblDesc, partitionDescs,
           isPartSpecSet, replicationSpec, table,
@@ -390,7 +394,7 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
     Path dataPath = new Path(fromURI.toString(), EximUtil.DATA_PATH_NAME);
     Path destPath = null, loadPath = null;
     LoadFileType lft;
-    if (AcidUtils.isTransactionalTable(table)) {
+    if (AcidUtils.isTransactionalTable(table) && !replicationSpec.isInReplicationScope()) {
       String mmSubdir = replace ? AcidUtils.baseDir(writeId)
           : AcidUtils.deltaSubdir(writeId, writeId, stmtId);
       destPath = new Path(tgtPath, mmSubdir);
@@ -428,13 +432,26 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       copyTask = TaskFactory.get(new CopyWork(dataPath, destPath, false));
     }
 
-    LoadTableDesc loadTableWork = new LoadTableDesc(
-        loadPath, Utilities.getTableDesc(table), new TreeMap<>(), lft, writeId);
-    loadTableWork.setStmtId(stmtId);
+    MoveWork moveWork = new MoveWork(x.getInputs(), x.getOutputs(), null, null, false);
+
+
+    if (replicationSpec.isInReplicationScope() && AcidUtils.isTransactionalTable(table)) {
+      LoadMultiFilesDesc loadFilesWork = new LoadMultiFilesDesc(
+              Collections.singletonList(destPath),
+              Collections.singletonList(tgtPath),
+              true, null, null);
+      moveWork.setMultiFilesDesc(loadFilesWork);
+      moveWork.setNeedCleanTarget(false);
+    } else {
+      LoadTableDesc loadTableWork = new LoadTableDesc(
+              loadPath, Utilities.getTableDesc(table), new TreeMap<>(), lft, writeId);
+      loadTableWork.setStmtId(stmtId);
+      moveWork.setLoadTableWork(loadTableWork);
+    }
+
     //if Importing into existing table, FileFormat is checked by
     // ImportSemanticAnalzyer.checked checkTable()
-    MoveWork mv = new MoveWork(x.getInputs(), x.getOutputs(), loadTableWork, null, false);
-    Task<?> loadTableTask = TaskFactory.get(mv, x.getConf());
+    Task<?> loadTableTask = TaskFactory.get(moveWork, x.getConf());
     copyTask.addDependentTask(loadTableTask);
     x.getTasks().add(copyTask);
     return loadTableTask;
@@ -498,8 +515,10 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
           + partSpecToString(partSpec.getPartSpec())
           + " with source location: " + srcLocation);
       Path tgtLocation = new Path(partSpec.getLocation());
-      Path destPath = !AcidUtils.isTransactionalTable(table.getParameters()) ?
-          x.getCtx().getExternalTmpPath(tgtLocation)
+      //Replication scope the write id will be invalid
+      Boolean useStagingDirectory = !AcidUtils.isTransactionalTable(table.getParameters()) ||
+              replicationSpec.isInReplicationScope();
+      Path destPath =  useStagingDirectory ? x.getCtx().getExternalTmpPath(tgtLocation)
           : new Path(tgtLocation, AcidUtils.deltaSubdir(writeId, writeId, stmtId));
       Path moveTaskSrc =  !AcidUtils.isTransactionalTable(table.getParameters()) ? destPath : tgtLocation;
       if (Utilities.FILE_OP_LOGGER.isTraceEnabled()) {
@@ -523,17 +542,29 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
       Task<?> addPartTask = TaskFactory.get(
               new DDLWork(x.getInputs(), x.getOutputs(), addPartitionDesc), x.getConf());
 
+      MoveWork moveWork = new MoveWork(x.getInputs(), x.getOutputs(),
+              null, null, false);
+
       // Note: this sets LoadFileType incorrectly for ACID; is that relevant for import?
       //       See setLoadFileType and setIsAcidIow calls elsewhere for an example.
-      LoadTableDesc loadTableWork = new LoadTableDesc(moveTaskSrc, Utilities.getTableDesc(table),
-          partSpec.getPartSpec(),
-          replicationSpec.isReplace() ? LoadFileType.REPLACE_ALL : LoadFileType.OVERWRITE_EXISTING,
-              writeId);
-      loadTableWork.setStmtId(stmtId);
-      loadTableWork.setInheritTableSpecs(false);
-      Task<?> loadPartTask = TaskFactory.get(
-              new MoveWork(x.getInputs(), x.getOutputs(), loadTableWork, null, false),
-              x.getConf());
+      if (replicationSpec.isInReplicationScope() && AcidUtils.isTransactionalTable(tblDesc.getTblProps())) {
+        LoadMultiFilesDesc loadFilesWork = new LoadMultiFilesDesc(
+                Collections.singletonList(destPath),
+                Collections.singletonList(tgtLocation),
+                true, null, null);
+        moveWork.setMultiFilesDesc(loadFilesWork);
+        moveWork.setNeedCleanTarget(false);
+      } else {
+        LoadTableDesc loadTableWork = new LoadTableDesc(moveTaskSrc, Utilities.getTableDesc(table),
+                partSpec.getPartSpec(),
+                replicationSpec.isReplace() ? LoadFileType.REPLACE_ALL : LoadFileType.OVERWRITE_EXISTING,
+                writeId);
+        loadTableWork.setStmtId(stmtId);
+        loadTableWork.setInheritTableSpecs(false);
+        moveWork.setLoadTableWork(loadTableWork);
+      }
+
+      Task<?> loadPartTask = TaskFactory.get(moveWork, x.getConf());
       copyTask.addDependentTask(loadPartTask);
       addPartTask.addDependentTask(loadPartTask);
       x.getTasks().add(copyTask);
@@ -1005,7 +1036,8 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
             t.addDependentTask(
                 addSinglePartition(fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, x, writeId, stmtId));
             if (updatedMetadata != null) {
-              updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+              updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                      addPartitionDesc.getPartition(0).getPartSpec());
             }
           }
         } else {
@@ -1057,13 +1089,15 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
               x.getTasks().add(addSinglePartition(
                   fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, x, writeId, stmtId));
               if (updatedMetadata != null) {
-                updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+                updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                        addPartitionDesc.getPartition(0).getPartSpec());
               }
             } else {
               x.getTasks().add(alterSinglePartition(
                       fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, null, x));
               if (updatedMetadata != null) {
-                updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+                updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                        addPartitionDesc.getPartition(0).getPartSpec());
               }
             }
           } else {
@@ -1078,7 +1112,8 @@ public class ImportSemanticAnalyzer extends BaseSemanticAnalyzer {
                     fromURI, fs, tblDesc, table, wh, addPartitionDesc, replicationSpec, ptn, x));
               }
               if (updatedMetadata != null) {
-                updatedMetadata.addPartition(addPartitionDesc.getPartition(0).getPartSpec());
+                updatedMetadata.addPartition(table.getDbName(), table.getTableName(),
+                        addPartitionDesc.getPartition(0).getPartSpec());
               }
               if (lockType == WriteEntity.WriteType.DDL_NO_LOCK){
                 lockType = WriteEntity.WriteType.DDL_SHARED;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/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 576f337..1271799 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
@@ -7309,7 +7309,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         }
         try {
           if (ctx.getExplainConfig() != null) {
-            writeId = 0L; // For explain plan, txn won't be opened and doesn't make sense to allocate write id
+            writeId = null; // For explain plan, txn won't be opened and doesn't make sense to allocate write id
           } else {
             if (isMmTable) {
               writeId = txnMgr.getTableWriteId(dest_tab.getDbName(), dest_tab.getTableName());
@@ -7324,6 +7324,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         boolean isReplace = !qb.getParseInfo().isInsertIntoTable(
             dest_tab.getDbName(), dest_tab.getTableName());
         ltd = new LoadTableDesc(queryTmpdir, table_desc, dpCtx, acidOp, isReplace, writeId);
+        if (writeId != null) {
+          ltd.setStmtId(txnMgr.getCurrentStmtId());
+        }
         // For Acid table, Insert Overwrite shouldn't replace the table content. We keep the old
         // deltas and base and leave them up to the cleaner to clean up
         boolean isInsertInto = qb.getParseInfo().isInsertIntoTable(
@@ -7419,6 +7422,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         throw new SemanticException("Failed to allocate write Id", ex);
       }
       ltd = new LoadTableDesc(queryTmpdir, table_desc, dest_part.getSpec(), acidOp, writeId);
+      if (writeId != null) {
+        ltd.setStmtId(txnMgr.getCurrentStmtId());
+      }
       // For the current context for generating File Sink Operator, it is either INSERT INTO or INSERT OVERWRITE.
       // So the next line works.
       boolean isInsertInto = !qb.getParseInfo().isDestToOpTypeInsertOverwrite(dest);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index ce7e65a..8df2904 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -179,9 +179,23 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     String newTableName = getTmptTableNameForExport(exportTable); //this is db.table
     Map<String, String> tblProps = new HashMap<>();
     tblProps.put(hive_metastoreConstants.TABLE_IS_TRANSACTIONAL, Boolean.FALSE.toString());
+    String location;
+
+    // for temporary tables we set the location to something in the session's scratch dir
+    // it has the same life cycle as the tmp table
+    try {
+      // Generate a unique ID for temp table path.
+      // This path will be fixed for the life of the temp table.
+      Path path = new Path(SessionState.getTempTableSpace(conf), UUID.randomUUID().toString());
+      path = Warehouse.getDnsPath(path, conf);
+      location = path.toString();
+    } catch (MetaException err) {
+      throw new SemanticException("Error while generating temp table path:", err);
+    }
+
     CreateTableLikeDesc ctlt = new CreateTableLikeDesc(newTableName,
         false, true, null,
-        null, null, null, null,
+        null, location, null, null,
         tblProps,
         true, //important so we get an exception on name collision
         Warehouse.getQualifiedName(exportTable.getTTable()), false);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
index 61bf6b9..75dcaa3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/CopyUtils.java
@@ -400,7 +400,7 @@ public class CopyUtils {
     return result;
   }
 
-  private Path getCopyDestination(ReplChangeManager.FileInfo fileInfo, Path destRoot) {
+  public static Path getCopyDestination(ReplChangeManager.FileInfo fileInfo, Path destRoot) {
     if (fileInfo.getSubDir() == null) {
       return destRoot;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
index c0701c5..62d699f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
@@ -186,10 +186,6 @@ public class Utils {
         return false;
       }
 
-      boolean isAcidTable = AcidUtils.isTransactionalTable(tableHandle);
-      if (isAcidTable) {
-        return hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_INCLUDE_ACID_TABLES);
-      }
       return !tableHandle.isTemporary();
     }
     return true;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
index db97d7c..f04cd93 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/CommitTxnHandler.java
@@ -18,9 +18,27 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.dump.events;
 
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStore;
+import org.apache.hadoop.hive.metastore.ReplChangeManager;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
+import org.apache.hadoop.hive.metastore.utils.StringUtils;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.parse.EximUtil;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.repl.DumpType;
 import org.apache.hadoop.hive.ql.parse.repl.load.DumpMetaData;
+import org.apache.hadoop.fs.FileSystem;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.List;
 
 class CommitTxnHandler extends AbstractEventHandler {
 
@@ -28,11 +46,116 @@ class CommitTxnHandler extends AbstractEventHandler {
     super(event);
   }
 
+  private BufferedWriter writer(Context withinContext, Path dataPath) throws IOException {
+    Path filesPath = new Path(dataPath, EximUtil.FILES_NAME);
+    FileSystem fs = dataPath.getFileSystem(withinContext.hiveConf);
+    return new BufferedWriter(new OutputStreamWriter(fs.create(filesPath)));
+  }
+
+  private void writeDumpFiles(Context withinContext, Iterable<String> files, Path dataPath) throws IOException {
+    // encoded filename/checksum of files, write into _files
+    try (BufferedWriter fileListWriter = writer(withinContext, dataPath)) {
+      for (String file : files) {
+        fileListWriter.write(file + "\n");
+      }
+    }
+  }
+
+  private void createDumpFile(Context withinContext, org.apache.hadoop.hive.ql.metadata.Table qlMdTable,
+                  List<Partition> qlPtns, List<List<String>> fileListArray) throws IOException, SemanticException {
+    if (fileListArray == null || fileListArray.isEmpty()) {
+      return;
+    }
+
+    Path metaDataPath = new Path(withinContext.eventRoot, EximUtil.METADATA_NAME);
+    withinContext.replicationSpec.setIsReplace(true);
+    EximUtil.createExportDump(metaDataPath.getFileSystem(withinContext.hiveConf), metaDataPath,
+            qlMdTable, qlPtns,
+            withinContext.replicationSpec,
+            withinContext.hiveConf);
+
+    if ((null == qlPtns) || qlPtns.isEmpty()) {
+      Path dataPath = new Path(withinContext.eventRoot, EximUtil.DATA_PATH_NAME);
+      writeDumpFiles(withinContext, fileListArray.get(0), dataPath);
+    } else {
+      for (int idx = 0; idx < qlPtns.size(); idx++) {
+        Path dataPath = new Path(withinContext.eventRoot, qlPtns.get(idx).getName());
+        writeDumpFiles(withinContext, fileListArray.get(idx), dataPath);
+      }
+    }
+  }
+
+  private void createDumpFileForTable(Context withinContext, org.apache.hadoop.hive.ql.metadata.Table qlMdTable,
+                    List<Partition> qlPtns, List<List<String>> fileListArray) throws IOException, SemanticException {
+    Path newPath = HiveUtils.getDumpPath(withinContext.eventRoot, qlMdTable.getDbName(), qlMdTable.getTableName());
+    Context context = new Context(withinContext);
+    context.setEventRoot(newPath);
+    createDumpFile(context, qlMdTable, qlPtns, fileListArray);
+  }
+
   @Override
   public void handle(Context withinContext) throws Exception {
     LOG.info("Processing#{} COMMIT_TXN message : {}", fromEventId(), event.getMessage());
+    String payload = event.getMessage();
+
+    if (!withinContext.hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_METADATA_ONLY)) {
+      CommitTxnMessage commitTxnMessage = deserializer.getCommitTxnMessage(event.getMessage());
+
+      String contextDbName =  withinContext.dbName == null ? null :
+              StringUtils.normalizeIdentifier(withinContext.dbName);
+      String contextTableName =  withinContext.tableName == null ? null :
+              StringUtils.normalizeIdentifier(withinContext.tableName);
+      List<WriteEventInfo> writeEventInfoList = HiveMetaStore.HMSHandler.getMSForConf(withinContext.hiveConf).
+              getAllWriteEventInfo(commitTxnMessage.getTxnId(), contextDbName, contextTableName);
+      int numEntry = (writeEventInfoList != null ? writeEventInfoList.size() : 0);
+      if (numEntry != 0) {
+        commitTxnMessage.addWriteEventInfo(writeEventInfoList);
+        payload = commitTxnMessage.toString();
+        LOG.debug("payload for commit txn event : " + payload);
+      }
+
+      org.apache.hadoop.hive.ql.metadata.Table qlMdTablePrev = null;
+      org.apache.hadoop.hive.ql.metadata.Table qlMdTable = null;
+      List<Partition> qlPtns = new ArrayList<>();
+      List<List<String>> filesTobeAdded = new ArrayList<>();
+
+      // The below loop creates dump directory for each table. It reads through the list of write notification events,
+      // groups the entries per table and creates the lists of files to be replicated. The event directory in the dump
+      // path will have subdirectory for each table. This folder will have metadata for the table and the list of files
+      // to be replicated. The entries are added in the table with txn id, db name,table name, partition name
+      // combination as primary key, so the entries with same table will come together. Only basic table metadata is
+      // used during import, so we need not dump the latest table metadata.
+      for (int idx = 0; idx < numEntry; idx++) {
+        qlMdTable = new org.apache.hadoop.hive.ql.metadata.Table(commitTxnMessage.getTableObj(idx));
+        if (qlMdTablePrev == null) {
+          qlMdTablePrev = qlMdTable;
+        }
+
+        // one dump directory per table
+        if (!qlMdTablePrev.getCompleteName().equals(qlMdTable.getCompleteName())) {
+          createDumpFileForTable(withinContext, qlMdTablePrev, qlPtns, filesTobeAdded);
+          qlPtns = new ArrayList<>();
+          filesTobeAdded = new ArrayList<>();
+          qlMdTablePrev = qlMdTable;
+        }
+
+        if (qlMdTable.isPartitioned() && (null != commitTxnMessage.getPartitionObj(idx))) {
+          qlPtns.add(new org.apache.hadoop.hive.ql.metadata.Partition(qlMdTable,
+                  commitTxnMessage.getPartitionObj(idx)));
+        }
+
+        filesTobeAdded.add(Lists.newArrayList(
+                ReplChangeManager.getListFromSeparatedString(commitTxnMessage.getFiles(idx))));
+      }
+
+      //Dump last table in the list
+      if (qlMdTablePrev != null) {
+        createDumpFileForTable(withinContext, qlMdTablePrev, qlPtns, filesTobeAdded);
+      }
+    }
+
     DumpMetaData dmd = withinContext.createDmd(this);
-    dmd.setPayload(event.getMessage());
+    dmd.setPayload(payload);
     dmd.write();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
index c0fa7b2..ec35f4e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/EventHandler.java
@@ -35,18 +35,37 @@ public interface EventHandler {
   DumpType dumpType();
 
   class Context {
-    final Path eventRoot, cmRoot;
+    Path eventRoot;
+    final Path  cmRoot;
     final Hive db;
     final HiveConf hiveConf;
     final ReplicationSpec replicationSpec;
+    final String dbName;
+    final String tableName;
 
     public Context(Path eventRoot, Path cmRoot, Hive db, HiveConf hiveConf,
-        ReplicationSpec replicationSpec) {
+        ReplicationSpec replicationSpec, String dbName, String tableName) {
       this.eventRoot = eventRoot;
       this.cmRoot = cmRoot;
       this.db = db;
       this.hiveConf = hiveConf;
       this.replicationSpec = replicationSpec;
+      this.dbName = dbName;
+      this.tableName = tableName;
+    }
+
+    public Context(Context other) {
+      this.eventRoot = other.eventRoot;
+      this.cmRoot = other.cmRoot;
+      this.db = other.db;
+      this.hiveConf = other.hiveConf;
+      this.replicationSpec = other.replicationSpec;
+      this.dbName = other.dbName;
+      this.tableName = other.tableName;
+    }
+
+    public void setEventRoot(Path eventRoot) {
+      this.eventRoot = eventRoot;
     }
 
     DumpMetaData createDmd(EventHandler eventHandler) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
index 5ac3af0..cf3822a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/events/InsertHandler.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.parse.EximUtil;
 import org.apache.hadoop.hive.ql.parse.repl.DumpType;
@@ -53,6 +54,9 @@ class InsertHandler extends AbstractEventHandler {
       return;
     }
 
+    // In case of ACID tables, insert event should not have fired.
+    assert(!AcidUtils.isTransactionalTable(qlMdTable));
+
     List<Partition> qlPtns = null;
     if (qlMdTable.isPartitioned() && (null != insertMsg.getPtnObj())) {
       qlPtns = Collections.singletonList(partitionObject(qlMdTable, insertMsg));

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
index d76f419..614e071 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/UpdatedMetaDataTracker.java
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.load;
 
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hive.common.util.HiveStringUtils;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.List;
 
@@ -25,52 +28,113 @@ import java.util.List;
  * Utility class to help track and return the metadata which are updated by repl load
  */
 public class UpdatedMetaDataTracker {
-  private String replState;
-  private String dbName;
-  private String tableName;
-  private List<Map <String, String>> partitionsList;
 
-  public UpdatedMetaDataTracker() {
-    this.replState = null;
-    this.dbName = null;
-    this.tableName = null;
-    this.partitionsList = new ArrayList<>();
+  /**
+   * Utility class to store replication state of a table.
+   */
+  public static class UpdateMetaData {
+    private String replState;
+    private String dbName;
+    private String tableName;
+    private List<Map <String, String>> partitionsList;
+
+    UpdateMetaData(String replState, String dbName, String tableName, Map <String, String> partSpec) {
+      this.replState = replState;
+      this.dbName = dbName;
+      this.tableName = tableName;
+      this.partitionsList = new ArrayList<>();
+      if (partSpec != null) {
+        this.partitionsList.add(partSpec);
+      }
+    }
+
+    public String getReplState() {
+      return replState;
+    }
+
+    public String getDbName() {
+      return dbName;
+    }
+
+    public String getTableName() {
+      return tableName;
+    }
+
+    public List<Map <String, String>> getPartitionsList() {
+      return partitionsList;
+    }
+
+    public void addPartition(Map<String, String> partSpec) {
+      this.partitionsList.add(partSpec);
+    }
   }
 
-  public void copyUpdatedMetadata(UpdatedMetaDataTracker other) {
-    this.replState = other.replState;
-    this.dbName = other.dbName;
-    this.tableName = other.tableName;
-    this.partitionsList = other.getPartitions();
+  private List<UpdateMetaData> updateMetaDataList;
+  private Map<String, Integer> updateMetaDataMap;
+
+  public UpdatedMetaDataTracker() {
+    updateMetaDataList = new ArrayList<>();
+    updateMetaDataMap = new HashMap<>();
   }
 
-  public void set(String replState, String dbName, String tableName, Map <String, String> partSpec) {
-    this.replState = replState;
-    this.dbName = dbName;
-    this.tableName = tableName;
-    if (partSpec != null) {
-      addPartition(partSpec);
+  public void copyUpdatedMetadata(UpdatedMetaDataTracker other) {
+    int size = updateMetaDataList.size();
+    for (UpdateMetaData updateMetaDataOther : other.updateMetaDataList) {
+      String key = getKey(normalizeIdentifier(updateMetaDataOther.getDbName()),
+              normalizeIdentifier(updateMetaDataOther.getTableName()));
+      Integer idx = updateMetaDataMap.get(key);
+      if (idx == null) {
+        updateMetaDataList.add(updateMetaDataOther);
+        updateMetaDataMap.put(key, size++);
+      } else if (updateMetaDataOther.partitionsList != null && updateMetaDataOther.partitionsList.size() != 0) {
+        UpdateMetaData updateMetaData = updateMetaDataList.get(idx);
+        for (Map<String, String> partSpec : updateMetaDataOther.partitionsList) {
+          updateMetaData.addPartition(partSpec);
+        }
+      }
     }
   }
 
-  public void addPartition(Map <String, String> partSpec) {
-    partitionsList.add(partSpec);
+  public void set(String replState, String dbName, String tableName, Map <String, String> partSpec)
+          throws SemanticException {
+    if (dbName == null) {
+      throw new SemanticException("db name can not be null");
+    }
+    String key = getKey(normalizeIdentifier(dbName), normalizeIdentifier(tableName));
+    Integer idx = updateMetaDataMap.get(key);
+    if (idx == null) {
+      updateMetaDataList.add(new UpdateMetaData(replState, dbName, tableName, partSpec));
+      updateMetaDataMap.put(key, updateMetaDataList.size() - 1);
+    } else {
+      updateMetaDataList.get(idx).addPartition(partSpec);
+    }
   }
 
-  public String getReplicationState() {
-    return replState;
+  public void addPartition(String dbName, String tableName, Map <String, String> partSpec) throws SemanticException {
+    if (dbName == null) {
+      throw new SemanticException("db name can not be null");
+    }
+    String key = getKey(normalizeIdentifier(dbName), normalizeIdentifier(tableName));
+    Integer idx = updateMetaDataMap.get(key);
+    if (idx == null) {
+      throw new SemanticException("add partition to metadata map failed as list is not yet set for table : " + key);
+    }
+    updateMetaDataList.get(idx).addPartition(partSpec);
   }
 
-  public String getDatabase() {
-    return dbName;
+  public List<UpdateMetaData> getUpdateMetaDataList() {
+    return updateMetaDataList;
   }
 
-  public String getTable() {
-    return tableName;
+  private String getKey(String dbName, String tableName) {
+    if (tableName == null) {
+      return dbName + ".*";
+    }
+    return dbName + "." + tableName;
   }
 
-  public List<Map <String, String>> getPartitions() {
-    return partitionsList;
+  private String normalizeIdentifier(String name) {
+    return name == null ? null : HiveStringUtils.normalizeIdentifier(name);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
index afc7426..d3f3306 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AbortTxnHandler.java
@@ -48,7 +48,12 @@ public class AbortTxnHandler extends AbstractMessageHandler {
                 msg.getTxnId(), ReplTxnWork.OperationType.REPL_ABORT_TXN, context.eventOnlyReplicationSpec()),
         context.hiveConf
     );
-    updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+
+    // For warehouse level dump, don't update the metadata of database as we don't know this txn is for which database.
+    // Anyways, if this event gets executed again, it is taken care of.
+    if (!context.isDbNameEmpty()) {
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    }
     context.log.debug("Added Abort txn task : {}", abortTxnTask.getId());
     return Collections.singletonList(abortTxnTask);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
index 9bdbf64..63f2577 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/AllocWriteIdHandler.java
@@ -52,7 +52,7 @@ public class AllocWriteIdHandler extends AbstractMessageHandler {
             .getTableName());
 
     // Repl policy should be created based on the table name in context.
-    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(dbName, context.tableName), dbName, tableName,
+    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), dbName, tableName,
         ReplTxnWork.OperationType.REPL_ALLOC_WRITE_ID, msg.getTxnToWriteIdList(), context.eventOnlyReplicationSpec());
 
     Task<? extends Serializable> allocWriteIdTask = TaskFactory.get(work, context.hiveConf);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
index d25102e..0619bd3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/CommitTxnHandler.java
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.hive.ql.parse.repl.load.message;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
+import org.apache.hadoop.hive.ql.exec.repl.util.AddDependencyToLeaves;
+import org.apache.hadoop.hive.ql.exec.util.DAGTraversal;
+import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.ReplTxnWork;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -25,7 +30,7 @@ import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import java.io.Serializable;
-import java.util.Collections;
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -35,20 +40,75 @@ import java.util.List;
 public class CommitTxnHandler extends AbstractMessageHandler {
   @Override
   public List<Task<? extends Serializable>> handle(Context context)
-      throws SemanticException {
+          throws SemanticException {
     if (!AcidUtils.isAcidEnabled(context.hiveConf)) {
       context.log.error("Cannot load transaction events as acid is not enabled");
       throw new SemanticException("Cannot load transaction events as acid is not enabled");
     }
 
     CommitTxnMessage msg = deserializer.getCommitTxnMessage(context.dmd.getPayload());
-    Task<ReplTxnWork> commitTxnTask = TaskFactory.get(
-        new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName, context.tableName,
-              msg.getTxnId(), ReplTxnWork.OperationType.REPL_COMMIT_TXN, context.eventOnlyReplicationSpec()),
-        context.hiveConf
-    );
-    updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    int numEntry = (msg.getTables() == null ? 0 : msg.getTables().size());
+    List<Task<? extends Serializable>> tasks = new ArrayList<>();
+    String dbName = context.dbName;
+    String tableNamePrev = null;
+    String tblName = context.tableName;
+
+    ReplTxnWork work = new ReplTxnWork(HiveUtils.getReplPolicy(context.dbName, context.tableName), context.dbName,
+      context.tableName, msg.getTxnId(), ReplTxnWork.OperationType.REPL_COMMIT_TXN, context.eventOnlyReplicationSpec());
+
+    if (numEntry > 0) {
+      context.log.debug("Commit txn handler for txnid " + msg.getTxnId() + " databases : " + msg.getDatabases() +
+              " tables : " + msg.getTables() + " partitions : " + msg.getPartitions() + " files : " +
+              msg.getFilesList() + " write ids : " + msg.getWriteIds());
+    }
+
+    for (int idx = 0; idx < numEntry; idx++) {
+      String actualTblName = msg.getTables().get(idx);
+      String actualDBName = msg.getDatabases().get(idx);
+      String completeName = Table.getCompleteName(actualDBName, actualTblName);
+
+      // One import task per table. Events for same table are kept together in one dump directory during dump and are
+      // grouped together in commit txn message.
+      if (tableNamePrev == null || !(completeName.equals(tableNamePrev))) {
+        // The data location is created by source, so the location should be formed based on the table name in msg.
+        Path location = HiveUtils.getDumpPath(new Path(context.location), actualDBName, actualTblName);
+        tblName = context.isTableNameEmpty() ? actualTblName : context.tableName;
+        // for warehouse level dump, use db name from write event
+        dbName = (context.isDbNameEmpty() ? actualDBName : context.dbName);
+        Context currentContext = new Context(context, dbName, tblName);
+        currentContext.setLocation(location.toUri().toString());
+
+        // Piggybacking in Import logic for now
+        TableHandler tableHandler = new TableHandler();
+        tasks.addAll((tableHandler.handle(currentContext)));
+        readEntitySet.addAll(tableHandler.readEntities());
+        writeEntitySet.addAll(tableHandler.writeEntities());
+        getUpdatedMetadata().copyUpdatedMetadata(tableHandler.getUpdatedMetadata());
+        tableNamePrev = completeName;
+      }
+
+      try {
+        WriteEventInfo writeEventInfo = new WriteEventInfo(msg.getWriteIds().get(idx),
+                dbName, tblName, msg.getFiles(idx));
+        if (msg.getPartitions().get(idx) != null && !msg.getPartitions().get(idx).isEmpty()) {
+          writeEventInfo.setPartition(msg.getPartitions().get(idx));
+        }
+        work.addWriteEventInfo(writeEventInfo);
+      } catch (Exception e) {
+        throw new SemanticException("Failed to extract write event info from commit txn message : " + e.getMessage());
+      }
+    }
+
+    Task<ReplTxnWork> commitTxnTask = TaskFactory.get(work, context.hiveConf);
+
+    // For warehouse level dump, don't update the metadata of database as we don't know this txn is for which database.
+    // Anyways, if this event gets executed again, it is taken care of.
+    if (!context.isDbNameEmpty()) {
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    }
     context.log.debug("Added Commit txn task : {}", commitTxnTask.getId());
-    return Collections.singletonList(commitTxnTask);
+    DAGTraversal.traverse(tasks, new AddDependencyToLeaves(commitTxnTask));
+    return tasks;
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
index ef4a901..cdf51dd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/MessageHandler.java
@@ -46,8 +46,8 @@ public interface MessageHandler {
   UpdatedMetaDataTracker getUpdatedMetadata();
 
   class Context {
-    public String dbName;
-    public final String tableName, location;
+    public String location;
+    public final String tableName, dbName;
     public final Task<? extends Serializable> precursor;
     public DumpMetaData dmd;
     final HiveConf hiveConf;
@@ -101,5 +101,9 @@ public interface MessageHandler {
     public HiveTxnManager getTxnMgr() {
       return nestedContext.getHiveTxnManager();
     }
+
+    public void setLocation(String location) {
+      this.location = location;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
index 190e021..5dcc44e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/load/message/OpenTxnHandler.java
@@ -47,7 +47,12 @@ public class OpenTxnHandler extends AbstractMessageHandler {
                 msg.getTxnIds(), ReplTxnWork.OperationType.REPL_OPEN_TXN, context.eventOnlyReplicationSpec()),
         context.hiveConf
     );
-    updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+
+    // For warehouse level dump, don't update the metadata of database as we don't know this txn is for which database.
+    // Anyways, if this event gets executed again, it is taken care of.
+    if (!context.isDbNameEmpty()) {
+      updatedMetadata.set(context.dmd.getEventTo().toString(), context.dbName, context.tableName, null);
+    }
     context.log.debug("Added Open txn task : {}", openTxnTask.getId());
     return Collections.singletonList(openTxnTask);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
index 9a1e3a1..47a56d5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MoveWork.java
@@ -40,6 +40,7 @@ public class MoveWork implements Serializable {
   private LoadMultiFilesDesc loadMultiFilesWork;
   private boolean checkFileFormat;
   private boolean srcLocal;
+  private boolean needCleanTarget;
 
   /**
    * ReadEntitites that are passed to the hooks.
@@ -63,6 +64,7 @@ public class MoveWork implements Serializable {
   private MoveWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) {
     this.inputs = inputs;
     this.outputs = outputs;
+    this.needCleanTarget = true;
   }
 
   public MoveWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
@@ -93,6 +95,7 @@ public class MoveWork implements Serializable {
     srcLocal = o.isSrcLocal();
     inputs = o.getInputs();
     outputs = o.getOutputs();
+    needCleanTarget = o.needCleanTarget;
   }
 
   @Explain(displayName = "tables", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
@@ -153,5 +156,12 @@ public class MoveWork implements Serializable {
   public void setSrcLocal(boolean srcLocal) {
     this.srcLocal = srcLocal;
   }
-  
+
+  public boolean isNeedCleanTarget() {
+    return needCleanTarget;
+  }
+
+  public void setNeedCleanTarget(boolean needCleanTarget) {
+    this.needCleanTarget = needCleanTarget;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
index 3c853c9..a6ab836 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReplTxnWork.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hive.ql.plan;
 import java.io.Serializable;
 
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
@@ -40,6 +42,7 @@ public class ReplTxnWork implements Serializable {
   private List<Long> txnIds;
   private List<TxnToWriteId> txnToWriteIdList;
   private ReplicationSpec replicationSpec;
+  private List<WriteEventInfo> writeEventInfos;
 
   /**
    * OperationType.
@@ -60,6 +63,7 @@ public class ReplTxnWork implements Serializable {
     this.replPolicy = replPolicy;
     this.txnToWriteIdList = txnToWriteIdList;
     this.replicationSpec = replicationSpec;
+    this.writeEventInfos = null;
   }
 
   public ReplTxnWork(String replPolicy, String dbName, String tableName, List<Long> txnIds, OperationType type,
@@ -86,6 +90,13 @@ public class ReplTxnWork implements Serializable {
     this.operation = type;
   }
 
+  public void addWriteEventInfo(WriteEventInfo writeEventInfo) {
+    if (this.writeEventInfos == null) {
+      this.writeEventInfos = new ArrayList<>();
+    }
+    this.writeEventInfos.add(writeEventInfo);
+  }
+
   public List<Long> getTxnIds() {
     return txnIds;
   }
@@ -121,4 +132,8 @@ public class ReplTxnWork implements Serializable {
   public ReplicationSpec getReplicationSpec() {
     return replicationSpec;
   }
+
+  public List<WriteEventInfo> getWriteEventInfos() {
+    return writeEventInfos;
+  }
 }


[03/46] hive git commit: HIVE-20008: Fix second compilation errors in ql (Zoltan Haindrich reviewed by Vineet Garg)

Posted by se...@apache.org.
HIVE-20008: Fix second compilation errors in ql (Zoltan Haindrich reviewed by Vineet Garg)

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


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

Branch: refs/heads/master-txnstats
Commit: b9bac8e641f99e2191040b2ec43c730cb217a6bb
Parents: 221dbe0
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Mon Jul 2 09:37:43 2018 +0200
Committer: Zoltan Haindrich <ki...@rxd.hu>
Committed: Mon Jul 2 09:37:43 2018 +0200

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |   3 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  15 +--
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  28 +++--
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |   5 +-
 .../hive/ql/parse/LoadSemanticAnalyzer.java     |  28 ++---
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  | 107 ++++++++++---------
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |  16 +--
 .../apache/hadoop/hive/ql/plan/ExplainWork.java |  18 ++--
 .../hadoop/hive/ql/reexec/ReExecDriver.java     |   2 +-
 .../ql/parse/TestMacroSemanticAnalyzer.java     |   9 +-
 .../hadoop/hive/ql/parse/TestQBCompact.java     |  15 ++-
 .../authorization/AuthorizationTestUtil.java    |   8 +-
 12 files changed, 126 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 0bbd751..2dfd2aa 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -32,7 +32,6 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.io.PrintStream;
-import java.io.Serializable;
 import java.io.StringWriter;
 import java.io.UnsupportedEncodingException;
 import java.net.URL;
@@ -1781,7 +1780,7 @@ public class QTestUtil {
   }
 
 
-  public List<Task<? extends Serializable>> analyzeAST(ASTNode ast) throws Exception {
+  public List<Task<?>> analyzeAST(ASTNode ast) throws Exception {
 
     // Do semantic analysis and plan generation
     Context ctx = new Context(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index ebea31d..be43686 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.io.UnsupportedEncodingException;
 import java.text.ParseException;
 import java.util.ArrayList;
@@ -36,8 +35,8 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
-import org.antlr.runtime.tree.Tree;
 import org.antlr.runtime.TokenRewriteStream;
+import org.antlr.runtime.tree.Tree;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.conf.Configuration;
@@ -246,7 +245,7 @@ public abstract class BaseSemanticAnalyzer {
       this.queryState = queryState;
       this.conf = queryState.getConf();
       this.db = db;
-      rootTasks = new ArrayList<Task<? extends Serializable>>();
+      rootTasks = new ArrayList<Task<?>>();
       LOG = LoggerFactory.getLogger(this.getClass().getName());
       console = new LogHelper(LOG);
       idToTableNameMap = new HashMap<String, String>();
@@ -289,7 +288,7 @@ public abstract class BaseSemanticAnalyzer {
     // Implementations may choose to override this
   }
 
-  public List<Task<? extends Serializable>> getRootTasks() {
+  public List<Task<?>> getRootTasks() {
     return rootTasks;
   }
 
@@ -309,7 +308,7 @@ public abstract class BaseSemanticAnalyzer {
   }
 
   protected void reset(boolean clearPartsCache) {
-    rootTasks = new ArrayList<Task<? extends Serializable>>();
+    rootTasks = new ArrayList<Task<?>>();
   }
 
   public static String stripIdentifierQuotes(String val) {
@@ -841,7 +840,9 @@ public abstract class BaseSemanticAnalyzer {
   // it throws an error.
   // This method is used to validate check expression since check expression isn't allowed to have subquery
   private static void validateCheckExprAST(ASTNode checkExpr) throws SemanticException {
-    if(checkExpr == null) return;
+    if(checkExpr == null) {
+      return;
+    }
     if(checkExpr.getType() == HiveParser.TOK_SUBQUERY_EXPR) {
       throw new SemanticException(ErrorMsg.INVALID_CSTR_SYNTAX.getMsg("Subqueries are not allowed "
                                                                           + "in Check Constraints"));
@@ -2241,7 +2242,7 @@ public abstract class BaseSemanticAnalyzer {
     return detail == null ? message.getMsg() : message.getMsg(detail.toString());
   }
 
-  public List<Task<? extends Serializable>> getAllRootTasks() {
+  public List<Task<?>> getAllRootTasks() {
     return rootTasks;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/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 f9d6d41..9ad4689 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
@@ -22,7 +22,6 @@ 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.FileNotFoundException;
-import java.io.Serializable;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.net.URI;
@@ -73,7 +72,6 @@ import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.exec.ArchiveUtils;
 import org.apache.hadoop.hive.ql.exec.ColumnStatsUpdateTask;
-import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
@@ -110,7 +108,6 @@ import org.apache.hadoop.hive.ql.plan.AlterResourcePlanDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableAlterPartDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
-import org.apache.hadoop.hive.ql.plan.DDLDesc.DDLDescWithWriteId;
 import org.apache.hadoop.hive.ql.plan.AlterTableExchangePartition;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
 import org.apache.hadoop.hive.ql.plan.AlterWMTriggerDesc;
@@ -124,6 +121,7 @@ import org.apache.hadoop.hive.ql.plan.CreateOrDropTriggerToPoolMappingDesc;
 import org.apache.hadoop.hive.ql.plan.CreateResourcePlanDesc;
 import org.apache.hadoop.hive.ql.plan.CreateWMTriggerDesc;
 import org.apache.hadoop.hive.ql.plan.DDLDesc;
+import org.apache.hadoop.hive.ql.plan.DDLDesc.DDLDescWithWriteId;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.DescFunctionDesc;
@@ -686,7 +684,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeGrantRevokeRole(boolean grant, ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task;
+    Task<?> task;
     if(grant) {
       task = hiveAuthorizationTaskFactory.createGrantRoleTask(ast, getInputs(), getOutputs());
     } else {
@@ -698,7 +696,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeShowGrant(ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+    Task<?> task = hiveAuthorizationTaskFactory.
         createShowGrantTask(ast, ctx.getResFile(), getInputs(), getOutputs());
     if(task != null) {
       rootTasks.add(task);
@@ -707,7 +705,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeGrant(ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+    Task<?> task = hiveAuthorizationTaskFactory.
         createGrantTask(ast, getInputs(), getOutputs());
     if(task != null) {
       rootTasks.add(task);
@@ -715,7 +713,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeRevoke(ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+    Task<?> task = hiveAuthorizationTaskFactory.
         createRevokeTask(ast, getInputs(), getOutputs());
     if(task != null) {
       rootTasks.add(task);
@@ -723,7 +721,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeCreateRole(ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+    Task<?> task = hiveAuthorizationTaskFactory.
         createCreateRoleTask(ast, getInputs(), getOutputs());
     if(task != null) {
       rootTasks.add(task);
@@ -731,7 +729,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeDropRole(ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+    Task<?> task = hiveAuthorizationTaskFactory.
         createDropRoleTask(ast, getInputs(), getOutputs());
     if(task != null) {
       rootTasks.add(task);
@@ -739,7 +737,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
   private void analyzeShowRoleGrant(ASTNode ast) throws SemanticException {
-    Task<? extends Serializable> task = hiveAuthorizationTaskFactory.
+    Task<?> task = hiveAuthorizationTaskFactory.
         createShowRoleGrantTask(ast, ctx.getResFile(), getInputs(), getOutputs());
     if(task != null) {
       rootTasks.add(task);
@@ -1483,7 +1481,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     TruncateTableDesc truncateTblDesc = new TruncateTableDesc(tableName, partSpec, null);
 
     DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), truncateTblDesc);
-    Task<? extends Serializable> truncateTask = TaskFactory.get(ddlWork);
+    Task<?> truncateTask = TaskFactory.get(ddlWork);
 
     // Is this a truncate column command
     List<String> columnNames = null;
@@ -1613,7 +1611,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
           basicStatsWork.setClearAggregatorStats(true);
           StatsWork columnStatsWork = new StatsWork(table, basicStatsWork, conf);
 
-          Task<? extends Serializable> statTask = TaskFactory.get(columnStatsWork);
+          Task<?> statTask = TaskFactory.get(columnStatsWork);
           moveTsk.addDependentTask(statTask);
         }
       } catch (HiveException e) {
@@ -2075,7 +2073,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       addInputsOutputsAlterTable(tableName, partSpec, AlterTableTypes.MERGEFILES);
       DDLWork ddlWork = new DDLWork(getInputs(), getOutputs(), mergeDesc);
       ddlWork.setNeedLock(true);
-      Task<? extends Serializable> mergeTask = TaskFactory.get(ddlWork);
+      Task<?> mergeTask = TaskFactory.get(ddlWork);
       TableDesc tblDesc = Utilities.getTableDesc(tblObj);
       Path queryTmpdir = ctx.getExternalTmpPath(newTblPartLoc);
       mergeDesc.setOutputDir(queryTmpdir);
@@ -2100,7 +2098,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         basicStatsWork.setClearAggregatorStats(true);
         StatsWork columnStatsWork = new StatsWork(tblObj, basicStatsWork, conf);
 
-        Task<? extends Serializable> statTask = TaskFactory.get(columnStatsWork);
+        Task<?> statTask = TaskFactory.get(columnStatsWork);
         moveTsk.addDependentTask(statTask);
       }
 
@@ -3176,7 +3174,7 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
         case HiveParser.TOK_RESTRICT:
           break;
         default:
-          constraintChild = (ASTNode) child;
+          constraintChild = child;
       }
     }
     List<SQLPrimaryKey> primaryKeys = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/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 3a7d99d..6f0a803 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
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.parse;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -165,7 +164,7 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
     sem.validate();
 
     ctx.setResFile(ctx.getLocalTmpPath());
-    List<Task<? extends Serializable>> tasks = sem.getAllRootTasks();
+    List<Task<?>> tasks = sem.getAllRootTasks();
     if (tasks == null) {
       tasks = Collections.emptyList();
     }
@@ -262,7 +261,7 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
 
   @Override
   public boolean skipAuthorization() {
-    List<Task<? extends Serializable>> rootTasks = getRootTasks();
+    List<Task<?>> rootTasks = getRootTasks();
     assert rootTasks != null && rootTasks.size() == 1;
     Task task = rootTasks.get(0);
     return task instanceof ExplainTask && ((ExplainTask)task).getWork().isAuthorize();

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/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 cbacd05..8d33cf5 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
@@ -18,27 +18,26 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import org.apache.commons.codec.DecoderException;
-import org.apache.commons.codec.net.URLCodec;
-import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
 import java.io.IOException;
-import java.io.Serializable;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.ArrayList;
-import java.util.HashSet;
 
 import org.antlr.runtime.tree.Tree;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.net.URLCodec;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.Context;
@@ -54,19 +53,18 @@ import org.apache.hadoop.hive.ql.lockmgr.LockException;
 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.plan.StatsWork;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.BasicStatsWork;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc;
 import org.apache.hadoop.hive.ql.plan.LoadTableDesc.LoadFileType;
 import org.apache.hadoop.hive.ql.plan.MoveWork;
-import org.apache.hadoop.hive.ql.plan.BasicStatsWork;
+import org.apache.hadoop.hive.ql.plan.StatsWork;
 import org.apache.hadoop.mapred.InputFormat;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.mapred.TextInputFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+
 /**
  * LoadSemanticAnalyzer.
  *
@@ -339,7 +337,9 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
 
     // make sure the arguments make sense
     List<FileStatus> files = applyConstraintsAndGetFiles(fromURI, ts.tableHandle);
-    if (queryReWritten) return;
+    if (queryReWritten) {
+      return;
+    }
 
     // for managed tables, make sure the file formats match
     if (TableType.MANAGED_TABLE.equals(ts.tableHandle.getTableType())
@@ -407,7 +407,7 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
       loadTableWork.setInheritTableSpecs(false);
     }
 
-    Task<? extends Serializable> childTask = TaskFactory.get(
+    Task<?> childTask = TaskFactory.get(
         new MoveWork(getInputs(), getOutputs(), loadTableWork, null, true,
             isLocal)
     );
@@ -418,7 +418,7 @@ public class LoadSemanticAnalyzer extends SemanticAnalyzer {
     // Some stats like number of rows require a scan of the data
     // However, some other stats, like number of files, do not require a complete scan
     // Update the stats which do not require a complete scan.
-    Task<? extends Serializable> statTask = null;
+    Task<?> statTask = null;
     if (conf.getBoolVar(HiveConf.ConfVars.HIVESTATSAUTOGATHER)) {
       BasicStatsWork basicStatsWork = new BasicStatsWork(loadTableWork);
       basicStatsWork.setNoStatsAggregator(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/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 b389a9b..2731f19 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
@@ -18,13 +18,32 @@
 
 package org.apache.hadoop.hive.ql.parse;
 
-import com.google.common.base.Splitter;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import com.google.common.math.IntMath;
-import com.google.common.math.LongMath;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.AccessControlException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Deque;
+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.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.function.Supplier;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.Collectors;
+
 import org.antlr.runtime.ClassicToken;
 import org.antlr.runtime.CommonToken;
 import org.antlr.runtime.Token;
@@ -111,11 +130,11 @@ import org.apache.hadoop.hive.ql.io.AcidInputFormat;
 import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.AcidUtils.Operation;
-import org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe;
 import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
 import org.apache.hadoop.hive.ql.io.NullRowsInputFormat;
+import org.apache.hadoop.hive.ql.io.arrow.ArrowColumnarBatchSerDe;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.Dispatcher;
 import org.apache.hadoop.hive.ql.lib.GraphWalker;
@@ -168,7 +187,6 @@ import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowType;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
 import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
-import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
 import org.apache.hadoop.hive.ql.plan.CreateViewDesc;
@@ -254,33 +272,13 @@ import org.apache.hadoop.mapred.OutputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.Serializable;
-import java.security.AccessControlException;
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Deque;
-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.Optional;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.function.Supplier;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-import java.util.stream.Collectors;
-
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVESTATSDBCLASS;
+import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import com.google.common.math.IntMath;
+import com.google.common.math.LongMath;
 
 /**
  * Implementation of the semantic analyzer. It generates the query plan.
@@ -667,7 +665,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
    */
   private boolean isInsertInto(QBParseInfo qbp, String dest) {
     // get the destination and check if it is TABLE
-    if(qbp == null || dest == null ) return false;
+    if(qbp == null || dest == null ) {
+      return false;
+    }
     ASTNode destNode = qbp.getDestForClause(dest);
     if(destNode != null && destNode.getType() == HiveParser.TOK_TAB) {
       return true;
@@ -680,7 +680,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
    * e.g. VALUES(1,3..)
    */
   private boolean isValueClause(ASTNode select) {
-    if(select == null) return false;
+    if(select == null) {
+      return false;
+    }
     if(select.getChildCount() == 1) {
       ASTNode selectExpr = (ASTNode)select.getChild(0);
       if(selectExpr.getChildCount() == 1 ) {
@@ -1262,7 +1264,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   private final CTEClause rootClause = new CTEClause(null, null);
 
   @Override
-  public List<Task<? extends Serializable>> getAllRootTasks() {
+  public List<Task<?>> getAllRootTasks() {
     if (!rootTasksResolved) {
       rootTasks = toRealRootTasks(rootClause.asExecutionOrder());
       rootTasksResolved = true;
@@ -1308,7 +1310,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     Table table;
     SemanticAnalyzer source;
 
-    List<Task<? extends Serializable>> getTasks() {
+    List<Task<?>> getTasks() {
       return source == null ? null : source.rootTasks;
     }
 
@@ -1333,11 +1335,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
   }
 
-  private List<Task<? extends Serializable>> toRealRootTasks(List<CTEClause> execution) {
-    List<Task<? extends Serializable>> cteRoots = new ArrayList<>();
-    List<Task<? extends Serializable>> cteLeafs = new ArrayList<>();
-    List<Task<? extends Serializable>> curTopRoots = null;
-    List<Task<? extends Serializable>> curBottomLeafs = null;
+  private List<Task<?>> toRealRootTasks(List<CTEClause> execution) {
+    List<Task<?>> cteRoots = new ArrayList<>();
+    List<Task<?>> cteLeafs = new ArrayList<>();
+    List<Task<?>> curTopRoots = null;
+    List<Task<?>> curBottomLeafs = null;
     for (int i = 0; i < execution.size(); i++) {
       CTEClause current = execution.get(i);
       if (current.parents.isEmpty() && curTopRoots != null) {
@@ -1345,7 +1347,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         cteLeafs.addAll(curBottomLeafs);
         curTopRoots = curBottomLeafs = null;
       }
-      List<Task<? extends Serializable>> curTasks = current.getTasks();
+      List<Task<?>> curTasks = current.getTasks();
       if (curTasks == null) {
         continue;
       }
@@ -6987,7 +6989,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       checkExpr.addChild(ASTBuilder.createAST(oldColChild.getType(), newColRef));
     }
     else {
-      for(int i=0; i< ((ASTNode)checkExpr).getChildCount(); i++) {
+      for(int i=0; i< checkExpr.getChildCount(); i++) {
         replaceColumnReference((ASTNode)(checkExpr.getChild(i)), col2Col, inputRR);
       }
     }
@@ -12808,19 +12810,19 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         HiveConf.ConfVars.HIVE_REWORK_MAPREDWORK);
 
     // validate all tasks
-    for (Task<? extends Serializable> rootTask : rootTasks) {
+    for (Task<?> rootTask : rootTasks) {
       validate(rootTask, reworkMapredWork);
     }
   }
 
-  private void validate(Task<? extends Serializable> task, boolean reworkMapredWork)
+  private void validate(Task<?> task, boolean reworkMapredWork)
       throws SemanticException {
     Utilities.reworkMapRedWork(task, reworkMapredWork, conf);
     if (task.getChildTasks() == null) {
       return;
     }
 
-    for (Task<? extends Serializable> childTask : task.getChildTasks()) {
+    for (Task<?> childTask : task.getChildTasks()) {
       validate(childTask, reworkMapredWork);
     }
   }
@@ -14921,10 +14923,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       rewrittenQueryStr.append(" partition (");
       boolean first = true;
       for (FieldSchema fschema : partCols) {
-        if (first)
+        if (first) {
           first = false;
-        else
+        } else {
           rewrittenQueryStr.append(", ");
+        }
         //would be nice if there was a way to determine if quotes are needed
         rewrittenQueryStr.append(HiveUtils.unparseIdentifier(fschema.getName(), this.conf));
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
index d9483f8..ce7e65a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/UpdateDeleteSemanticAnalyzer.java
@@ -302,7 +302,7 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
   /**
    * Makes the exportTask run after all other tasks of the "insert into T ..." are done.
    */
-  private void addExportTask(List<Task<? extends Serializable>> rootTasks,
+  private void addExportTask(List<Task<?>> rootTasks,
       Task<ExportWork> exportTask, Task<DDLWork> alterTable) {
     for(Task<? extends  Serializable> t : rootTasks) {
       if(t.getNumChild() <= 0) {
@@ -315,8 +315,9 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
       }
     }
   }
-  private List<Task<? extends Serializable>> findStatsTasks(
-      List<Task<? extends Serializable>> rootTasks, List<Task<? extends Serializable>> statsTasks) {
+
+  private List<Task<?>> findStatsTasks(
+      List<Task<?>> rootTasks, List<Task<?>> statsTasks) {
     for(Task<? extends  Serializable> t : rootTasks) {
       if (t instanceof StatsTask) {
         if(statsTasks == null) {
@@ -330,16 +331,17 @@ public class UpdateDeleteSemanticAnalyzer extends SemanticAnalyzer {
     }
     return statsTasks;
   }
-  private void removeStatsTasks(List<Task<? extends Serializable>> rootTasks) {
-    List<Task<? extends Serializable>> statsTasks = findStatsTasks(rootTasks, null);
+
+  private void removeStatsTasks(List<Task<?>> rootTasks) {
+    List<Task<?>> statsTasks = findStatsTasks(rootTasks, null);
     if(statsTasks == null) {
       return;
     }
-    for(Task<? extends Serializable> statsTask : statsTasks) {
+    for (Task<?> statsTask : statsTasks) {
       if(statsTask.getParentTasks() == null) {
         continue; //should never happen
       }
-      for(Task<? extends Serializable> t : new ArrayList<>(statsTask.getParentTasks())) {
+      for (Task<?> t : new ArrayList<>(statsTask.getParentTasks())) {
         t.removeDependentTask(statsTask);
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/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 cde7852..2cdf8cf 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
@@ -39,8 +39,8 @@ public class ExplainWork implements Serializable {
   private static final long serialVersionUID = 1L;
 
   private Path resFile;
-  private ArrayList<Task<? extends Serializable>> rootTasks;
-  private Task<? extends Serializable> fetchTask;
+  private ArrayList<Task<?>> rootTasks;
+  private Task<?> fetchTask;
   private HashSet<ReadEntity> inputs;
   private ParseContext pCtx;
 
@@ -57,13 +57,13 @@ public class ExplainWork implements Serializable {
 
   public ExplainWork(Path resFile,
       ParseContext pCtx,
-      List<Task<? extends Serializable>> rootTasks,
-      Task<? extends Serializable> fetchTask,
+      List<Task<?>> rootTasks,
+      Task<?> fetchTask,
       BaseSemanticAnalyzer analyzer,
       ExplainConfiguration config,
       String cboInfo) {
     this.resFile = resFile;
-    this.rootTasks = new ArrayList<Task<? extends Serializable>>(rootTasks);
+    this.rootTasks = new ArrayList<Task<?>>(rootTasks);
     this.fetchTask = fetchTask;
     this.analyzer = analyzer;
     if (analyzer != null) {
@@ -82,19 +82,19 @@ public class ExplainWork implements Serializable {
     this.resFile = resFile;
   }
 
-  public ArrayList<Task<? extends Serializable>> getRootTasks() {
+  public ArrayList<Task<?>> getRootTasks() {
     return rootTasks;
   }
 
-  public void setRootTasks(ArrayList<Task<? extends Serializable>> rootTasks) {
+  public void setRootTasks(ArrayList<Task<?>> rootTasks) {
     this.rootTasks = rootTasks;
   }
 
-  public Task<? extends Serializable> getFetchTask() {
+  public Task<?> getFetchTask() {
     return fetchTask;
   }
 
-  public void setFetchTask(Task<? extends Serializable> fetchTask) {
+  public void setFetchTask(Task<?> fetchTask) {
     this.fetchTask = fetchTask;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
index 501f0b4..3bc3b29 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReExecDriver.java
@@ -80,7 +80,7 @@ public class ReExecDriver implements IDriver {
     }
 
     @Override
-    public void postAnalyze(HiveSemanticAnalyzerHookContext context, List<Task<? extends Serializable>> rootTasks)
+    public void postAnalyze(HiveSemanticAnalyzerHookContext context, List<Task<?>> rootTasks)
         throws SemanticException {
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
index 906d70d..0334cf2 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestMacroSemanticAnalyzer.java
@@ -17,11 +17,8 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
-import java.io.Serializable;
 import java.util.List;
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Context;
@@ -33,6 +30,8 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro;
 import org.junit.Before;
 import org.junit.Test;
 
+import junit.framework.Assert;
+
 public class TestMacroSemanticAnalyzer {
 
   private MacroSemanticAnalyzer analyzer;
@@ -54,9 +53,9 @@ public class TestMacroSemanticAnalyzer {
   }
   private void analyze(ASTNode ast) throws Exception {
     analyzer.analyze(ast, context);
-    List<Task<? extends Serializable>> rootTasks = analyzer.getRootTasks();
+    List<Task<?>> rootTasks = analyzer.getRootTasks();
     Assert.assertEquals(1, rootTasks.size());
-    for(Task<? extends Serializable> task : rootTasks) {
+    for (Task<?> task : rootTasks) {
       Assert.assertEquals(0, task.executeTask(null));
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
index 49d900b..9a45ccb 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
-import junit.framework.Assert;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.Context;
@@ -31,15 +34,11 @@ import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.junit.BeforeClass;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import junit.framework.Assert;
 
 /**
  * Tests for parsing and semantic analysis of ALTER TABLE ... compact.
@@ -81,7 +80,7 @@ public class TestQBCompact {
     ASTNode head = (ASTNode)hd.parse(query).getChild(0);
     BaseSemanticAnalyzer a = SemanticAnalyzerFactory.get(queryState, head);
     a.analyze(head, new Context(conf));
-    List<Task<? extends Serializable>> roots = a.getRootTasks();
+    List<Task<?>> roots = a.getRootTasks();
     Assert.assertEquals(1, roots.size());
     return ((DDLWork)roots.get(0).getWork()).getAlterTblSimpleDesc();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b9bac8e6/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
index a76e2ea..40753b6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/authorization/AuthorizationTestUtil.java
@@ -17,22 +17,20 @@
  */
 package org.apache.hadoop.hive.ql.parse.authorization;
 
-import java.io.Serializable;
 import java.util.List;
 
-import junit.framework.Assert;
-
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.QueryState;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
-import org.apache.hadoop.hive.ql.parse.ParseDriver;
 import org.apache.hadoop.hive.ql.parse.ParseUtils;
 import org.apache.hadoop.hive.ql.plan.DDLWork;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
+import junit.framework.Assert;
+
 /**
  * Util function for authorization tests
  */
@@ -50,7 +48,7 @@ public class AuthorizationTestUtil {
     DDLSemanticAnalyzer analyzer = new DDLSemanticAnalyzer(queryState, db);
     SessionState.start(queryState.getConf());
     analyzer.analyze(ast, new Context(queryState.getConf()));
-    List<Task<? extends Serializable>> rootTasks = analyzer.getRootTasks();
+    List<Task<?>> rootTasks = analyzer.getRootTasks();
     return (DDLWork) inList(rootTasks).ofSize(1).get(0).getWork();
   }
 


[07/46] hive git commit: HIVE-19581: add test to show unicode works as predicate in views. (Andrew Sherman, reviewed by Naveen Gangam)

Posted by se...@apache.org.
HIVE-19581: add test to show unicode works as predicate in views. (Andrew Sherman, reviewed by Naveen Gangam)


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

Branch: refs/heads/master-txnstats
Commit: 88da0e8b78c5a912be07ebdaef40492981baf252
Parents: d78d646
Author: Naveen Gangam <ng...@apache.org>
Authored: Mon Jul 2 09:40:42 2018 -0400
Committer: Naveen Gangam <ng...@apache.org>
Committed: Mon Jul 2 09:44:10 2018 -0400

----------------------------------------------------------------------
 .../test/queries/clientpositive/unicode_data.q  |  15 +++
 .../results/clientpositive/unicode_data.q.out   | 123 +++++++++++++++++++
 2 files changed, 138 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/88da0e8b/ql/src/test/queries/clientpositive/unicode_data.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/unicode_data.q b/ql/src/test/queries/clientpositive/unicode_data.q
new file mode 100644
index 0000000..61aab73
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/unicode_data.q
@@ -0,0 +1,15 @@
+create database unicode_data_db;
+use unicode_data_db;
+
+-- test simple unicode data
+create table t_test (name string, value int) ;
+insert into table t_test VALUES ('李四', 100),('ちぱっち', 200);
+select  name, value from t_test;
+select name, value from t_test where name='李四';
+
+-- test view with unicode predicate
+create view t_view_test as select value from t_test where name='李四';
+explain select * from t_view_test;
+select  * from t_view_test;
+
+drop database unicode_data_db cascade;

http://git-wip-us.apache.org/repos/asf/hive/blob/88da0e8b/ql/src/test/results/clientpositive/unicode_data.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/unicode_data.q.out b/ql/src/test/results/clientpositive/unicode_data.q.out
new file mode 100644
index 0000000..4c1decc
--- /dev/null
+++ b/ql/src/test/results/clientpositive/unicode_data.q.out
@@ -0,0 +1,123 @@
+PREHOOK: query: create database unicode_data_db
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:unicode_data_db
+POSTHOOK: query: create database unicode_data_db
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:unicode_data_db
+PREHOOK: query: use unicode_data_db
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:unicode_data_db
+POSTHOOK: query: use unicode_data_db
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:unicode_data_db
+PREHOOK: query: create table t_test (name string, value int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:unicode_data_db
+PREHOOK: Output: unicode_data_db@t_test
+POSTHOOK: query: create table t_test (name string, value int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:unicode_data_db
+POSTHOOK: Output: unicode_data_db@t_test
+PREHOOK: query: insert into table t_test VALUES ('李四', 100),('ちぱっち', 200)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: unicode_data_db@t_test
+POSTHOOK: query: insert into table t_test VALUES ('李四', 100),('ちぱっち', 200)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: unicode_data_db@t_test
+POSTHOOK: Lineage: t_test.name SCRIPT []
+POSTHOOK: Lineage: t_test.value SCRIPT []
+PREHOOK: query: select  name, value from t_test
+PREHOOK: type: QUERY
+PREHOOK: Input: unicode_data_db@t_test
+#### A masked pattern was here ####
+POSTHOOK: query: select  name, value from t_test
+POSTHOOK: type: QUERY
+POSTHOOK: Input: unicode_data_db@t_test
+#### A masked pattern was here ####
+李四	100
+ちぱっち	200
+PREHOOK: query: select name, value from t_test where name='李四'
+PREHOOK: type: QUERY
+PREHOOK: Input: unicode_data_db@t_test
+#### A masked pattern was here ####
+POSTHOOK: query: select name, value from t_test where name='李四'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: unicode_data_db@t_test
+#### A masked pattern was here ####
+李四	100
+PREHOOK: query: create view t_view_test as select value from t_test where name='李四'
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: unicode_data_db@t_test
+PREHOOK: Output: database:unicode_data_db
+PREHOOK: Output: unicode_data_db@t_view_test
+POSTHOOK: query: create view t_view_test as select value from t_test where name='李四'
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: unicode_data_db@t_test
+POSTHOOK: Output: database:unicode_data_db
+POSTHOOK: Output: unicode_data_db@t_view_test
+POSTHOOK: Lineage: t_view_test.value SIMPLE [(t_test)t_test.FieldSchema(name:value, type:int, comment:null), ]
+PREHOOK: query: explain select * from t_view_test
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from t_view_test
+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: t_test
+            filterExpr: (name = '李四') (type: boolean)
+            properties:
+              insideView TRUE
+            Statistics: Num rows: 2 Data size: 26 Basic stats: COMPLETE Column stats: NONE
+            Filter Operator
+              predicate: (name = '李四') (type: boolean)
+              Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE
+              Select Operator
+                expressions: value (type: int)
+                outputColumnNames: _col0
+                Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1 Data size: 13 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+      Execution mode: vectorized
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select  * from t_view_test
+PREHOOK: type: QUERY
+PREHOOK: Input: unicode_data_db@t_test
+PREHOOK: Input: unicode_data_db@t_view_test
+#### A masked pattern was here ####
+POSTHOOK: query: select  * from t_view_test
+POSTHOOK: type: QUERY
+POSTHOOK: Input: unicode_data_db@t_test
+POSTHOOK: Input: unicode_data_db@t_view_test
+#### A masked pattern was here ####
+100
+PREHOOK: query: drop database unicode_data_db cascade
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:unicode_data_db
+PREHOOK: Output: database:unicode_data_db
+PREHOOK: Output: unicode_data_db@t_test
+PREHOOK: Output: unicode_data_db@t_view_test
+POSTHOOK: query: drop database unicode_data_db cascade
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:unicode_data_db
+POSTHOOK: Output: database:unicode_data_db
+POSTHOOK: Output: unicode_data_db@t_test
+POSTHOOK: Output: unicode_data_db@t_view_test


[33/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)


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

Branch: refs/heads/master-txnstats
Commit: f519db7eafacb4b4d2d9fe2a9e10e908d8077224
Parents: 285a9b4
Author: Sankar Hariappan <sa...@apache.org>
Authored: Tue Jul 3 15:32:05 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Tue Jul 3 15:32:05 2018 +0530

----------------------------------------------------------------------
 .../listener/DbNotificationListener.java        |  209 +-
 .../listener/DummyRawStoreFailEvent.java        |   15 +
 .../listener/TestDbNotificationListener.java    |    5 +
 .../hive/ql/parse/TestReplicationScenarios.java |   72 -
 .../TestReplicationScenariosAcidTables.java     |  602 ++-
 ...TestReplicationScenariosAcrossInstances.java |   15 +-
 .../hadoop/hive/ql/parse/WarehouseInstance.java |    5 +
 .../metastore/SynchronizedMetaStoreClient.java  |    5 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    6 +-
 .../hadoop/hive/ql/exec/ReplCopyTask.java       |    5 +-
 .../apache/hadoop/hive/ql/exec/ReplTxnTask.java |   31 +-
 .../hadoop/hive/ql/exec/repl/ReplDumpTask.java  |    4 +-
 .../IncrementalLoadTasksBuilder.java            |   73 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |   37 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   24 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   18 +-
 .../hadoop/hive/ql/lockmgr/DummyTxnManager.java |    7 +-
 .../hadoop/hive/ql/lockmgr/HiveTxnManager.java  |   10 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |  167 +-
 .../hadoop/hive/ql/metadata/HiveUtils.java      |   11 +-
 .../hive/ql/parse/ImportSemanticAnalyzer.java   |   83 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |    8 +-
 .../ql/parse/UpdateDeleteSemanticAnalyzer.java  |   16 +-
 .../hadoop/hive/ql/parse/repl/CopyUtils.java    |    2 +-
 .../hadoop/hive/ql/parse/repl/dump/Utils.java   |    4 -
 .../repl/dump/events/CommitTxnHandler.java      |  125 +-
 .../ql/parse/repl/dump/events/EventHandler.java |   23 +-
 .../parse/repl/dump/events/InsertHandler.java   |    4 +
 .../parse/repl/load/UpdatedMetaDataTracker.java |  124 +-
 .../repl/load/message/AbortTxnHandler.java      |    7 +-
 .../repl/load/message/AllocWriteIdHandler.java  |    2 +-
 .../repl/load/message/CommitTxnHandler.java     |   78 +-
 .../parse/repl/load/message/MessageHandler.java |    8 +-
 .../parse/repl/load/message/OpenTxnHandler.java |    7 +-
 .../apache/hadoop/hive/ql/plan/MoveWork.java    |   12 +-
 .../apache/hadoop/hive/ql/plan/ReplTxnWork.java |   15 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 2675 +++++++-----
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.h    |  126 +
 .../ThriftHiveMetastore_server.skeleton.cpp     |    5 +
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp | 3905 ++++++++++--------
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  218 +-
 .../metastore/api/AddDynamicPartitions.java     |   32 +-
 .../api/AllocateTableWriteIdsRequest.java       |   68 +-
 .../api/AllocateTableWriteIdsResponse.java      |   36 +-
 .../metastore/api/ClearFileMetadataRequest.java |   32 +-
 .../hive/metastore/api/ClientCapabilities.java  |   32 +-
 .../hive/metastore/api/CommitTxnRequest.java    |  168 +-
 .../hive/metastore/api/CompactionRequest.java   |   44 +-
 .../hive/metastore/api/CreationMetadata.java    |   32 +-
 .../metastore/api/FindSchemasByColsResp.java    |   36 +-
 .../hive/metastore/api/FireEventRequest.java    |   32 +-
 .../metastore/api/GetAllFunctionsResponse.java  |   36 +-
 .../api/GetFileMetadataByExprRequest.java       |   32 +-
 .../api/GetFileMetadataByExprResult.java        |   48 +-
 .../metastore/api/GetFileMetadataRequest.java   |   32 +-
 .../metastore/api/GetFileMetadataResult.java    |   44 +-
 .../hive/metastore/api/GetTablesRequest.java    |   32 +-
 .../hive/metastore/api/GetTablesResult.java     |   36 +-
 .../metastore/api/GetValidWriteIdsRequest.java  |   32 +-
 .../metastore/api/GetValidWriteIdsResponse.java |   36 +-
 .../api/HeartbeatTxnRangeResponse.java          |   64 +-
 .../metastore/api/InsertEventRequestData.java   |  227 +-
 .../hadoop/hive/metastore/api/LockRequest.java  |   36 +-
 .../hive/metastore/api/Materialization.java     |   32 +-
 .../api/NotificationEventResponse.java          |   36 +-
 .../metastore/api/PutFileMetadataRequest.java   |   64 +-
 .../api/ReplTblWriteIdStateRequest.java         |   32 +-
 .../hive/metastore/api/SchemaVersion.java       |   36 +-
 .../hive/metastore/api/ShowCompactResponse.java |   36 +-
 .../hive/metastore/api/ShowLocksResponse.java   |   36 +-
 .../hive/metastore/api/TableValidWriteIds.java  |   32 +-
 .../hive/metastore/api/ThriftHiveMetastore.java | 3468 ++++++++++------
 .../hive/metastore/api/WMFullResourcePlan.java  |  144 +-
 .../api/WMGetAllResourcePlanResponse.java       |   36 +-
 .../WMGetTriggersForResourePlanResponse.java    |   36 +-
 .../api/WMValidateResourcePlanResponse.java     |   64 +-
 .../hive/metastore/api/WriteEventInfo.java      | 1012 +++++
 .../api/WriteNotificationLogRequest.java        |  949 +++++
 .../api/WriteNotificationLogResponse.java       |  283 ++
 .../gen-php/metastore/ThriftHiveMetastore.php   | 1630 ++++----
 .../src/gen/thrift/gen-php/metastore/Types.php  | 1630 +++++---
 .../hive_metastore/ThriftHiveMetastore-remote   |    7 +
 .../hive_metastore/ThriftHiveMetastore.py       | 1139 ++---
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  933 +++--
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   86 +-
 .../gen/thrift/gen-rb/thrift_hive_metastore.rb  |   54 +
 .../hadoop/hive/metastore/HiveMetaStore.java    |   86 +
 .../hive/metastore/HiveMetaStoreClient.java     |   10 +-
 .../hadoop/hive/metastore/IMetaStoreClient.java |   16 +-
 .../hive/metastore/MetaStoreEventListener.java  |   12 +
 .../metastore/MetaStoreListenerNotifier.java    |    6 +
 .../hadoop/hive/metastore/ObjectStore.java      |   60 +
 .../apache/hadoop/hive/metastore/RawStore.java  |   14 +
 .../hive/metastore/ReplChangeManager.java       |   10 +-
 .../hive/metastore/cache/CachedStore.java       |   12 +
 .../hive/metastore/events/AcidWriteEvent.java   |   91 +
 .../metastore/messaging/AcidWriteMessage.java   |   50 +
 .../metastore/messaging/CommitTxnMessage.java   |   23 +
 .../hive/metastore/messaging/EventMessage.java  |    3 +-
 .../messaging/MessageDeserializer.java          |    9 +
 .../metastore/messaging/MessageFactory.java     |   12 +
 .../messaging/json/JSONAcidWriteMessage.java    |  150 +
 .../messaging/json/JSONCommitTxnMessage.java    |   95 +
 .../messaging/json/JSONMessageDeserializer.java |    9 +
 .../messaging/json/JSONMessageFactory.java      |    8 +
 .../model/MTxnWriteNotificationLog.java         |  123 +
 .../hive/metastore/tools/SQLGenerator.java      |    9 +
 .../hadoop/hive/metastore/txn/TxnDbUtil.java    |   28 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  187 +-
 .../hadoop/hive/metastore/txn/TxnStore.java     |   11 +
 .../hadoop/hive/metastore/utils/FileUtils.java  |   12 +-
 .../src/main/resources/package.jdo              |   35 +
 .../main/sql/derby/hive-schema-3.1.0.derby.sql  |   15 +
 .../main/sql/derby/hive-schema-4.0.0.derby.sql  |   15 +
 .../sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql  |    1 -
 .../sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql  |   16 +
 .../main/sql/mssql/hive-schema-3.1.0.mssql.sql  |   17 +
 .../main/sql/mssql/hive-schema-4.0.0.mssql.sql  |   17 +
 .../sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql  |   16 +
 .../main/sql/mysql/hive-schema-3.0.0.mysql.sql  |    1 -
 .../main/sql/mysql/hive-schema-3.1.0.mysql.sql  |   16 +
 .../main/sql/mysql/hive-schema-4.0.0.mysql.sql  |   16 +
 .../sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql  |    4 +-
 .../sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql  |   16 +
 .../sql/oracle/hive-schema-3.0.0.oracle.sql     |    1 -
 .../sql/oracle/hive-schema-3.1.0.oracle.sql     |   15 +
 .../sql/oracle/hive-schema-4.0.0.oracle.sql     |   15 +
 .../oracle/upgrade-2.3.0-to-3.0.0.oracle.sql    |    4 +-
 .../oracle/upgrade-3.0.0-to-3.1.0.oracle.sql    |   16 +
 .../sql/postgres/hive-schema-3.0.0.postgres.sql |    2 -
 .../sql/postgres/hive-schema-3.1.0.postgres.sql |   15 +
 .../sql/postgres/hive-schema-4.0.0.postgres.sql |   15 +
 .../upgrade-3.0.0-to-3.1.0.postgres.sql         |   16 +
 .../src/main/thrift/hive_metastore.thrift       |   30 +-
 .../DummyRawStoreControlledCommit.java          |   11 +
 .../DummyRawStoreForJdoConnection.java          |   10 +
 .../HiveMetaStoreClientPreCatalog.java          |   10 +-
 137 files changed, 15896 insertions(+), 7205 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
index 6321f9b..717cc8a 100644
--- a/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
+++ b/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/DbNotificationListener.java
@@ -23,6 +23,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
@@ -75,11 +76,14 @@ import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage.EventType;
 import org.apache.hadoop.hive.metastore.messaging.MessageFactory;
 import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.PartitionFiles;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
+import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -269,10 +273,16 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     public PartitionFiles next() {
       try {
         Partition p = partitionIter.next();
-        List<String> files = Lists.newArrayList(new FileIterator(p.getSd().getLocation()));
+        Iterator<String> fileIterator;
+        //For transactional tables, the actual file copy will be done by acid write event during replay of commit txn.
+        if (!TxnUtils.isTransactionalTable(t)) {
+          List<String> files = Lists.newArrayList(new FileIterator(p.getSd().getLocation()));
+          fileIterator = files.iterator();
+        } else {
+          fileIterator = Collections.emptyIterator();
+        }
         PartitionFiles partitionFiles =
-            new PartitionFiles(Warehouse.makePartName(t.getPartitionKeys(), p.getValues()),
-            files.iterator());
+            new PartitionFiles(Warehouse.makePartName(t.getPartitionKeys(), p.getValues()), fileIterator);
         return partitionFiles;
       } catch (MetaException e) {
         throw new RuntimeException(e);
@@ -414,10 +424,15 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
   class FileChksumIterator implements Iterator<String> {
     private List<String> files;
     private List<String> chksums;
+    private List<String> subDirs;
     int i = 0;
     FileChksumIterator(List<String> files, List<String> chksums) {
+      this(files, chksums, null);
+    }
+    FileChksumIterator(List<String> files, List<String> chksums, List<String> subDirs) {
       this.files = files;
       this.chksums = chksums;
+      this.subDirs = subDirs;
     }
     @Override
     public boolean hasNext() {
@@ -428,7 +443,8 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     public String next() {
       String result;
       try {
-        result = ReplChangeManager.encodeFileUri(files.get(i), chksums != null ? chksums.get(i) : null, null);
+        result = ReplChangeManager.encodeFileUri(files.get(i), chksums != null ? chksums.get(i) : null,
+                subDirs != null ? subDirs.get(i) : null);
       } catch (IOException e) {
         // File operations failed
         LOG.error("Encoding file URI failed with error " + e.getMessage());
@@ -623,6 +639,23 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     }
   }
 
+  @Override
+  public void onAcidWrite(AcidWriteEvent acidWriteEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
+    AcidWriteMessage msg = msgFactory.buildAcidWriteMessage(acidWriteEvent,
+            new FileChksumIterator(acidWriteEvent.getFiles(), acidWriteEvent.getChecksums(),
+                    acidWriteEvent.getSubDirs()));
+    NotificationEvent event = new NotificationEvent(0, now(), EventType.ACID_WRITE.toString(), msg.toString());
+    event.setMessageFormat(msgFactory.getMessageFormat());
+    event.setDbName(acidWriteEvent.getDatabase());
+    event.setTableName(acidWriteEvent.getTable());
+    try {
+      addWriteNotificationLog(event, acidWriteEvent, dbConn, sqlGenerator, msg);
+    } catch (SQLException e) {
+      throw new MetaException("Unable to add write notification log " + StringUtils.stringifyException(e));
+    }
+  }
+
   private int now() {
     long millis = System.currentTimeMillis();
     millis /= 1000;
@@ -634,12 +667,133 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
     return (int)millis;
   }
 
+  /**
+   * Close statement instance.
+   * @param stmt statement instance.
+   */
+  private static void closeStmt(Statement stmt) {
+    try {
+      if (stmt != null && !stmt.isClosed()) {
+        stmt.close();
+      }
+    } catch (SQLException e) {
+      LOG.warn("Failed to close statement " + e.getMessage());
+    }
+  }
+
+  /**
+   * Close the ResultSet.
+   * @param rs may be {@code null}
+   */
+  private static void close(ResultSet rs) {
+    try {
+      if (rs != null && !rs.isClosed()) {
+        rs.close();
+      }
+    } catch(SQLException ex) {
+      LOG.warn("Failed to close result set " + ex.getMessage());
+    }
+  }
+
+  private long getNextNLId(Statement stmt, SQLGenerator sqlGenerator, String sequence)
+          throws SQLException, MetaException {
+    String s = sqlGenerator.addForUpdateClause("select \"NEXT_VAL\" from " +
+            "\"SEQUENCE_TABLE\" where \"SEQUENCE_NAME\" = " + quoteString(sequence));
+    LOG.debug("Going to execute query <" + s + ">");
+    ResultSet rs = null;
+    try {
+      rs = stmt.executeQuery(s);
+      if (!rs.next()) {
+        throw new MetaException("Transaction database not properly configured, can't find next NL id.");
+      }
+
+      long nextNLId = rs.getLong(1);
+      long updatedNLId = nextNLId + 1;
+      s = "update \"SEQUENCE_TABLE\" set \"NEXT_VAL\" = " + updatedNLId + " where \"SEQUENCE_NAME\" = " +
+              quoteString(sequence);
+      LOG.debug("Going to execute update <" + s + ">");
+      stmt.executeUpdate(s);
+      return nextNLId;
+    }finally {
+      close(rs);
+    }
+  }
+
+  private void addWriteNotificationLog(NotificationEvent event, AcidWriteEvent acidWriteEvent, Connection dbConn,
+                                 SQLGenerator sqlGenerator, AcidWriteMessage msg) throws MetaException, SQLException {
+    LOG.debug("DbNotificationListener: adding write notification log for : {}", event.getMessage());
+    assert ((dbConn != null) && (sqlGenerator != null));
+
+    Statement stmt =null;
+    ResultSet rs = null;
+    String dbName = acidWriteEvent.getDatabase();
+    String tblName = acidWriteEvent.getTable();
+    String partition = acidWriteEvent.getPartition();
+    String tableObj = msg.getTableObjStr();
+    String partitionObj = msg.getPartitionObjStr();
+    String files = ReplChangeManager.joinWithSeparator(msg.getFiles());
+
+    try {
+      stmt = dbConn.createStatement();
+      if (sqlGenerator.getDbProduct() == MYSQL) {
+        stmt.execute("SET @@session.sql_mode=ANSI_QUOTES");
+      }
+
+      String s = sqlGenerator.addForUpdateClause("select \"WNL_FILES\", \"WNL_ID\" from" +
+                      " \"TXN_WRITE_NOTIFICATION_LOG\" " +
+                      "where \"WNL_DATABASE\" = " + quoteString(dbName) +
+                      "and \"WNL_TABLE\" = " + quoteString(tblName) +  " and \"WNL_PARTITION\" = " +
+                      quoteString(partition) + " and \"WNL_TXNID\" = " + Long.toString(acidWriteEvent.getTxnId()));
+      LOG.debug("Going to execute query <" + s + ">");
+      rs = stmt.executeQuery(s);
+      if (!rs.next()) {
+        // if rs is empty then no lock is taken and thus it can not cause deadlock.
+        long nextNLId = getNextNLId(stmt, sqlGenerator,
+                "org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog");
+        s = "insert into \"TXN_WRITE_NOTIFICATION_LOG\" (\"WNL_ID\", \"WNL_TXNID\", \"WNL_WRITEID\"," +
+                " \"WNL_DATABASE\", \"WNL_TABLE\"," +
+                " \"WNL_PARTITION\", \"WNL_TABLE_OBJ\", \"WNL_PARTITION_OBJ\", \"WNL_FILES\", \"WNL_EVENT_TIME\")" +
+                " values (" + nextNLId
+                + "," + acidWriteEvent.getTxnId() +  "," + acidWriteEvent.getWriteId()+  "," +
+                quoteString(dbName)+  "," +  quoteString(tblName)+  "," + quoteString(partition)+  "," +
+                quoteString(tableObj)+  "," + quoteString(partitionObj) +  "," +  quoteString(files)+
+                "," +  now() + ")";
+        LOG.info("Going to execute insert <" + s + ">");
+        stmt.execute(sqlGenerator.addEscapeCharacters(s));
+      } else {
+        String existingFiles = rs.getString(1);
+        if (existingFiles.contains(sqlGenerator.addEscapeCharacters(files))) {
+          // If list of files are already present then no need to update it again. This scenario can come in case of
+          // retry done to the meta store for the same operation.
+          LOG.info("file list " + files + " already present");
+          return;
+        }
+        long nlId = rs.getLong(2);
+        files = ReplChangeManager.joinWithSeparator(Lists.newArrayList(files, existingFiles));
+        s = "update \"TXN_WRITE_NOTIFICATION_LOG\" set \"WNL_TABLE_OBJ\" = " +  quoteString(tableObj) + "," +
+                " \"WNL_PARTITION_OBJ\" = " + quoteString(partitionObj) + "," +
+                " \"WNL_FILES\" = " + quoteString(files) + "," +
+                " \"WNL_EVENT_TIME\" = " + now() +
+                " where \"WNL_ID\" = " + nlId;
+        LOG.info("Going to execute update <" + s + ">");
+        stmt.executeUpdate(sqlGenerator.addEscapeCharacters(s));
+      }
+    } catch (SQLException e) {
+      LOG.warn("failed to add write notification log" + e.getMessage());
+      throw e;
+    } finally {
+      closeStmt(stmt);
+      close(rs);
+    }
+  }
+
   static String quoteString(String input) {
     return "'" + input + "'";
   }
 
   private void addNotificationLog(NotificationEvent event, ListenerEvent listenerEvent, Connection dbConn,
                                   SQLGenerator sqlGenerator) throws MetaException, SQLException {
+    LOG.debug("DbNotificationListener: adding notification log for : {}", event.getMessage());
     if ((dbConn == null) || (sqlGenerator == null)) {
       LOG.info("connection or sql generator is not set so executing sql via DN");
       process(event, listenerEvent);
@@ -669,22 +823,8 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       LOG.debug("Going to execute update <" + s + ">");
       stmt.executeUpdate(s);
 
-      s = sqlGenerator.addForUpdateClause("select \"NEXT_VAL\" from " +
-              "\"SEQUENCE_TABLE\" where \"SEQUENCE_NAME\" = " +
-              " 'org.apache.hadoop.hive.metastore.model.MNotificationLog'");
-      LOG.debug("Going to execute query <" + s + ">");
-      rs = stmt.executeQuery(s);
-      if (!rs.next()) {
-        throw new MetaException("failed to get next NEXT_VAL from SEQUENCE_TABLE");
-      }
-
-      long nextNLId = rs.getLong(1);
-      long updatedNLId = nextNLId + 1;
-      s = "update \"SEQUENCE_TABLE\" set \"NEXT_VAL\" = " + updatedNLId + " where \"SEQUENCE_NAME\" = " +
-
-              " 'org.apache.hadoop.hive.metastore.model.MNotificationLog'";
-      LOG.debug("Going to execute update <" + s + ">");
-      stmt.executeUpdate(s);
+      long nextNLId = getNextNLId(stmt, sqlGenerator,
+              "org.apache.hadoop.hive.metastore.model.MNotificationLog");
 
       List<String> insert = new ArrayList<>();
 
@@ -712,20 +852,8 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       LOG.warn("failed to add notification log" + e.getMessage());
       throw e;
     } finally {
-      if (stmt != null && !stmt.isClosed()) {
-        try {
-          stmt.close();
-        } catch (SQLException e) {
-          LOG.warn("Failed to close statement " + e.getMessage());
-        }
-      }
-      if (rs != null && !rs.isClosed()) {
-        try {
-          rs.close();
-        } catch (SQLException e) {
-          LOG.warn("Failed to close result set " + e.getMessage());
-        }
-      }
+      closeStmt(stmt);
+      close(rs);
     }
   }
 
@@ -742,12 +870,12 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
         event.getMessage());
     HMSHandler.getMSForConf(conf).addNotificationEvent(event);
 
-      // Set the DB_NOTIFICATION_EVENT_ID for future reference by other listeners.
-      if (event.isSetEventId()) {
-        listenerEvent.putParameter(
-            MetaStoreEventListenerConstants.DB_NOTIFICATION_EVENT_ID_KEY_NAME,
-            Long.toString(event.getEventId()));
-      }
+    // Set the DB_NOTIFICATION_EVENT_ID for future reference by other listeners.
+    if (event.isSetEventId()) {
+      listenerEvent.putParameter(
+          MetaStoreEventListenerConstants.DB_NOTIFICATION_EVENT_ID_KEY_NAME,
+          Long.toString(event.getEventId()));
+    }
   }
 
   private static class CleanerThread extends Thread {
@@ -768,6 +896,7 @@ public class DbNotificationListener extends TransactionalMetaStoreEventListener
       while (true) {
         try {
           rs.cleanNotificationEvents(ttl);
+          rs.cleanWriteNotificationEvents(ttl);
         } catch (Exception ex) {
           //catching exceptions here makes sure that the thread doesn't die in case of unexpected
           //exceptions

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
index abf67a8..b4b118e 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/DummyRawStoreFailEvent.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.thrift.TException;
@@ -880,6 +881,20 @@ public class DummyRawStoreFailEvent implements RawStore, Configurable {
   }
 
   @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    if (!shouldEventSucceed) {
+      //throw exception to simulate an issue with cleaner thread
+      throw new RuntimeException("Dummy exception while cleaning write notifications");
+    }
+    objectStore.cleanWriteNotificationEvents(olderThan);
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return objectStore.getAllWriteEventInfo(txnId, dbName, tableName);
+  }
+
+  @Override
   public CurrentNotificationEventId getCurrentNotificationEventId() {
     return objectStore.getCurrentNotificationEventId();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index eef917e..82429e3 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.ListenerEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
 import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
@@ -238,6 +239,10 @@ public class TestDbNotificationListener {
     public void onAllocWriteId(AllocWriteIdEvent allocWriteIdEvent) throws MetaException {
       pushEventId(EventType.ALLOC_WRITE_ID, allocWriteIdEvent);
     }
+
+    public void onAcidWrite(AcidWriteEvent acidWriteEvent) throws MetaException {
+      pushEventId(EventType.ACID_WRITE, acidWriteEvent);
+    }
   }
 
   @SuppressWarnings("rawtypes")

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 46c623d..c82a933 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -2833,78 +2833,6 @@ public class TestReplicationScenarios {
     verifyRun("SELECT max(a) from " + replDbName + ".ptned2 where b=1", new String[]{"8"}, driverMirror);
   }
 
-  // TODO: This test should be removed once ACID tables replication is supported.
-  @Test
-  public void testSkipTables() throws Exception {
-    String testName = "skipTables";
-    String dbName = createDB(testName, driver);
-    String replDbName = dbName + "_dupe";
-
-    // TODO: this is wrong; this test sets up dummy txn manager and so it cannot create ACID tables.
-    //       If I change it to use proper txn manager, the setup for some tests hangs.
-    //       This used to work by accident, now this works due a test flag. The test needs to be fixed.
-    // Create table
-    run("CREATE TABLE " + dbName + ".acid_table (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
-    run("CREATE TABLE " + dbName + ".mm_table (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true'," +
-        " 'transactional_properties'='insert_only')", driver);
-    verifyIfTableExist(dbName, "acid_table", metaStoreClient);
-    verifyIfTableExist(dbName, "mm_table", metaStoreClient);
-
-    // Bootstrap test
-    Tuple bootstrapDump = bootstrapLoadAndVerify(dbName, replDbName);
-    String replDumpId = bootstrapDump.lastReplId;
-    verifyIfTableNotExist(replDbName, "acid_table", metaStoreClientMirror);
-    verifyIfTableNotExist(replDbName, "mm_table", metaStoreClientMirror);
-
-    // Test alter table
-    run("ALTER TABLE " + dbName + ".acid_table RENAME TO " + dbName + ".acid_table_rename", driver);
-    verifyIfTableExist(dbName, "acid_table_rename", metaStoreClient);
-
-    // Dummy create table command to mark proper last repl ID after dump
-    run("CREATE TABLE " + dbName + ".dummy (a int)", driver);
-
-    // Perform REPL-DUMP/LOAD
-    Tuple incrementalDump = incrementalLoadAndVerify(dbName, replDumpId, replDbName);
-    replDumpId = incrementalDump.lastReplId;
-    verifyIfTableNotExist(replDbName, "acid_table_rename", metaStoreClientMirror);
-
-    // Create another table for incremental repl verification
-    run("CREATE TABLE " + dbName + ".acid_table_incremental (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true')", driver);
-    run("CREATE TABLE " + dbName + ".mm_table_incremental (key int, value int) PARTITIONED BY (load_date date) " +
-        "CLUSTERED BY(key) INTO 2 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true'," +
-        " 'transactional_properties'='insert_only')", driver);
-    verifyIfTableExist(dbName, "acid_table_incremental", metaStoreClient);
-    verifyIfTableExist(dbName, "mm_table_incremental", metaStoreClient);
-
-    // Dummy insert into command to mark proper last repl ID after dump
-    run("INSERT INTO " + dbName + ".dummy values(1)", driver);
-
-    // Perform REPL-DUMP/LOAD
-    incrementalDump = incrementalLoadAndVerify(dbName, replDumpId, replDbName);
-    replDumpId = incrementalDump.lastReplId;
-    verifyIfTableNotExist(replDbName, "acid_table_incremental", metaStoreClientMirror);
-    verifyIfTableNotExist(replDbName, "mm_table_incremental", metaStoreClientMirror);
-
-    // Test adding a constraint
-    run("ALTER TABLE " + dbName + ".acid_table_incremental ADD CONSTRAINT key_pk PRIMARY KEY (key) DISABLE NOVALIDATE", driver);
-    try {
-      List<SQLPrimaryKey> pks = metaStoreClient.getPrimaryKeys(new PrimaryKeysRequest(dbName, "acid_table_incremental"));
-      assertEquals(pks.size(), 1);
-    } catch (TException te) {
-      assertNull(te);
-    }
-
-    // Dummy insert into command to mark proper last repl ID after dump
-    run("INSERT INTO " + dbName + ".dummy values(2)", driver);
-
-    // Perform REPL-DUMP/LOAD
-    incrementalLoadAndVerify(dbName, replDumpId, replDbName);
-    verifyIfTableNotExist(replDbName, "acid_table_incremental", metaStoreClientMirror);
-  }
-
   @Test
   public void testDeleteStagingDir() throws IOException {
     String testName = "deleteStagingDir";

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
index 86c0405..8c683cf 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcidTables.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.junit.rules.TestName;
+
 import org.junit.rules.TestRule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,6 +54,8 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import javax.annotation.Nullable;
+import java.util.Collections;
+import com.google.common.collect.Lists;
 
 /**
  * TestReplicationScenariosAcidTables - test replication for ACID tables
@@ -66,8 +69,13 @@ public class TestReplicationScenariosAcidTables {
 
   protected static final Logger LOG = LoggerFactory.getLogger(TestReplicationScenarios.class);
   private static WarehouseInstance primary, replica, replicaNonAcid;
-  private String primaryDbName, replicatedDbName;
   private static HiveConf conf;
+  private String primaryDbName, replicatedDbName, primaryDbNameExtra;
+  private enum OperationType {
+    REPL_TEST_ACID_INSERT, REPL_TEST_ACID_INSERT_SELECT, REPL_TEST_ACID_CTAS,
+    REPL_TEST_ACID_INSERT_OVERWRITE, REPL_TEST_ACID_INSERT_IMPORT, REPL_TEST_ACID_INSERT_LOADLOCAL,
+    REPL_TEST_ACID_INSERT_UNION
+  }
 
   @BeforeClass
   public static void classLevelSetup() throws Exception {
@@ -80,9 +88,13 @@ public class TestReplicationScenariosAcidTables {
         put("fs.defaultFS", miniDFSCluster.getFileSystem().getUri().toString());
         put("hive.support.concurrency", "true");
         put("hive.txn.manager", "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager");
-        put("hive.repl.dump.include.acid.tables", "true");
         put("hive.metastore.client.capability.check", "false");
         put("hive.repl.bootstrap.dump.open.txn.timeout", "1s");
+        put("hive.exec.dynamic.partition.mode", "nonstrict");
+        put("hive.strict.checks.bucketing", "false");
+        put("hive.mapred.mode", "nonstrict");
+        put("mapred.input.dir.recursive", "true");
+        put("hive.metastore.disallow.incompatible.col.type.changes", "false");
     }};
     primary = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
     replica = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
@@ -90,7 +102,6 @@ public class TestReplicationScenariosAcidTables {
         put("fs.defaultFS", miniDFSCluster.getFileSystem().getUri().toString());
         put("hive.support.concurrency", "false");
         put("hive.txn.manager", "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager");
-        put("hive.repl.dump.include.acid.tables", "true");
         put("hive.metastore.client.capability.check", "false");
     }};
     replicaNonAcid = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf1);
@@ -109,6 +120,9 @@ public class TestReplicationScenariosAcidTables {
     replicatedDbName = "replicated_" + primaryDbName;
     primary.run("create database " + primaryDbName + " WITH DBPROPERTIES ( '" +
             SOURCE_OF_REPLICATION + "' = '1,2,3')");
+    primaryDbNameExtra = primaryDbName+"_extra";
+    primary.run("create database " + primaryDbNameExtra + " WITH DBPROPERTIES ( '" +
+            SOURCE_OF_REPLICATION + "' = '1,2,3')");
   }
 
   @After
@@ -116,6 +130,7 @@ public class TestReplicationScenariosAcidTables {
     primary.run("drop database if exists " + primaryDbName + " cascade");
     replica.run("drop database if exists " + replicatedDbName + " cascade");
     replicaNonAcid.run("drop database if exists " + replicatedDbName + " cascade");
+    primary.run("drop database if exists " + primaryDbName + "_extra cascade");
   }
 
   @Test
@@ -482,4 +497,585 @@ public class TestReplicationScenariosAcidTables {
     primary.run("DROP TABLE " + dbName + ".normal");
     primary.run("drop database " + dbName);
   }
+
+  @Test
+  public void testAcidTableIncrementalReplication() throws Throwable {
+    WarehouseInstance.Tuple bootStrapDump = primary.dump(primaryDbName, null);
+    replica.load(replicatedDbName, bootStrapDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName)
+            .verifyResult(bootStrapDump.lastReplicationId);
+    List<String> selectStmtList = new ArrayList<>();
+    List<String[]> expectedValues = new ArrayList<>();
+
+    appendInsert(selectStmtList, expectedValues);
+    appendDelete(selectStmtList, expectedValues);
+    appendUpdate(selectStmtList, expectedValues);
+    appendTruncate(selectStmtList, expectedValues);
+    appendInsertIntoFromSelect(selectStmtList, expectedValues);
+    appendMerge(selectStmtList, expectedValues);
+    appendCreateAsSelect(selectStmtList, expectedValues);
+    appendImport(selectStmtList, expectedValues);
+    appendInsertOverwrite(selectStmtList, expectedValues);
+    //appendLoadLocal(selectStmtList, expectedValues);
+    appendInsertUnion(selectStmtList, expectedValues);
+    appendMultiStatementTxn(selectStmtList, expectedValues);
+    appendMultiStatementTxnUpdateDelete(selectStmtList, expectedValues);
+
+    verifyIncrementalLoad(selectStmtList, expectedValues, bootStrapDump.lastReplicationId);
+  }
+
+  private void appendInsert(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsert";
+    String tableNameMM = tableName + "_MM";
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendDelete(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testDelete";
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    deleteRecords(tableName);
+    selectStmtList.add("select count(*) from " + tableName);
+    expectedValues.add(new String[] {"0"});
+  }
+
+  private void appendUpdate(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testUpdate";
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    updateRecords(tableName);
+    selectStmtList.add("select value from " + tableName + " order by value");
+    expectedValues.add(new String[] {"1", "100", "100", "100", "100"});
+  }
+
+  private void appendTruncate(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testTruncate";
+    String tableNameMM = tableName + "_MM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    truncateTable(primaryDbName, tableName);
+    selectStmtList.add("select count(*) from " + tableName);
+    expectedValues.add(new String[] {"0"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    truncateTable(primaryDbName, tableNameMM);
+    selectStmtList.add("select count(*) from " + tableNameMM);
+    expectedValues.add(new String[] {"0"});
+  }
+
+  private void appendInsertIntoFromSelect(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsertIntoFromSelect";
+    String tableNameMM =tableName + "_MM";
+    String tableNameSelect = testName.getMethodName() + "_Select";
+    String tableNameSelectMM = testName.getMethodName() + "_SelectMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameSelect, false, OperationType.REPL_TEST_ACID_INSERT_SELECT);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameSelect + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameSelectMM, true, OperationType.REPL_TEST_ACID_INSERT_SELECT);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameSelectMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendMerge(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testMerge";
+    String tableNameMerge = testName.getMethodName() + "_Merge";
+
+    insertForMerge(tableName, tableNameMerge, false);
+    selectStmtList.add("select last_update_user from " + tableName + " order by last_update_user");
+    expectedValues.add(new String[] {"creation", "creation", "creation", "creation", "creation",
+            "creation", "creation", "merge_update", "merge_insert", "merge_insert"});
+    selectStmtList.add("select ID from " + tableNameMerge + " order by ID");
+    expectedValues.add(new String[] {"1", "4", "7", "8", "8", "11"});
+  }
+
+  private void appendCreateAsSelect(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testCreateAsSelect";
+    String tableNameMM = tableName + "_MM";
+    String tableNameCTAS = testName.getMethodName() + "_CTAS";
+    String tableNameCTASMM = testName.getMethodName() + "_CTASMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameCTAS, false, OperationType.REPL_TEST_ACID_CTAS);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameCTAS + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameCTASMM, true, OperationType.REPL_TEST_ACID_CTAS);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameCTASMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendImport(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testImport";
+    String tableNameMM = tableName + "_MM";
+    String tableNameImport = testName.getMethodName() + "_Import";
+    String tableNameImportMM = testName.getMethodName() + "_ImportMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameImport, false, OperationType.REPL_TEST_ACID_INSERT_IMPORT);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameImport + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameImportMM, true, OperationType.REPL_TEST_ACID_INSERT_IMPORT);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameImportMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendInsertOverwrite(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsertOverwrite";
+    String tableNameOW = testName.getMethodName() +"_OW";
+    String tableNameMM = tableName + "_MM";
+    String tableNameOWMM = testName.getMethodName() +"_OWMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameOW, false, OperationType.REPL_TEST_ACID_INSERT_OVERWRITE);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameOW + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameOWMM, true, OperationType.REPL_TEST_ACID_INSERT_OVERWRITE);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameOWMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  //TODO: need to check why its failing. Loading to acid table from local path is failing.
+  private void appendLoadLocal(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testLoadLocal";
+    String tableNameLL = testName.getMethodName() +"_LL";
+    String tableNameMM = tableName + "_MM";
+    String tableNameLLMM = testName.getMethodName() +"_LLMM";
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameLL, false, OperationType.REPL_TEST_ACID_INSERT_LOADLOCAL);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameLL + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameLLMM, true, OperationType.REPL_TEST_ACID_INSERT_LOADLOCAL);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+    selectStmtList.add("select key from " + tableNameLLMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendInsertUnion(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testInsertUnion";
+    String tableNameUnion = testName.getMethodName() +"_UNION";
+    String tableNameMM = tableName + "_MM";
+    String tableNameUnionMM = testName.getMethodName() +"_UNIONMM";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String[] resultArrayUnion = new String[]{"1", "1", "2", "2", "3", "3", "4", "4", "5", "5"};
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableName, tableNameUnion, false, OperationType.REPL_TEST_ACID_INSERT_UNION);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add( "select key from " + tableNameUnion + " order by key");
+    expectedValues.add(resultArrayUnion);
+    selectStmtList.add("select key from " + tableName + "_nopart" + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add("select key from " + tableNameUnion + "_nopart" + " order by key");
+    expectedValues.add(resultArrayUnion);
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    insertRecords(tableNameMM, tableNameUnionMM, true, OperationType.REPL_TEST_ACID_INSERT_UNION);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add( "select key from " + tableNameUnionMM + " order by key");
+    expectedValues.add(resultArrayUnion);
+    selectStmtList.add("select key from " + tableNameMM + "_nopart" + " order by key");
+    expectedValues.add(resultArray);
+    selectStmtList.add("select key from " + tableNameUnionMM + "_nopart" + " order by key");
+    expectedValues.add(resultArrayUnion);
+  }
+
+  private void appendMultiStatementTxn(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable {
+    String tableName = testName.getMethodName() + "testMultiStatementTxn";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableNameMM = tableName + "_MM";
+    String tableProperty = "'transactional'='true'";
+
+    insertIntoDB(primaryDbName, tableName, tableProperty, resultArray, true);
+    selectStmtList.add("select key from " + tableName + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+
+    tableProperty = setMMtableProperty(tableProperty);
+    insertIntoDB(primaryDbName, tableNameMM, tableProperty, resultArray, true);
+    selectStmtList.add("select key from " + tableNameMM + " order by key");
+    expectedValues.add(new String[]{"1", "2", "3", "4", "5"});
+  }
+
+  private void appendMultiStatementTxnUpdateDelete(List<String> selectStmtList, List<String[]> expectedValues)
+          throws Throwable {
+    String tableName = testName.getMethodName() + "testMultiStatementTxnUpdate";
+    String tableNameDelete = testName.getMethodName() + "testMultiStatementTxnDelete";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableProperty = "'transactional'='true'";
+
+    insertIntoDB(primaryDbName, tableName, tableProperty, resultArray, true);
+    updateRecords(tableName);
+    selectStmtList.add("select value from " + tableName + " order by value");
+    expectedValues.add(new String[] {"1", "100", "100", "100", "100"});
+
+    insertIntoDB(primaryDbName, tableNameDelete, tableProperty, resultArray, true);
+    deleteRecords(tableNameDelete);
+    selectStmtList.add("select count(*) from " + tableNameDelete);
+    expectedValues.add(new String[] {"0"});
+  }
+
+  @Test
+  public void testReplCM() throws Throwable {
+    String tableName = testName.getMethodName();
+    String tableNameMM = testName.getMethodName() + "_MM";
+    String[] result = new String[]{"5"};
+
+    WarehouseInstance.Tuple incrementalDump;
+    WarehouseInstance.Tuple bootStrapDump = primary.dump(primaryDbName, null);
+    replica.load(replicatedDbName, bootStrapDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName)
+            .verifyResult(bootStrapDump.lastReplicationId);
+
+    insertRecords(tableName, null, false, OperationType.REPL_TEST_ACID_INSERT);
+    incrementalDump = primary.dump(primaryDbName, bootStrapDump.lastReplicationId);
+    truncateTable(primaryDbName, tableName);
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(Lists.newArrayList("select count(*) from " + tableName,
+                                              "select count(*) from " + tableName + "_nopart"),
+                            Lists.newArrayList(result, result));
+
+    insertRecords(tableNameMM, null, true, OperationType.REPL_TEST_ACID_INSERT);
+    incrementalDump = primary.dump(primaryDbName, bootStrapDump.lastReplicationId);
+    truncateTable(primaryDbName, tableNameMM);
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(Lists.newArrayList("select count(*) from " + tableNameMM,
+            "select count(*) from " + tableNameMM + "_nopart"),
+            Lists.newArrayList(result, result));
+  }
+
+  @Test
+  public void testMultiDBTxn() throws Throwable {
+    String tableName = testName.getMethodName();
+    String dbName1 = tableName + "_db1";
+    String dbName2 = tableName + "_db2";
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableProperty = "'transactional'='true'";
+    String txnStrStart = "START TRANSACTION";
+    String txnStrCommit = "COMMIT";
+
+    WarehouseInstance.Tuple incrementalDump;
+    primary.run("alter database default set dbproperties ('repl.source.for' = '1, 2, 3')");
+    WarehouseInstance.Tuple bootStrapDump = primary.dump("`*`", null);
+
+    primary.run("use " + primaryDbName)
+          .run("create database " + dbName1 + " WITH DBPROPERTIES ( '" + SOURCE_OF_REPLICATION + "' = '1,2,3')")
+          .run("create database " + dbName2 + " WITH DBPROPERTIES ( '" + SOURCE_OF_REPLICATION + "' = '1,2,3')")
+          .run("CREATE TABLE " + dbName1 + "." + tableName + " (key int, value int) PARTITIONED BY (load_date date) " +
+                  "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+          .run("use " + dbName1)
+          .run("SHOW TABLES LIKE '" + tableName + "'")
+          .verifyResult(tableName)
+          .run("CREATE TABLE " + dbName2 + "." + tableName + " (key int, value int) PARTITIONED BY (load_date date) " +
+                  "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+          .run("use " + dbName2)
+          .run("SHOW TABLES LIKE '" + tableName + "'")
+          .verifyResult(tableName)
+          .run(txnStrStart)
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-02') VALUES (5, 5)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-01') VALUES (1, 1)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-01') VALUES (2, 2)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-01') VALUES (2, 2)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-02') VALUES (3, 3)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-02') VALUES (3, 3)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-03') VALUES (4, 4)")
+          .run("INSERT INTO " + dbName1 + "." + tableName + " partition (load_date='2016-03-02') VALUES (5, 5)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-01') VALUES (1, 1)")
+          .run("INSERT INTO " + dbName2 + "." + tableName + " partition (load_date='2016-03-03') VALUES (4, 4)")
+          .run("select key from " + dbName2 + "." + tableName + " order by key")
+          .verifyResults(resultArray)
+          .run("select key from " + dbName1 + "." + tableName + " order by key")
+          .verifyResults(resultArray)
+          .run(txnStrCommit);
+
+    incrementalDump = primary.dump("`*`", bootStrapDump.lastReplicationId);
+
+    // Due to the limitation that we can only have one instance of Persistence Manager Factory in a JVM
+    // we are not able to create multiple embedded derby instances for two different MetaStore instances.
+    primary.run("drop database " + primaryDbName + " cascade");
+    primary.run("drop database " + dbName1 + " cascade");
+    primary.run("drop database " + dbName2 + " cascade");
+    //End of additional steps
+
+    replica.loadWithoutExplain("", bootStrapDump.dumpLocation)
+            .run("REPL STATUS default")
+            .verifyResult(bootStrapDump.lastReplicationId);
+
+    replica.loadWithoutExplain("", incrementalDump.dumpLocation)
+          .run("REPL STATUS " + dbName1)
+          .run("select key from " + dbName1 + "." + tableName + " order by key")
+          .verifyResults(resultArray)
+          .run("select key from " + dbName2 + "." + tableName + " order by key")
+          .verifyResults(resultArray);
+
+    replica.run("drop database " + primaryDbName + " cascade");
+    replica.run("drop database " + dbName1 + " cascade");
+    replica.run("drop database " + dbName2 + " cascade");
+  }
+
+  private void verifyResultsInReplica(List<String> selectStmtList, List<String[]> expectedValues) throws Throwable  {
+    for (int idx = 0; idx < selectStmtList.size(); idx++) {
+      replica.run("use " + replicatedDbName)
+              .run(selectStmtList.get(idx))
+              .verifyResults(expectedValues.get(idx));
+    }
+  }
+
+  private WarehouseInstance.Tuple verifyIncrementalLoad(List<String> selectStmtList,
+                                                  List<String[]> expectedValues, String lastReplId) throws Throwable {
+    WarehouseInstance.Tuple incrementalDump = primary.dump(primaryDbName, lastReplId);
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(selectStmtList, expectedValues);
+
+    replica.loadWithoutExplain(replicatedDbName, incrementalDump.dumpLocation)
+            .run("REPL STATUS " + replicatedDbName).verifyResult(incrementalDump.lastReplicationId);
+    verifyResultsInReplica(selectStmtList, expectedValues);
+    return incrementalDump;
+  }
+
+  private void deleteRecords(String tableName) throws Throwable {
+    primary.run("use " + primaryDbName)
+            .run("delete from " + tableName)
+            .run("select count(*) from " + tableName)
+            .verifyResult("0");
+  }
+
+  private void updateRecords(String tableName) throws Throwable {
+    primary.run("use " + primaryDbName)
+            .run("update " + tableName + " set value = 100 where key >= 2")
+            .run("select value from " + tableName + " order by value")
+            .verifyResults(new String[] {"1", "100", "100", "100", "100"});
+  }
+
+  private void truncateTable(String dbName, String tableName) throws Throwable {
+    primary.run("use " + dbName)
+            .run("truncate table " + tableName)
+            .run("select count(*) from " + tableName)
+            .verifyResult("0")
+            .run("truncate table " + tableName + "_nopart")
+            .run("select count(*) from " + tableName + "_nopart")
+            .verifyResult("0");
+  }
+
+  private WarehouseInstance.Tuple verifyLoad(String tableName, String tableNameOp, String lastReplId) throws Throwable {
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    if (tableNameOp == null) {
+      return verifyIncrementalLoad(Lists.newArrayList("select key from " + tableName + " order by key",
+              "select key from " + tableName + "_nopart order by key"),
+              Lists.newArrayList(resultArray, resultArray), lastReplId);
+    }
+    return verifyIncrementalLoad(Lists.newArrayList("select key from " + tableName + " order by key",
+                                                    "select key from " + tableNameOp + " order by key",
+                                                    "select key from " + tableName + "_nopart" + " order by key",
+                                                    "select key from " + tableNameOp + "_nopart" + " order by key"),
+                    Lists.newArrayList(resultArray, resultArray, resultArray, resultArray), lastReplId);
+  }
+
+  private void insertIntoDB(String dbName, String tableName, String tableProperty, String[] resultArray, boolean isTxn)
+          throws Throwable {
+    String txnStrStart = "START TRANSACTION";
+    String txnStrCommit = "COMMIT";
+    if (!isTxn) {
+      txnStrStart = "use " + dbName; //dummy
+      txnStrCommit = "use " + dbName; //dummy
+    }
+    primary.run("use " + dbName);
+    primary.run("CREATE TABLE " + tableName + " (key int, value int) PARTITIONED BY (load_date date) " +
+            "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+            .run("SHOW TABLES LIKE '" + tableName + "'")
+            .verifyResult(tableName)
+            .run("CREATE TABLE " + tableName + "_nopart (key int, value int) " +
+                    "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+            .run("SHOW TABLES LIKE '" + tableName + "_nopart'")
+            .run("ALTER TABLE " + tableName + " ADD PARTITION (load_date='2016-03-03')")
+            .run(txnStrStart)
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-01') VALUES (1, 1)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-01') VALUES (2, 2)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-02') VALUES (3, 3)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-03') VALUES (4, 4)")
+            .run("INSERT INTO " + tableName + " partition (load_date='2016-03-02') VALUES (5, 5)")
+            .run("select key from " + tableName + " order by key")
+            .verifyResults(resultArray)
+            .run("INSERT INTO " + tableName + "_nopart (key, value) select key, value from " + tableName)
+            .run("select key from " + tableName + "_nopart" + " order by key")
+            .verifyResults(resultArray)
+            .run(txnStrCommit);
+  }
+
+  private void insertIntoDB(String dbName, String tableName, String tableProperty, String[] resultArray)
+          throws Throwable {
+    insertIntoDB(dbName, tableName, tableProperty, resultArray, false);
+  }
+
+  private void insertRecords(String tableName, String tableNameOp, boolean isMMTable,
+                             OperationType opType) throws Throwable {
+    insertRecordsIntoDB(primaryDbName, tableName, tableNameOp, isMMTable, opType);
+  }
+
+  private void insertRecordsIntoDB(String DbName, String tableName, String tableNameOp, boolean isMMTable,
+                             OperationType opType) throws Throwable {
+    String[] resultArray = new String[]{"1", "2", "3", "4", "5"};
+    String tableProperty = "'transactional'='true'";
+    if (isMMTable) {
+      tableProperty = setMMtableProperty(tableProperty);
+    }
+    primary.run("use " + DbName);
+
+    switch (opType) {
+      case REPL_TEST_ACID_INSERT:
+        insertIntoDB(DbName, tableName, tableProperty, resultArray);
+        insertIntoDB(primaryDbNameExtra, tableName, tableProperty, resultArray);
+        return;
+      case REPL_TEST_ACID_INSERT_OVERWRITE:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+              "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( "+ tableProperty + " )")
+        .run("INSERT INTO " + tableNameOp + " partition (load_date='2016-03-01') VALUES (2, 2)")
+        .run("INSERT INTO " + tableNameOp + " partition (load_date='2016-03-01') VALUES (10, 12)")
+        .run("INSERT INTO " + tableNameOp + " partition (load_date='2016-03-02') VALUES (11, 1)")
+        .run("select key from " + tableNameOp + " order by key")
+        .verifyResults(new String[]{"2", "10", "11"})
+        .run("insert overwrite table " + tableNameOp + " select * from " + tableName)
+        .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( "+ tableProperty + " )")
+        .run("INSERT INTO " + tableNameOp + "_nopart VALUES (2, 2)")
+        .run("INSERT INTO " + tableNameOp + "_nopart VALUES (10, 12)")
+        .run("INSERT INTO " + tableNameOp + "_nopart VALUES (11, 1)")
+        .run("select key from " + tableNameOp + "_nopart" + " order by key")
+        .verifyResults(new String[]{"2", "10", "11"})
+        .run("insert overwrite table " + tableNameOp + "_nopart select * from " + tableName + "_nopart")
+        .run("select key from " + tableNameOp + "_nopart" + " order by key");
+        break;
+      case REPL_TEST_ACID_INSERT_SELECT:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+            "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + " )")
+        .run("insert into " + tableNameOp + " partition (load_date) select * from " + tableName)
+        .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + " )")
+        .run("insert into " + tableNameOp + "_nopart select * from " + tableName + "_nopart");
+        break;
+      case REPL_TEST_ACID_INSERT_IMPORT:
+        String path = "hdfs:///tmp/" + DbName + "/";
+        String exportPath = "'" + path + tableName + "/'";
+        String exportPathNoPart = "'" + path + tableName + "_nopart/'";
+        primary.run("export table " + tableName + " to " + exportPath)
+        .run("import table " + tableNameOp + " from " + exportPath)
+        .run("export table " + tableName + "_nopart to " + exportPathNoPart)
+        .run("import table " + tableNameOp + "_nopart from " + exportPathNoPart);
+        break;
+      case REPL_TEST_ACID_CTAS:
+        primary.run("create table " + tableNameOp + " as select * from " + tableName)
+                .run("create table " + tableNameOp + "_nopart as select * from " + tableName + "_nopart");
+        break;
+      case REPL_TEST_ACID_INSERT_LOADLOCAL:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+              "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+        .run("SHOW TABLES LIKE '" + tableNameOp + "'")
+        .verifyResult(tableNameOp)
+        .run("INSERT OVERWRITE LOCAL DIRECTORY './test.dat' SELECT a.* FROM " + tableName + " a")
+        .run("LOAD DATA LOCAL INPATH './test.dat' OVERWRITE INTO TABLE " + tableNameOp +
+                " PARTITION (load_date='2008-08-15')")
+        .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                      "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+        .run("SHOW TABLES LIKE '" + tableNameOp + "_nopart'")
+        .verifyResult(tableNameOp + "_nopart")
+        .run("LOAD DATA LOCAL INPATH './test.dat' OVERWRITE INTO TABLE " + tableNameOp + "_nopart");
+        break;
+      case REPL_TEST_ACID_INSERT_UNION:
+        primary.run("CREATE TABLE " + tableNameOp + " (key int, value int) PARTITIONED BY (load_date date) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+                .run("SHOW TABLES LIKE '" + tableNameOp + "'")
+                .verifyResult(tableNameOp)
+                .run("insert overwrite table " + tableNameOp + " partition (load_date) select * from " + tableName +
+                    " union all select * from " + tableName)
+                .run("CREATE TABLE " + tableNameOp + "_nopart (key int, value int) " +
+                "CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ( " + tableProperty + ")")
+                .run("insert overwrite table " + tableNameOp + "_nopart select * from " + tableName +
+                        "_nopart union all select * from " + tableName + "_nopart");
+        resultArray = new String[]{"1", "2", "3", "4", "5", "1", "2", "3", "4", "5"};
+        break;
+      default:
+        return;
+    }
+    primary.run("select key from " + tableNameOp + " order by key").verifyResults(resultArray);
+    primary.run("select key from " + tableNameOp + "_nopart" + " order by key").verifyResults(resultArray);
+  }
+
+  private String setMMtableProperty(String tableProperty) throws Throwable  {
+    return tableProperty.concat(", 'transactional_properties' = 'insert_only'");
+  }
+
+  private void insertForMerge(String tableName, String tableNameMerge, boolean isMMTable) throws Throwable  {
+    String tableProperty = "'transactional'='true'";
+    if (isMMTable) {
+      tableProperty = setMMtableProperty(tableProperty);
+    }
+    primary.run("use " + primaryDbName)
+        .run("CREATE TABLE " + tableName + "( ID int, TranValue string, last_update_user string) PARTITIONED BY " +
+                "(tran_date string) CLUSTERED BY (ID) into 5 buckets STORED AS ORC TBLPROPERTIES " +
+                " ( "+ tableProperty + " )")
+        .run("SHOW TABLES LIKE '" + tableName + "'")
+        .verifyResult(tableName)
+        .run("CREATE TABLE " + tableNameMerge + " ( ID int, TranValue string, tran_date string) STORED AS ORC ")
+        .run("SHOW TABLES LIKE '" + tableNameMerge + "'")
+        .verifyResult(tableNameMerge)
+        .run("INSERT INTO " + tableName + " PARTITION (tran_date) VALUES (1, 'value_01', 'creation', '20170410')," +
+                " (2, 'value_02', 'creation', '20170410'), (3, 'value_03', 'creation', '20170410'), " +
+                " (4, 'value_04', 'creation', '20170410'), (5, 'value_05', 'creation', '20170413'), " +
+                " (6, 'value_06', 'creation', '20170413'), (7, 'value_07', 'creation', '20170413'),  " +
+                " (8, 'value_08', 'creation', '20170413'), (9, 'value_09', 'creation', '20170413'), " +
+                " (10, 'value_10','creation', '20170413')")
+        .run("select ID from " + tableName + " order by ID")
+        .verifyResults(new String[] {"1", "2", "3", "4", "5", "6", "7", "8", "9", "10"})
+        .run("INSERT INTO " + tableNameMerge + " VALUES (1, 'value_01', '20170410'), " +
+                " (4, NULL, '20170410'), (7, 'value_77777', '20170413'), " +
+                " (8, NULL, '20170413'), (8, 'value_08', '20170415'), " +
+                "(11, 'value_11', '20170415')")
+        .run("select ID from " + tableNameMerge + " order by ID")
+        .verifyResults(new String[] {"1", "4", "7", "8", "8", "11"})
+        .run("MERGE INTO " + tableName + " AS T USING " + tableNameMerge + " AS S ON T.ID = S.ID and" +
+                " T.tran_date = S.tran_date WHEN MATCHED AND (T.TranValue != S.TranValue AND S.TranValue " +
+                " IS NOT NULL) THEN UPDATE SET TranValue = S.TranValue, last_update_user = " +
+                " 'merge_update' WHEN MATCHED AND S.TranValue IS NULL THEN DELETE WHEN NOT MATCHED " +
+                " THEN INSERT VALUES (S.ID, S.TranValue,'merge_insert', S.tran_date)")
+        .run("select last_update_user from " + tableName + " order by last_update_user")
+        .verifyResults(new String[] {"creation", "creation", "creation", "creation", "creation",
+                "creation", "creation", "merge_update", "merge_insert", "merge_insert"});
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
index ff7f9bc..16c124c 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
@@ -321,8 +321,7 @@ public class TestReplicationScenariosAcrossInstances {
             "clustered by(key) into 2 buckets stored as orc tblproperties ('transactional'='true')")
         .run("create table table1 (i int, j int)")
         .run("insert into table1 values (1,2)")
-        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     replica.load(replicatedDbName, tuple.dumpLocation)
         .run("use " + replicatedDbName)
@@ -341,8 +340,7 @@ public class TestReplicationScenariosAcrossInstances {
         .run("create table table2 (a int, city string) partitioned by (country string)")
         .run("create table table3 (i int, j int)")
         .run("insert into table1 values (1,2)")
-        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump(primaryDbName, null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     replica.load(replicatedDbName, bootstrapTuple.dumpLocation)
         .run("use " + replicatedDbName)
@@ -467,8 +465,7 @@ public class TestReplicationScenariosAcrossInstances {
                 SOURCE_OF_REPLICATION + "' = '1,2,3')")
         .run("use " + dbTwo)
         .run("create table t1 (i int, j int)")
-        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     /*
       Due to the limitation that we can only have one instance of Persistence Manager Factory in a JVM
@@ -527,8 +524,7 @@ public class TestReplicationScenariosAcrossInstances {
         .run("use " + dbOne)
         .run("create table t1 (i int, j int) partitioned by (load_date date) "
             + "clustered by(i) into 2 buckets stored as orc tblproperties ('transactional'='true') ")
-        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-            "'hive.repl.dump.include.acid.tables'='true'"));
+        .dump("`*`", null, Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     String dbTwo = primaryDbName + randomTwo;
     WarehouseInstance.Tuple incrementalTuple = primary
@@ -539,8 +535,7 @@ public class TestReplicationScenariosAcrossInstances {
         .run("use " + dbOne)
         .run("create table t2 (a int, b int)")
         .dump("`*`", bootstrapTuple.lastReplicationId,
-            Arrays.asList("'hive.repl.dump.metadata.only'='true'",
-                "'hive.repl.dump.include.acid.tables'='true'"));
+            Arrays.asList("'hive.repl.dump.metadata.only'='true'"));
 
     /*
       Due to the limitation that we can only have one instance of Persistence Manager Factory in a JVM

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index f666df1..1e3478d 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -249,6 +249,11 @@ public class WarehouseInstance implements Closeable {
     return this;
   }
 
+  WarehouseInstance loadWithoutExplain(String replicatedDbName, String dumpLocation) throws Throwable {
+    run("REPL LOAD " + replicatedDbName + " FROM '" + dumpLocation + "'");
+    return this;
+  }
+
   WarehouseInstance load(String replicatedDbName, String dumpLocation, List<String> withClauseOptions)
           throws Throwable {
     String replLoadCmd = "REPL LOAD " + replicatedDbName + " FROM '" + dumpLocation + "'";

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
index f87a6aa..2ba6d07 100644
--- a/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/metastore/SynchronizedMetaStoreClient.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.thrift.TException;
 
 
@@ -109,6 +110,10 @@ public final class SynchronizedMetaStoreClient {
     return client.fireListenerEvent(rqst);
   }
 
+  public synchronized void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException {
+    client.addWriteNotificationLog(rqst);
+  }
+
   public synchronized void close() {
     client.close();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 19097f5..bf7749d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -139,7 +139,11 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
       if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_INSERT_INTO_MULTILEVEL_DIRS)) {
         deletePath = createTargetPath(targetPath, tgtFs);
       }
-      Hive.clearDestForSubDirSrc(conf, targetPath, sourcePath, false);
+      //For acid table incremental replication, just copy the content of staging directory to destination.
+      //No need to clean it.
+      if (work.isNeedCleanTarget()) {
+        Hive.clearDestForSubDirSrc(conf, targetPath, sourcePath, false);
+      }
       // Set isManaged to false as this is not load data operation for which it is needed.
       if (!Hive.moveFile(conf, sourcePath, targetPath, true, false, false)) {
         try {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
index 3a7f1bc..d095de6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplCopyTask.java
@@ -151,10 +151,11 @@ public class ReplCopyTask extends Task<ReplCopyWork> implements Serializable {
           continue;
         }
         String destFileName = srcFile.getCmPath().getName();
-        Path destFile = new Path(toPath, destFileName);
+        Path destRoot = CopyUtils.getCopyDestination(srcFile, toPath);
+        Path destFile = new Path(destRoot, destFileName);
         if (dstFs.exists(destFile)) {
           String destFileWithSourceName = srcFile.getSourcePath().getName();
-          Path newDestFile = new Path(toPath, destFileWithSourceName);
+          Path newDestFile = new Path(destRoot, destFileWithSourceName);
           boolean result = dstFs.rename(destFile, newDestFile);
           if (!result) {
             throw new IllegalStateException(

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
index 5bbc25a..c2953c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReplTxnTask.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
@@ -60,8 +62,19 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
           return 0;
         }
       } catch (InvalidTableException e) {
-        LOG.info("Table does not exist so, ignoring the operation as it might be a retry(idempotent) case.");
-        return 0;
+        // In scenarios like import to mm tables, the alloc write id event is generated before create table event.
+        try {
+          Database database = Hive.get().getDatabase(work.getDbName());
+          if (!replicationSpec.allowReplacementInto(database.getParameters())) {
+            // if the event is already replayed, then no need to replay it again.
+            LOG.debug("ReplTxnTask: Event is skipped as it is already replayed. Event Id: " +
+                    replicationSpec.getReplicationState() + "Event Type: " + work.getOperationType());
+            return 0;
+          }
+        } catch (HiveException e1) {
+          LOG.error("Get database failed with exception " + e1.getMessage());
+          return 1;
+        }
       } catch (HiveException e) {
         LOG.error("Get table failed with exception " + e.getMessage());
         return 1;
@@ -85,10 +98,16 @@ public class ReplTxnTask extends Task<ReplTxnWork> {
         }
         return 0;
       case REPL_COMMIT_TXN:
-        for (long txnId : work.getTxnIds()) {
-          txnManager.replCommitTxn(replPolicy, txnId);
-          LOG.info("Replayed CommitTxn Event for policy " + replPolicy + " with srcTxn " + txnId);
-        }
+        // Currently only one commit txn per event is supported.
+        assert (work.getTxnIds().size() == 1);
+
+        long txnId = work.getTxnIds().get(0);
+        CommitTxnRequest commitTxnRequest = new CommitTxnRequest(txnId);
+        commitTxnRequest.setReplPolicy(work.getReplPolicy());
+        commitTxnRequest.setWriteEventInfos(work.getWriteEventInfos());
+        txnManager.replCommitTxn(commitTxnRequest);
+        LOG.info("Replayed CommitTxn Event for replPolicy: " + replPolicy + " with srcTxn: " + txnId +
+                "WriteEventInfos: " + work.getWriteEventInfos());
         return 0;
       case REPL_ALLOC_WRITE_ID:
         assert work.getTxnToWriteIdList() != null;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
index e48657c..82ecad1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/repl/ReplDumpTask.java
@@ -199,7 +199,9 @@ public class ReplDumpTask extends Task<ReplDumpWork> implements Serializable {
         cmRoot,
         getHive(),
         conf,
-        getNewEventOnlyReplicationSpec(ev.getEventId())
+        getNewEventOnlyReplicationSpec(ev.getEventId()),
+        work.dbNameOrPattern,
+        work.tableNameOrPattern
     );
     EventHandler eventHandler = EventHandlerFactory.handlerFor(ev);
     eventHandler.handle(context);


[29/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index bc4d168..7f06b3b 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -16592,6 +16592,11 @@ void CommitTxnRequest::__set_replPolicy(const std::string& val) {
 __isset.replPolicy = true;
 }
 
+void CommitTxnRequest::__set_writeEventInfos(const std::vector<WriteEventInfo> & val) {
+  this->writeEventInfos = val;
+__isset.writeEventInfos = true;
+}
+
 uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -16630,6 +16635,26 @@ uint32_t CommitTxnRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->writeEventInfos.clear();
+            uint32_t _size673;
+            ::apache::thrift::protocol::TType _etype676;
+            xfer += iprot->readListBegin(_etype676, _size673);
+            this->writeEventInfos.resize(_size673);
+            uint32_t _i677;
+            for (_i677 = 0; _i677 < _size673; ++_i677)
+            {
+              xfer += this->writeEventInfos[_i677].read(iprot);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.writeEventInfos = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -16658,6 +16683,19 @@ uint32_t CommitTxnRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeString(this->replPolicy);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.writeEventInfos) {
+    xfer += oprot->writeFieldBegin("writeEventInfos", ::apache::thrift::protocol::T_LIST, 3);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->writeEventInfos.size()));
+      std::vector<WriteEventInfo> ::const_iterator _iter678;
+      for (_iter678 = this->writeEventInfos.begin(); _iter678 != this->writeEventInfos.end(); ++_iter678)
+      {
+        xfer += (*_iter678).write(oprot);
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -16667,18 +16705,21 @@ void swap(CommitTxnRequest &a, CommitTxnRequest &b) {
   using ::std::swap;
   swap(a.txnid, b.txnid);
   swap(a.replPolicy, b.replPolicy);
+  swap(a.writeEventInfos, b.writeEventInfos);
   swap(a.__isset, b.__isset);
 }
 
-CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other673) {
-  txnid = other673.txnid;
-  replPolicy = other673.replPolicy;
-  __isset = other673.__isset;
+CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other679) {
+  txnid = other679.txnid;
+  replPolicy = other679.replPolicy;
+  writeEventInfos = other679.writeEventInfos;
+  __isset = other679.__isset;
 }
-CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other674) {
-  txnid = other674.txnid;
-  replPolicy = other674.replPolicy;
-  __isset = other674.__isset;
+CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other680) {
+  txnid = other680.txnid;
+  replPolicy = other680.replPolicy;
+  writeEventInfos = other680.writeEventInfos;
+  __isset = other680.__isset;
   return *this;
 }
 void CommitTxnRequest::printTo(std::ostream& out) const {
@@ -16686,6 +16727,231 @@ void CommitTxnRequest::printTo(std::ostream& out) const {
   out << "CommitTxnRequest(";
   out << "txnid=" << to_string(txnid);
   out << ", " << "replPolicy="; (__isset.replPolicy ? (out << to_string(replPolicy)) : (out << "<null>"));
+  out << ", " << "writeEventInfos="; (__isset.writeEventInfos ? (out << to_string(writeEventInfos)) : (out << "<null>"));
+  out << ")";
+}
+
+
+WriteEventInfo::~WriteEventInfo() throw() {
+}
+
+
+void WriteEventInfo::__set_writeId(const int64_t val) {
+  this->writeId = val;
+}
+
+void WriteEventInfo::__set_database(const std::string& val) {
+  this->database = val;
+}
+
+void WriteEventInfo::__set_table(const std::string& val) {
+  this->table = val;
+}
+
+void WriteEventInfo::__set_files(const std::string& val) {
+  this->files = val;
+}
+
+void WriteEventInfo::__set_partition(const std::string& val) {
+  this->partition = val;
+__isset.partition = true;
+}
+
+void WriteEventInfo::__set_tableObj(const std::string& val) {
+  this->tableObj = val;
+__isset.tableObj = true;
+}
+
+void WriteEventInfo::__set_partitionObj(const std::string& val) {
+  this->partitionObj = val;
+__isset.partitionObj = true;
+}
+
+uint32_t WriteEventInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_writeId = false;
+  bool isset_database = false;
+  bool isset_table = false;
+  bool isset_files = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_I64) {
+          xfer += iprot->readI64(this->writeId);
+          isset_writeId = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->database);
+          isset_database = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->table);
+          isset_table = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->files);
+          isset_files = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->partition);
+          this->__isset.partition = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->tableObj);
+          this->__isset.tableObj = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readString(this->partitionObj);
+          this->__isset.partitionObj = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_writeId)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_database)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_table)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_files)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t WriteEventInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+  xfer += oprot->writeStructBegin("WriteEventInfo");
+
+  xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 1);
+  xfer += oprot->writeI64(this->writeId);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeString(this->database);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRING, 3);
+  xfer += oprot->writeString(this->table);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("files", ::apache::thrift::protocol::T_STRING, 4);
+  xfer += oprot->writeString(this->files);
+  xfer += oprot->writeFieldEnd();
+
+  if (this->__isset.partition) {
+    xfer += oprot->writeFieldBegin("partition", ::apache::thrift::protocol::T_STRING, 5);
+    xfer += oprot->writeString(this->partition);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.tableObj) {
+    xfer += oprot->writeFieldBegin("tableObj", ::apache::thrift::protocol::T_STRING, 6);
+    xfer += oprot->writeString(this->tableObj);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.partitionObj) {
+    xfer += oprot->writeFieldBegin("partitionObj", ::apache::thrift::protocol::T_STRING, 7);
+    xfer += oprot->writeString(this->partitionObj);
+    xfer += oprot->writeFieldEnd();
+  }
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(WriteEventInfo &a, WriteEventInfo &b) {
+  using ::std::swap;
+  swap(a.writeId, b.writeId);
+  swap(a.database, b.database);
+  swap(a.table, b.table);
+  swap(a.files, b.files);
+  swap(a.partition, b.partition);
+  swap(a.tableObj, b.tableObj);
+  swap(a.partitionObj, b.partitionObj);
+  swap(a.__isset, b.__isset);
+}
+
+WriteEventInfo::WriteEventInfo(const WriteEventInfo& other681) {
+  writeId = other681.writeId;
+  database = other681.database;
+  table = other681.table;
+  files = other681.files;
+  partition = other681.partition;
+  tableObj = other681.tableObj;
+  partitionObj = other681.partitionObj;
+  __isset = other681.__isset;
+}
+WriteEventInfo& WriteEventInfo::operator=(const WriteEventInfo& other682) {
+  writeId = other682.writeId;
+  database = other682.database;
+  table = other682.table;
+  files = other682.files;
+  partition = other682.partition;
+  tableObj = other682.tableObj;
+  partitionObj = other682.partitionObj;
+  __isset = other682.__isset;
+  return *this;
+}
+void WriteEventInfo::printTo(std::ostream& out) const {
+  using ::apache::thrift::to_string;
+  out << "WriteEventInfo(";
+  out << "writeId=" << to_string(writeId);
+  out << ", " << "database=" << to_string(database);
+  out << ", " << "table=" << to_string(table);
+  out << ", " << "files=" << to_string(files);
+  out << ", " << "partition="; (__isset.partition ? (out << to_string(partition)) : (out << "<null>"));
+  out << ", " << "tableObj="; (__isset.tableObj ? (out << to_string(tableObj)) : (out << "<null>"));
+  out << ", " << "partitionObj="; (__isset.partitionObj ? (out << to_string(partitionObj)) : (out << "<null>"));
   out << ")";
 }
 
@@ -16789,14 +17055,14 @@ uint32_t ReplTblWriteIdStateRequest::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size675;
-            ::apache::thrift::protocol::TType _etype678;
-            xfer += iprot->readListBegin(_etype678, _size675);
-            this->partNames.resize(_size675);
-            uint32_t _i679;
-            for (_i679 = 0; _i679 < _size675; ++_i679)
+            uint32_t _size683;
+            ::apache::thrift::protocol::TType _etype686;
+            xfer += iprot->readListBegin(_etype686, _size683);
+            this->partNames.resize(_size683);
+            uint32_t _i687;
+            for (_i687 = 0; _i687 < _size683; ++_i687)
             {
-              xfer += iprot->readString(this->partNames[_i679]);
+              xfer += iprot->readString(this->partNames[_i687]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16856,10 +17122,10 @@ uint32_t ReplTblWriteIdStateRequest::write(::apache::thrift::protocol::TProtocol
     xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 6);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-      std::vector<std::string> ::const_iterator _iter680;
-      for (_iter680 = this->partNames.begin(); _iter680 != this->partNames.end(); ++_iter680)
+      std::vector<std::string> ::const_iterator _iter688;
+      for (_iter688 = this->partNames.begin(); _iter688 != this->partNames.end(); ++_iter688)
       {
-        xfer += oprot->writeString((*_iter680));
+        xfer += oprot->writeString((*_iter688));
       }
       xfer += oprot->writeListEnd();
     }
@@ -16881,23 +17147,23 @@ void swap(ReplTblWriteIdStateRequest &a, ReplTblWriteIdStateRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other681) {
-  validWriteIdlist = other681.validWriteIdlist;
-  user = other681.user;
-  hostName = other681.hostName;
-  dbName = other681.dbName;
-  tableName = other681.tableName;
-  partNames = other681.partNames;
-  __isset = other681.__isset;
-}
-ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other682) {
-  validWriteIdlist = other682.validWriteIdlist;
-  user = other682.user;
-  hostName = other682.hostName;
-  dbName = other682.dbName;
-  tableName = other682.tableName;
-  partNames = other682.partNames;
-  __isset = other682.__isset;
+ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other689) {
+  validWriteIdlist = other689.validWriteIdlist;
+  user = other689.user;
+  hostName = other689.hostName;
+  dbName = other689.dbName;
+  tableName = other689.tableName;
+  partNames = other689.partNames;
+  __isset = other689.__isset;
+}
+ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other690) {
+  validWriteIdlist = other690.validWriteIdlist;
+  user = other690.user;
+  hostName = other690.hostName;
+  dbName = other690.dbName;
+  tableName = other690.tableName;
+  partNames = other690.partNames;
+  __isset = other690.__isset;
   return *this;
 }
 void ReplTblWriteIdStateRequest::printTo(std::ostream& out) const {
@@ -16952,14 +17218,14 @@ uint32_t GetValidWriteIdsRequest::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->fullTableNames.clear();
-            uint32_t _size683;
-            ::apache::thrift::protocol::TType _etype686;
-            xfer += iprot->readListBegin(_etype686, _size683);
-            this->fullTableNames.resize(_size683);
-            uint32_t _i687;
-            for (_i687 = 0; _i687 < _size683; ++_i687)
+            uint32_t _size691;
+            ::apache::thrift::protocol::TType _etype694;
+            xfer += iprot->readListBegin(_etype694, _size691);
+            this->fullTableNames.resize(_size691);
+            uint32_t _i695;
+            for (_i695 = 0; _i695 < _size691; ++_i695)
             {
-              xfer += iprot->readString(this->fullTableNames[_i687]);
+              xfer += iprot->readString(this->fullTableNames[_i695]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17000,10 +17266,10 @@ uint32_t GetValidWriteIdsRequest::write(::apache::thrift::protocol::TProtocol* o
   xfer += oprot->writeFieldBegin("fullTableNames", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->fullTableNames.size()));
-    std::vector<std::string> ::const_iterator _iter688;
-    for (_iter688 = this->fullTableNames.begin(); _iter688 != this->fullTableNames.end(); ++_iter688)
+    std::vector<std::string> ::const_iterator _iter696;
+    for (_iter696 = this->fullTableNames.begin(); _iter696 != this->fullTableNames.end(); ++_iter696)
     {
-      xfer += oprot->writeString((*_iter688));
+      xfer += oprot->writeString((*_iter696));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17024,13 +17290,13 @@ void swap(GetValidWriteIdsRequest &a, GetValidWriteIdsRequest &b) {
   swap(a.validTxnList, b.validTxnList);
 }
 
-GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other689) {
-  fullTableNames = other689.fullTableNames;
-  validTxnList = other689.validTxnList;
+GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other697) {
+  fullTableNames = other697.fullTableNames;
+  validTxnList = other697.validTxnList;
 }
-GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other690) {
-  fullTableNames = other690.fullTableNames;
-  validTxnList = other690.validTxnList;
+GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other698) {
+  fullTableNames = other698.fullTableNames;
+  validTxnList = other698.validTxnList;
   return *this;
 }
 void GetValidWriteIdsRequest::printTo(std::ostream& out) const {
@@ -17112,14 +17378,14 @@ uint32_t TableValidWriteIds::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->invalidWriteIds.clear();
-            uint32_t _size691;
-            ::apache::thrift::protocol::TType _etype694;
-            xfer += iprot->readListBegin(_etype694, _size691);
-            this->invalidWriteIds.resize(_size691);
-            uint32_t _i695;
-            for (_i695 = 0; _i695 < _size691; ++_i695)
+            uint32_t _size699;
+            ::apache::thrift::protocol::TType _etype702;
+            xfer += iprot->readListBegin(_etype702, _size699);
+            this->invalidWriteIds.resize(_size699);
+            uint32_t _i703;
+            for (_i703 = 0; _i703 < _size699; ++_i703)
             {
-              xfer += iprot->readI64(this->invalidWriteIds[_i695]);
+              xfer += iprot->readI64(this->invalidWriteIds[_i703]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17180,10 +17446,10 @@ uint32_t TableValidWriteIds::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("invalidWriteIds", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->invalidWriteIds.size()));
-    std::vector<int64_t> ::const_iterator _iter696;
-    for (_iter696 = this->invalidWriteIds.begin(); _iter696 != this->invalidWriteIds.end(); ++_iter696)
+    std::vector<int64_t> ::const_iterator _iter704;
+    for (_iter704 = this->invalidWriteIds.begin(); _iter704 != this->invalidWriteIds.end(); ++_iter704)
     {
-      xfer += oprot->writeI64((*_iter696));
+      xfer += oprot->writeI64((*_iter704));
     }
     xfer += oprot->writeListEnd();
   }
@@ -17213,21 +17479,21 @@ void swap(TableValidWriteIds &a, TableValidWriteIds &b) {
   swap(a.__isset, b.__isset);
 }
 
-TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other697) {
-  fullTableName = other697.fullTableName;
-  writeIdHighWaterMark = other697.writeIdHighWaterMark;
-  invalidWriteIds = other697.invalidWriteIds;
-  minOpenWriteId = other697.minOpenWriteId;
-  abortedBits = other697.abortedBits;
-  __isset = other697.__isset;
-}
-TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other698) {
-  fullTableName = other698.fullTableName;
-  writeIdHighWaterMark = other698.writeIdHighWaterMark;
-  invalidWriteIds = other698.invalidWriteIds;
-  minOpenWriteId = other698.minOpenWriteId;
-  abortedBits = other698.abortedBits;
-  __isset = other698.__isset;
+TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other705) {
+  fullTableName = other705.fullTableName;
+  writeIdHighWaterMark = other705.writeIdHighWaterMark;
+  invalidWriteIds = other705.invalidWriteIds;
+  minOpenWriteId = other705.minOpenWriteId;
+  abortedBits = other705.abortedBits;
+  __isset = other705.__isset;
+}
+TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other706) {
+  fullTableName = other706.fullTableName;
+  writeIdHighWaterMark = other706.writeIdHighWaterMark;
+  invalidWriteIds = other706.invalidWriteIds;
+  minOpenWriteId = other706.minOpenWriteId;
+  abortedBits = other706.abortedBits;
+  __isset = other706.__isset;
   return *this;
 }
 void TableValidWriteIds::printTo(std::ostream& out) const {
@@ -17276,14 +17542,14 @@ uint32_t GetValidWriteIdsResponse::read(::apache::thrift::protocol::TProtocol* i
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tblValidWriteIds.clear();
-            uint32_t _size699;
-            ::apache::thrift::protocol::TType _etype702;
-            xfer += iprot->readListBegin(_etype702, _size699);
-            this->tblValidWriteIds.resize(_size699);
-            uint32_t _i703;
-            for (_i703 = 0; _i703 < _size699; ++_i703)
+            uint32_t _size707;
+            ::apache::thrift::protocol::TType _etype710;
+            xfer += iprot->readListBegin(_etype710, _size707);
+            this->tblValidWriteIds.resize(_size707);
+            uint32_t _i711;
+            for (_i711 = 0; _i711 < _size707; ++_i711)
             {
-              xfer += this->tblValidWriteIds[_i703].read(iprot);
+              xfer += this->tblValidWriteIds[_i711].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17314,10 +17580,10 @@ uint32_t GetValidWriteIdsResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("tblValidWriteIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tblValidWriteIds.size()));
-    std::vector<TableValidWriteIds> ::const_iterator _iter704;
-    for (_iter704 = this->tblValidWriteIds.begin(); _iter704 != this->tblValidWriteIds.end(); ++_iter704)
+    std::vector<TableValidWriteIds> ::const_iterator _iter712;
+    for (_iter712 = this->tblValidWriteIds.begin(); _iter712 != this->tblValidWriteIds.end(); ++_iter712)
     {
-      xfer += (*_iter704).write(oprot);
+      xfer += (*_iter712).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17333,11 +17599,11 @@ void swap(GetValidWriteIdsResponse &a, GetValidWriteIdsResponse &b) {
   swap(a.tblValidWriteIds, b.tblValidWriteIds);
 }
 
-GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other705) {
-  tblValidWriteIds = other705.tblValidWriteIds;
+GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other713) {
+  tblValidWriteIds = other713.tblValidWriteIds;
 }
-GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other706) {
-  tblValidWriteIds = other706.tblValidWriteIds;
+GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other714) {
+  tblValidWriteIds = other714.tblValidWriteIds;
   return *this;
 }
 void GetValidWriteIdsResponse::printTo(std::ostream& out) const {
@@ -17418,14 +17684,14 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnIds.clear();
-            uint32_t _size707;
-            ::apache::thrift::protocol::TType _etype710;
-            xfer += iprot->readListBegin(_etype710, _size707);
-            this->txnIds.resize(_size707);
-            uint32_t _i711;
-            for (_i711 = 0; _i711 < _size707; ++_i711)
+            uint32_t _size715;
+            ::apache::thrift::protocol::TType _etype718;
+            xfer += iprot->readListBegin(_etype718, _size715);
+            this->txnIds.resize(_size715);
+            uint32_t _i719;
+            for (_i719 = 0; _i719 < _size715; ++_i719)
             {
-              xfer += iprot->readI64(this->txnIds[_i711]);
+              xfer += iprot->readI64(this->txnIds[_i719]);
             }
             xfer += iprot->readListEnd();
           }
@@ -17446,14 +17712,14 @@ uint32_t AllocateTableWriteIdsRequest::read(::apache::thrift::protocol::TProtoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->srcTxnToWriteIdList.clear();
-            uint32_t _size712;
-            ::apache::thrift::protocol::TType _etype715;
-            xfer += iprot->readListBegin(_etype715, _size712);
-            this->srcTxnToWriteIdList.resize(_size712);
-            uint32_t _i716;
-            for (_i716 = 0; _i716 < _size712; ++_i716)
+            uint32_t _size720;
+            ::apache::thrift::protocol::TType _etype723;
+            xfer += iprot->readListBegin(_etype723, _size720);
+            this->srcTxnToWriteIdList.resize(_size720);
+            uint32_t _i724;
+            for (_i724 = 0; _i724 < _size720; ++_i724)
             {
-              xfer += this->srcTxnToWriteIdList[_i716].read(iprot);
+              xfer += this->srcTxnToWriteIdList[_i724].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17495,10 +17761,10 @@ uint32_t AllocateTableWriteIdsRequest::write(::apache::thrift::protocol::TProtoc
     xfer += oprot->writeFieldBegin("txnIds", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txnIds.size()));
-      std::vector<int64_t> ::const_iterator _iter717;
-      for (_iter717 = this->txnIds.begin(); _iter717 != this->txnIds.end(); ++_iter717)
+      std::vector<int64_t> ::const_iterator _iter725;
+      for (_iter725 = this->txnIds.begin(); _iter725 != this->txnIds.end(); ++_iter725)
       {
-        xfer += oprot->writeI64((*_iter717));
+        xfer += oprot->writeI64((*_iter725));
       }
       xfer += oprot->writeListEnd();
     }
@@ -17513,10 +17779,10 @@ uint32_t AllocateTableWriteIdsRequest::write(::apache::thrift::protocol::TProtoc
     xfer += oprot->writeFieldBegin("srcTxnToWriteIdList", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->srcTxnToWriteIdList.size()));
-      std::vector<TxnToWriteId> ::const_iterator _iter718;
-      for (_iter718 = this->srcTxnToWriteIdList.begin(); _iter718 != this->srcTxnToWriteIdList.end(); ++_iter718)
+      std::vector<TxnToWriteId> ::const_iterator _iter726;
+      for (_iter726 = this->srcTxnToWriteIdList.begin(); _iter726 != this->srcTxnToWriteIdList.end(); ++_iter726)
       {
-        xfer += (*_iter718).write(oprot);
+        xfer += (*_iter726).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -17537,21 +17803,21 @@ void swap(AllocateTableWriteIdsRequest &a, AllocateTableWriteIdsRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other719) {
-  dbName = other719.dbName;
-  tableName = other719.tableName;
-  txnIds = other719.txnIds;
-  replPolicy = other719.replPolicy;
-  srcTxnToWriteIdList = other719.srcTxnToWriteIdList;
-  __isset = other719.__isset;
-}
-AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other720) {
-  dbName = other720.dbName;
-  tableName = other720.tableName;
-  txnIds = other720.txnIds;
-  replPolicy = other720.replPolicy;
-  srcTxnToWriteIdList = other720.srcTxnToWriteIdList;
-  __isset = other720.__isset;
+AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other727) {
+  dbName = other727.dbName;
+  tableName = other727.tableName;
+  txnIds = other727.txnIds;
+  replPolicy = other727.replPolicy;
+  srcTxnToWriteIdList = other727.srcTxnToWriteIdList;
+  __isset = other727.__isset;
+}
+AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other728) {
+  dbName = other728.dbName;
+  tableName = other728.tableName;
+  txnIds = other728.txnIds;
+  replPolicy = other728.replPolicy;
+  srcTxnToWriteIdList = other728.srcTxnToWriteIdList;
+  __isset = other728.__isset;
   return *this;
 }
 void AllocateTableWriteIdsRequest::printTo(std::ostream& out) const {
@@ -17657,13 +17923,13 @@ void swap(TxnToWriteId &a, TxnToWriteId &b) {
   swap(a.writeId, b.writeId);
 }
 
-TxnToWriteId::TxnToWriteId(const TxnToWriteId& other721) {
-  txnId = other721.txnId;
-  writeId = other721.writeId;
+TxnToWriteId::TxnToWriteId(const TxnToWriteId& other729) {
+  txnId = other729.txnId;
+  writeId = other729.writeId;
 }
-TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other722) {
-  txnId = other722.txnId;
-  writeId = other722.writeId;
+TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other730) {
+  txnId = other730.txnId;
+  writeId = other730.writeId;
   return *this;
 }
 void TxnToWriteId::printTo(std::ostream& out) const {
@@ -17709,14 +17975,14 @@ uint32_t AllocateTableWriteIdsResponse::read(::apache::thrift::protocol::TProtoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->txnToWriteIds.clear();
-            uint32_t _size723;
-            ::apache::thrift::protocol::TType _etype726;
-            xfer += iprot->readListBegin(_etype726, _size723);
-            this->txnToWriteIds.resize(_size723);
-            uint32_t _i727;
-            for (_i727 = 0; _i727 < _size723; ++_i727)
+            uint32_t _size731;
+            ::apache::thrift::protocol::TType _etype734;
+            xfer += iprot->readListBegin(_etype734, _size731);
+            this->txnToWriteIds.resize(_size731);
+            uint32_t _i735;
+            for (_i735 = 0; _i735 < _size731; ++_i735)
             {
-              xfer += this->txnToWriteIds[_i727].read(iprot);
+              xfer += this->txnToWriteIds[_i735].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -17747,10 +18013,10 @@ uint32_t AllocateTableWriteIdsResponse::write(::apache::thrift::protocol::TProto
   xfer += oprot->writeFieldBegin("txnToWriteIds", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->txnToWriteIds.size()));
-    std::vector<TxnToWriteId> ::const_iterator _iter728;
-    for (_iter728 = this->txnToWriteIds.begin(); _iter728 != this->txnToWriteIds.end(); ++_iter728)
+    std::vector<TxnToWriteId> ::const_iterator _iter736;
+    for (_iter736 = this->txnToWriteIds.begin(); _iter736 != this->txnToWriteIds.end(); ++_iter736)
     {
-      xfer += (*_iter728).write(oprot);
+      xfer += (*_iter736).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -17766,11 +18032,11 @@ void swap(AllocateTableWriteIdsResponse &a, AllocateTableWriteIdsResponse &b) {
   swap(a.txnToWriteIds, b.txnToWriteIds);
 }
 
-AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other729) {
-  txnToWriteIds = other729.txnToWriteIds;
+AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other737) {
+  txnToWriteIds = other737.txnToWriteIds;
 }
-AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other730) {
-  txnToWriteIds = other730.txnToWriteIds;
+AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other738) {
+  txnToWriteIds = other738.txnToWriteIds;
   return *this;
 }
 void AllocateTableWriteIdsResponse::printTo(std::ostream& out) const {
@@ -17848,9 +18114,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast731;
-          xfer += iprot->readI32(ecast731);
-          this->type = (LockType::type)ecast731;
+          int32_t ecast739;
+          xfer += iprot->readI32(ecast739);
+          this->type = (LockType::type)ecast739;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17858,9 +18124,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast732;
-          xfer += iprot->readI32(ecast732);
-          this->level = (LockLevel::type)ecast732;
+          int32_t ecast740;
+          xfer += iprot->readI32(ecast740);
+          this->level = (LockLevel::type)ecast740;
           isset_level = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17892,9 +18158,9 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast733;
-          xfer += iprot->readI32(ecast733);
-          this->operationType = (DataOperationType::type)ecast733;
+          int32_t ecast741;
+          xfer += iprot->readI32(ecast741);
+          this->operationType = (DataOperationType::type)ecast741;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -17994,27 +18260,27 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockComponent::LockComponent(const LockComponent& other734) {
-  type = other734.type;
-  level = other734.level;
-  dbname = other734.dbname;
-  tablename = other734.tablename;
-  partitionname = other734.partitionname;
-  operationType = other734.operationType;
-  isTransactional = other734.isTransactional;
-  isDynamicPartitionWrite = other734.isDynamicPartitionWrite;
-  __isset = other734.__isset;
-}
-LockComponent& LockComponent::operator=(const LockComponent& other735) {
-  type = other735.type;
-  level = other735.level;
-  dbname = other735.dbname;
-  tablename = other735.tablename;
-  partitionname = other735.partitionname;
-  operationType = other735.operationType;
-  isTransactional = other735.isTransactional;
-  isDynamicPartitionWrite = other735.isDynamicPartitionWrite;
-  __isset = other735.__isset;
+LockComponent::LockComponent(const LockComponent& other742) {
+  type = other742.type;
+  level = other742.level;
+  dbname = other742.dbname;
+  tablename = other742.tablename;
+  partitionname = other742.partitionname;
+  operationType = other742.operationType;
+  isTransactional = other742.isTransactional;
+  isDynamicPartitionWrite = other742.isDynamicPartitionWrite;
+  __isset = other742.__isset;
+}
+LockComponent& LockComponent::operator=(const LockComponent& other743) {
+  type = other743.type;
+  level = other743.level;
+  dbname = other743.dbname;
+  tablename = other743.tablename;
+  partitionname = other743.partitionname;
+  operationType = other743.operationType;
+  isTransactional = other743.isTransactional;
+  isDynamicPartitionWrite = other743.isDynamicPartitionWrite;
+  __isset = other743.__isset;
   return *this;
 }
 void LockComponent::printTo(std::ostream& out) const {
@@ -18086,14 +18352,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->component.clear();
-            uint32_t _size736;
-            ::apache::thrift::protocol::TType _etype739;
-            xfer += iprot->readListBegin(_etype739, _size736);
-            this->component.resize(_size736);
-            uint32_t _i740;
-            for (_i740 = 0; _i740 < _size736; ++_i740)
+            uint32_t _size744;
+            ::apache::thrift::protocol::TType _etype747;
+            xfer += iprot->readListBegin(_etype747, _size744);
+            this->component.resize(_size744);
+            uint32_t _i748;
+            for (_i748 = 0; _i748 < _size744; ++_i748)
             {
-              xfer += this->component[_i740].read(iprot);
+              xfer += this->component[_i748].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -18160,10 +18426,10 @@ uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->component.size()));
-    std::vector<LockComponent> ::const_iterator _iter741;
-    for (_iter741 = this->component.begin(); _iter741 != this->component.end(); ++_iter741)
+    std::vector<LockComponent> ::const_iterator _iter749;
+    for (_iter749 = this->component.begin(); _iter749 != this->component.end(); ++_iter749)
     {
-      xfer += (*_iter741).write(oprot);
+      xfer += (*_iter749).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -18202,21 +18468,21 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockRequest::LockRequest(const LockRequest& other742) {
-  component = other742.component;
-  txnid = other742.txnid;
-  user = other742.user;
-  hostname = other742.hostname;
-  agentInfo = other742.agentInfo;
-  __isset = other742.__isset;
-}
-LockRequest& LockRequest::operator=(const LockRequest& other743) {
-  component = other743.component;
-  txnid = other743.txnid;
-  user = other743.user;
-  hostname = other743.hostname;
-  agentInfo = other743.agentInfo;
-  __isset = other743.__isset;
+LockRequest::LockRequest(const LockRequest& other750) {
+  component = other750.component;
+  txnid = other750.txnid;
+  user = other750.user;
+  hostname = other750.hostname;
+  agentInfo = other750.agentInfo;
+  __isset = other750.__isset;
+}
+LockRequest& LockRequest::operator=(const LockRequest& other751) {
+  component = other751.component;
+  txnid = other751.txnid;
+  user = other751.user;
+  hostname = other751.hostname;
+  agentInfo = other751.agentInfo;
+  __isset = other751.__isset;
   return *this;
 }
 void LockRequest::printTo(std::ostream& out) const {
@@ -18276,9 +18542,9 @@ uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast744;
-          xfer += iprot->readI32(ecast744);
-          this->state = (LockState::type)ecast744;
+          int32_t ecast752;
+          xfer += iprot->readI32(ecast752);
+          this->state = (LockState::type)ecast752;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -18324,13 +18590,13 @@ void swap(LockResponse &a, LockResponse &b) {
   swap(a.state, b.state);
 }
 
-LockResponse::LockResponse(const LockResponse& other745) {
-  lockid = other745.lockid;
-  state = other745.state;
+LockResponse::LockResponse(const LockResponse& other753) {
+  lockid = other753.lockid;
+  state = other753.state;
 }
-LockResponse& LockResponse::operator=(const LockResponse& other746) {
-  lockid = other746.lockid;
-  state = other746.state;
+LockResponse& LockResponse::operator=(const LockResponse& other754) {
+  lockid = other754.lockid;
+  state = other754.state;
   return *this;
 }
 void LockResponse::printTo(std::ostream& out) const {
@@ -18452,17 +18718,17 @@ void swap(CheckLockRequest &a, CheckLockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckLockRequest::CheckLockRequest(const CheckLockRequest& other747) {
-  lockid = other747.lockid;
-  txnid = other747.txnid;
-  elapsed_ms = other747.elapsed_ms;
-  __isset = other747.__isset;
+CheckLockRequest::CheckLockRequest(const CheckLockRequest& other755) {
+  lockid = other755.lockid;
+  txnid = other755.txnid;
+  elapsed_ms = other755.elapsed_ms;
+  __isset = other755.__isset;
 }
-CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other748) {
-  lockid = other748.lockid;
-  txnid = other748.txnid;
-  elapsed_ms = other748.elapsed_ms;
-  __isset = other748.__isset;
+CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other756) {
+  lockid = other756.lockid;
+  txnid = other756.txnid;
+  elapsed_ms = other756.elapsed_ms;
+  __isset = other756.__isset;
   return *this;
 }
 void CheckLockRequest::printTo(std::ostream& out) const {
@@ -18546,11 +18812,11 @@ void swap(UnlockRequest &a, UnlockRequest &b) {
   swap(a.lockid, b.lockid);
 }
 
-UnlockRequest::UnlockRequest(const UnlockRequest& other749) {
-  lockid = other749.lockid;
+UnlockRequest::UnlockRequest(const UnlockRequest& other757) {
+  lockid = other757.lockid;
 }
-UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other750) {
-  lockid = other750.lockid;
+UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other758) {
+  lockid = other758.lockid;
   return *this;
 }
 void UnlockRequest::printTo(std::ostream& out) const {
@@ -18689,19 +18955,19 @@ void swap(ShowLocksRequest &a, ShowLocksRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other751) {
-  dbname = other751.dbname;
-  tablename = other751.tablename;
-  partname = other751.partname;
-  isExtended = other751.isExtended;
-  __isset = other751.__isset;
+ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other759) {
+  dbname = other759.dbname;
+  tablename = other759.tablename;
+  partname = other759.partname;
+  isExtended = other759.isExtended;
+  __isset = other759.__isset;
 }
-ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other752) {
-  dbname = other752.dbname;
-  tablename = other752.tablename;
-  partname = other752.partname;
-  isExtended = other752.isExtended;
-  __isset = other752.__isset;
+ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other760) {
+  dbname = other760.dbname;
+  tablename = other760.tablename;
+  partname = other760.partname;
+  isExtended = other760.isExtended;
+  __isset = other760.__isset;
   return *this;
 }
 void ShowLocksRequest::printTo(std::ostream& out) const {
@@ -18854,9 +19120,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast753;
-          xfer += iprot->readI32(ecast753);
-          this->state = (LockState::type)ecast753;
+          int32_t ecast761;
+          xfer += iprot->readI32(ecast761);
+          this->state = (LockState::type)ecast761;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -18864,9 +19130,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast754;
-          xfer += iprot->readI32(ecast754);
-          this->type = (LockType::type)ecast754;
+          int32_t ecast762;
+          xfer += iprot->readI32(ecast762);
+          this->type = (LockType::type)ecast762;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -19082,43 +19348,43 @@ void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other755) {
-  lockid = other755.lockid;
-  dbname = other755.dbname;
-  tablename = other755.tablename;
-  partname = other755.partname;
-  state = other755.state;
-  type = other755.type;
-  txnid = other755.txnid;
-  lastheartbeat = other755.lastheartbeat;
-  acquiredat = other755.acquiredat;
-  user = other755.user;
-  hostname = other755.hostname;
-  heartbeatCount = other755.heartbeatCount;
-  agentInfo = other755.agentInfo;
-  blockedByExtId = other755.blockedByExtId;
-  blockedByIntId = other755.blockedByIntId;
-  lockIdInternal = other755.lockIdInternal;
-  __isset = other755.__isset;
+ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other763) {
+  lockid = other763.lockid;
+  dbname = other763.dbname;
+  tablename = other763.tablename;
+  partname = other763.partname;
+  state = other763.state;
+  type = other763.type;
+  txnid = other763.txnid;
+  lastheartbeat = other763.lastheartbeat;
+  acquiredat = other763.acquiredat;
+  user = other763.user;
+  hostname = other763.hostname;
+  heartbeatCount = other763.heartbeatCount;
+  agentInfo = other763.agentInfo;
+  blockedByExtId = other763.blockedByExtId;
+  blockedByIntId = other763.blockedByIntId;
+  lockIdInternal = other763.lockIdInternal;
+  __isset = other763.__isset;
 }
-ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other756) {
-  lockid = other756.lockid;
-  dbname = other756.dbname;
-  tablename = other756.tablename;
-  partname = other756.partname;
-  state = other756.state;
-  type = other756.type;
-  txnid = other756.txnid;
-  lastheartbeat = other756.lastheartbeat;
-  acquiredat = other756.acquiredat;
-  user = other756.user;
-  hostname = other756.hostname;
-  heartbeatCount = other756.heartbeatCount;
-  agentInfo = other756.agentInfo;
-  blockedByExtId = other756.blockedByExtId;
-  blockedByIntId = other756.blockedByIntId;
-  lockIdInternal = other756.lockIdInternal;
-  __isset = other756.__isset;
+ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other764) {
+  lockid = other764.lockid;
+  dbname = other764.dbname;
+  tablename = other764.tablename;
+  partname = other764.partname;
+  state = other764.state;
+  type = other764.type;
+  txnid = other764.txnid;
+  lastheartbeat = other764.lastheartbeat;
+  acquiredat = other764.acquiredat;
+  user = other764.user;
+  hostname = other764.hostname;
+  heartbeatCount = other764.heartbeatCount;
+  agentInfo = other764.agentInfo;
+  blockedByExtId = other764.blockedByExtId;
+  blockedByIntId = other764.blockedByIntId;
+  lockIdInternal = other764.lockIdInternal;
+  __isset = other764.__isset;
   return *this;
 }
 void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@ -19177,14 +19443,14 @@ uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->locks.clear();
-            uint32_t _size757;
-            ::apache::thrift::protocol::TType _etype760;
-            xfer += iprot->readListBegin(_etype760, _size757);
-            this->locks.resize(_size757);
-            uint32_t _i761;
-            for (_i761 = 0; _i761 < _size757; ++_i761)
+            uint32_t _size765;
+            ::apache::thrift::protocol::TType _etype768;
+            xfer += iprot->readListBegin(_etype768, _size765);
+            this->locks.resize(_size765);
+            uint32_t _i769;
+            for (_i769 = 0; _i769 < _size765; ++_i769)
             {
-              xfer += this->locks[_i761].read(iprot);
+              xfer += this->locks[_i769].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -19213,10 +19479,10 @@ uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->locks.size()));
-    std::vector<ShowLocksResponseElement> ::const_iterator _iter762;
-    for (_iter762 = this->locks.begin(); _iter762 != this->locks.end(); ++_iter762)
+    std::vector<ShowLocksResponseElement> ::const_iterator _iter770;
+    for (_iter770 = this->locks.begin(); _iter770 != this->locks.end(); ++_iter770)
     {
-      xfer += (*_iter762).write(oprot);
+      xfer += (*_iter770).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -19233,13 +19499,13 @@ void swap(ShowLocksResponse &a, ShowLocksResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other763) {
-  locks = other763.locks;
-  __isset = other763.__isset;
+ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other771) {
+  locks = other771.locks;
+  __isset = other771.__isset;
 }
-ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other764) {
-  locks = other764.locks;
-  __isset = other764.__isset;
+ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other772) {
+  locks = other772.locks;
+  __isset = other772.__isset;
   return *this;
 }
 void ShowLocksResponse::printTo(std::ostream& out) const {
@@ -19340,15 +19606,15 @@ void swap(HeartbeatRequest &a, HeartbeatRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other765) {
-  lockid = other765.lockid;
-  txnid = other765.txnid;
-  __isset = other765.__isset;
+HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other773) {
+  lockid = other773.lockid;
+  txnid = other773.txnid;
+  __isset = other773.__isset;
 }
-HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other766) {
-  lockid = other766.lockid;
-  txnid = other766.txnid;
-  __isset = other766.__isset;
+HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other774) {
+  lockid = other774.lockid;
+  txnid = other774.txnid;
+  __isset = other774.__isset;
   return *this;
 }
 void HeartbeatRequest::printTo(std::ostream& out) const {
@@ -19451,13 +19717,13 @@ void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) {
   swap(a.max, b.max);
 }
 
-HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other767) {
-  min = other767.min;
-  max = other767.max;
+HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other775) {
+  min = other775.min;
+  max = other775.max;
 }
-HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other768) {
-  min = other768.min;
-  max = other768.max;
+HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other776) {
+  min = other776.min;
+  max = other776.max;
   return *this;
 }
 void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@ -19508,15 +19774,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->aborted.clear();
-            uint32_t _size769;
-            ::apache::thrift::protocol::TType _etype772;
-            xfer += iprot->readSetBegin(_etype772, _size769);
-            uint32_t _i773;
-            for (_i773 = 0; _i773 < _size769; ++_i773)
+            uint32_t _size777;
+            ::apache::thrift::protocol::TType _etype780;
+            xfer += iprot->readSetBegin(_etype780, _size777);
+            uint32_t _i781;
+            for (_i781 = 0; _i781 < _size777; ++_i781)
             {
-              int64_t _elem774;
-              xfer += iprot->readI64(_elem774);
-              this->aborted.insert(_elem774);
+              int64_t _elem782;
+              xfer += iprot->readI64(_elem782);
+              this->aborted.insert(_elem782);
             }
             xfer += iprot->readSetEnd();
           }
@@ -19529,15 +19795,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->nosuch.clear();
-            uint32_t _size775;
-            ::apache::thrift::protocol::TType _etype778;
-            xfer += iprot->readSetBegin(_etype778, _size775);
-            uint32_t _i779;
-            for (_i779 = 0; _i779 < _size775; ++_i779)
+            uint32_t _size783;
+            ::apache::thrift::protocol::TType _etype786;
+            xfer += iprot->readSetBegin(_etype786, _size783);
+            uint32_t _i787;
+            for (_i787 = 0; _i787 < _size783; ++_i787)
             {
-              int64_t _elem780;
-              xfer += iprot->readI64(_elem780);
-              this->nosuch.insert(_elem780);
+              int64_t _elem788;
+              xfer += iprot->readI64(_elem788);
+              this->nosuch.insert(_elem788);
             }
             xfer += iprot->readSetEnd();
           }
@@ -19570,10 +19836,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->aborted.size()));
-    std::set<int64_t> ::const_iterator _iter781;
-    for (_iter781 = this->aborted.begin(); _iter781 != this->aborted.end(); ++_iter781)
+    std::set<int64_t> ::const_iterator _iter789;
+    for (_iter789 = this->aborted.begin(); _iter789 != this->aborted.end(); ++_iter789)
     {
-      xfer += oprot->writeI64((*_iter781));
+      xfer += oprot->writeI64((*_iter789));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -19582,10 +19848,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->nosuch.size()));
-    std::set<int64_t> ::const_iterator _iter782;
-    for (_iter782 = this->nosuch.begin(); _iter782 != this->nosuch.end(); ++_iter782)
+    std::set<int64_t> ::const_iterator _iter790;
+    for (_iter790 = this->nosuch.begin(); _iter790 != this->nosuch.end(); ++_iter790)
     {
-      xfer += oprot->writeI64((*_iter782));
+      xfer += oprot->writeI64((*_iter790));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -19602,13 +19868,13 @@ void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) {
   swap(a.nosuch, b.nosuch);
 }
 
-HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other783) {
-  aborted = other783.aborted;
-  nosuch = other783.nosuch;
+HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other791) {
+  aborted = other791.aborted;
+  nosuch = other791.nosuch;
 }
-HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other784) {
-  aborted = other784.aborted;
-  nosuch = other784.nosuch;
+HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other792) {
+  aborted = other792.aborted;
+  nosuch = other792.nosuch;
   return *this;
 }
 void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@ -19701,9 +19967,9 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast785;
-          xfer += iprot->readI32(ecast785);
-          this->type = (CompactionType::type)ecast785;
+          int32_t ecast793;
+          xfer += iprot->readI32(ecast793);
+          this->type = (CompactionType::type)ecast793;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -19721,17 +19987,17 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->properties.clear();
-            uint32_t _size786;
-            ::apache::thrift::protocol::TType _ktype787;
-            ::apache::thrift::protocol::TType _vtype788;
-            xfer += iprot->readMapBegin(_ktype787, _vtype788, _size786);
-            uint32_t _i790;
-            for (_i790 = 0; _i790 < _size786; ++_i790)
+            uint32_t _size794;
+            ::apache::thrift::protocol::TType _ktype795;
+            ::apache::thrift::protocol::TType _vtype796;
+            xfer += iprot->readMapBegin(_ktype795, _vtype796, _size794);
+            uint32_t _i798;
+            for (_i798 = 0; _i798 < _size794; ++_i798)
             {
-              std::string _key791;
-              xfer += iprot->readString(_key791);
-              std::string& _val792 = this->properties[_key791];
-              xfer += iprot->readString(_val792);
+              std::string _key799;
+              xfer += iprot->readString(_key799);
+              std::string& _val800 = this->properties[_key799];
+              xfer += iprot->readString(_val800);
             }
             xfer += iprot->readMapEnd();
           }
@@ -19789,11 +20055,11 @@ uint32_t CompactionRequest::write(::apache::thrift::protocol::TProtocol* oprot)
     xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
-      std::map<std::string, std::string> ::const_iterator _iter793;
-      for (_iter793 = this->properties.begin(); _iter793 != this->properties.end(); ++_iter793)
+      std::map<std::string, std::string> ::const_iterator _iter801;
+      for (_iter801 = this->properties.begin(); _iter801 != this->properties.end(); ++_iter801)
       {
-        xfer += oprot->writeString(_iter793->first);
-        xfer += oprot->writeString(_iter793->second);
+        xfer += oprot->writeString(_iter801->first);
+        xfer += oprot->writeString(_iter801->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -19815,23 +20081,23 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other794) {
-  dbname = other794.dbname;
-  tablename = other794.tablename;
-  partitionname = other794.partitionname;
-  type = other794.type;
-  runas = other794.runas;
-  properties = other794.properties;
-  __isset = other794.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other795) {
-  dbname = other795.dbname;
-  tablename = other795.tablename;
-  partitionname = other795.partitionname;
-  type = other795.type;
-  runas = other795.runas;
-  properties = other795.properties;
-  __isset = other795.__isset;
+CompactionRequest::CompactionRequest(const CompactionRequest& other802) {
+  dbname = other802.dbname;
+  tablename = other802.tablename;
+  partitionname = other802.partitionname;
+  type = other802.type;
+  runas = other802.runas;
+  properties = other802.properties;
+  __isset = other802.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other803) {
+  dbname = other803.dbname;
+  tablename = other803.tablename;
+  partitionname = other803.partitionname;
+  type = other803.type;
+  runas = other803.runas;
+  properties = other803.properties;
+  __isset = other803.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -19958,15 +20224,15 @@ void swap(CompactionResponse &a, CompactionResponse &b) {
   swap(a.accepted, b.accepted);
 }
 
-CompactionResponse::CompactionResponse(const CompactionResponse& other796) {
-  id = other796.id;
-  state = other796.state;
-  accepted = other796.accepted;
+CompactionResponse::CompactionResponse(const CompactionResponse& other804) {
+  id = other804.id;
+  state = other804.state;
+  accepted = other804.accepted;
 }
-CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other797) {
-  id = other797.id;
-  state = other797.state;
-  accepted = other797.accepted;
+CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other805) {
+  id = other805.id;
+  state = other805.state;
+  accepted = other805.accepted;
   return *this;
 }
 void CompactionResponse::printTo(std::ostream& out) const {
@@ -20027,11 +20293,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other798) {
-  (void) other798;
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other806) {
+  (void) other806;
 }
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other799) {
-  (void) other799;
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other807) {
+  (void) other807;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -20157,9 +20423,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast800;
-          xfer += iprot->readI32(ecast800);
-          this->type = (CompactionType::type)ecast800;
+          int32_t ecast808;
+          xfer += iprot->readI32(ecast808);
+          this->type = (CompactionType::type)ecast808;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20346,37 +20612,37 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other801) {
-  dbname = other801.dbname;
-  tablename = other801.tablename;
-  partitionname = other801.partitionname;
-  type = other801.type;
-  state = other801.state;
-  workerid = other801.workerid;
-  start = other801.start;
-  runAs = other801.runAs;
-  hightestTxnId = other801.hightestTxnId;
-  metaInfo = other801.metaInfo;
-  endTime = other801.endTime;
-  hadoopJobId = other801.hadoopJobId;
-  id = other801.id;
-  __isset = other801.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other802) {
-  dbname = other802.dbname;
-  tablename = other802.tablename;
-  partitionname = other802.partitionname;
-  type = other802.type;
-  state = other802.state;
-  workerid = other802.workerid;
-  start = other802.start;
-  runAs = other802.runAs;
-  hightestTxnId = other802.hightestTxnId;
-  metaInfo = other802.metaInfo;
-  endTime = other802.endTime;
-  hadoopJobId = other802.hadoopJobId;
-  id = other802.id;
-  __isset = other802.__isset;
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other809) {
+  dbname = other809.dbname;
+  tablename = other809.tablename;
+  partitionname = other809.partitionname;
+  type = other809.type;
+  state = other809.state;
+  workerid = other809.workerid;
+  start = other809.start;
+  runAs = other809.runAs;
+  hightestTxnId = other809.hightestTxnId;
+  metaInfo = other809.metaInfo;
+  endTime = other809.endTime;
+  hadoopJobId = other809.hadoopJobId;
+  id = other809.id;
+  __isset = other809.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other810) {
+  dbname = other810.dbname;
+  tablename = other810.tablename;
+  partitionname = other810.partitionname;
+  type = other810.type;
+  state = other810.state;
+  workerid = other810.workerid;
+  start = other810.start;
+  runAs = other810.runAs;
+  hightestTxnId = other810.hightestTxnId;
+  metaInfo = other810.metaInfo;
+  endTime = other810.endTime;
+  hadoopJobId = other810.hadoopJobId;
+  id = other810.id;
+  __isset = other810.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -20433,14 +20699,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size803;
-            ::apache::thrift::protocol::TType _etype806;
-            xfer += iprot->readListBegin(_etype806, _size803);
-            this->compacts.resize(_size803);
-            uint32_t _i807;
-            for (_i807 = 0; _i807 < _size803; ++_i807)
+            uint32_t _size811;
+            ::apache::thrift::protocol::TType _etype814;
+            xfer += iprot->readListBegin(_etype814, _size811);
+            this->compacts.resize(_size811);
+            uint32_t _i815;
+            for (_i815 = 0; _i815 < _size811; ++_i815)
             {
-              xfer += this->compacts[_i807].read(iprot);
+              xfer += this->compacts[_i815].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -20471,10 +20737,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter808;
-    for (_iter808 = this->compacts.begin(); _iter808 != this->compacts.end(); ++_iter808)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter816;
+    for (_iter816 = this->compacts.begin(); _iter816 != this->compacts.end(); ++_iter816)
     {
-      xfer += (*_iter808).write(oprot);
+      xfer += (*_iter816).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -20490,11 +20756,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other809) {
-  compacts = other809.compacts;
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other817) {
+  compacts = other817.compacts;
 }
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other810) {
-  compacts = other810.compacts;
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other818) {
+  compacts = other818.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -20596,14 +20862,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size811;
-            ::apache::thrift::protocol::TType _etype814;
-            xfer += iprot->readListBegin(_etype814, _size811);
-            this->partitionnames.resize(_size811);
-            uint32_t _i815;
-            for (_i815 = 0; _i815 < _size811; ++_i815)
+            uint32_t _size819;
+            ::apache::thrift::protocol::TType _etype822;
+            xfer += iprot->readListBegin(_etype822, _size819);
+            this->partitionnames.resize(_size819);
+            uint32_t _i823;
+            for (_i823 = 0; _i823 < _size819; ++_i823)
             {
-              xfer += iprot->readString(this->partitionnames[_i815]);
+              xfer += iprot->readString(this->partitionnames[_i823]);
             }
             xfer += iprot->readListEnd();
           }
@@ -20614,9 +20880,9 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast816;
-          xfer += iprot->readI32(ecast816);
-          this->operationType = (DataOperationType::type)ecast816;
+          int32_t ecast824;
+          xfer += iprot->readI32(ecast824);
+          this->operationType = (DataOperationType::type)ecast824;
           this->__isset.operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -20668,10 +20934,10 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter817;
-    for (_iter817 = this->partitionnames.begin(); _iter817 != this->partitionnames.end(); ++_iter817)
+    std::vector<std::string> ::const_iterator _iter825;
+    for (_iter825 = this->partitionnames.begin(); _iter825 != this->partitionnames.end(); ++_iter825)
     {
-      xfer += oprot->writeString((*_iter817));
+      xfer += oprot->writeString((*_iter825));
     }
     xfer += oprot->writeListEnd();
   }
@@ -20698,23 +20964,23 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other818) {
-  txnid = other818.txnid;
-  writeid = other818.writeid;
-  dbname = other818.dbname;
-  tablename = other818.tablename;
-  partitionnames = other818.partitionnames;
-  operationType = other818.operationType;
-  __isset = other818.__isset;
-}
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other819) {
-  txnid = other819.txnid;
-  writeid = other819.writeid;
-  dbname = other819.dbname;
-  tablename = other819.tablename;
-  partitionnames = other819.partitionnames;
-  operationType = other819.operationType;
-  __isset = other819.__isset;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other826) {
+  txnid = other826.txnid;
+  writeid = other826.writeid;
+  dbname = other826.dbname;
+  tablename = other826.tablename;
+  partitionnames = other826.partitionnames;
+  operationType = other826.operationType;
+  __isset = other826.__isset;
+}
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other827) {
+  txnid = other827.txnid;
+  writeid = other827.writeid;
+  dbname = other827.dbname;
+  tablename = other827.tablename;
+  partitionnames = other827.partitionnames;
+  operationType = other827.operationType;
+  __isset = other827.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -20897,23 +21163,23 @@ void swap(BasicTxnInfo &a, BasicTxnInfo &b) {
   swap(a.__isset, b.__isset);
 }
 
-BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other820) {
-  isnull = other820.isnull;
-  time = other820.time;
-  txnid = other820.txnid;
-  dbname = other820.dbname;
-  tablename = other820.tablename;
-  partitionname = other820.partitionname;
-  __isset = other820.__isset;
-}
-BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other821) {
-  isnull = other821.isnull;
-  time = other821.time;
-  txnid = other821.txnid;
-  dbname = other821.dbname;
-  tablename = other821.tablename;
-  partitionname = other821.partitionname;
-  __isset = other821.__isset;
+BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other828) {
+  isnull = other828.isnull;
+  time = other828.time;
+  txnid = other828.txnid;
+  dbname = other828.dbname;
+  tablename = other828.tablename;
+  partitionname = other828.partitionname;
+  __isset = other828.__isset;
+}
+BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other829) {
+  isnull = other829.isnull;
+  time = other829.time;
+  txnid = other829.txnid;
+  dbname = other829.dbname;
+  tablename = other829.tablename;
+  partitionname = other829.partitionname;
+  __isset = other829.__isset;
   return *this;
 }
 void BasicTxnInfo::printTo(std::ostream& out) const {
@@ -21007,15 +21273,15 @@ uint32_t CreationMetadata::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->tablesUsed.clear();
-            uint32_t _size822;
-            ::apache::thrift::protocol::TType _etype825;
-            xfer += iprot->readSetBegin(_etype825, _size822);
-            uint32_t _i826;
-            for (_i826 = 0; _i826 < _size822; ++_i826)
+            uint32_t _size830;
+            ::apache::thrift::protocol::TType _etype833;
+            xfer += iprot->readSetBegin(_etype833, _size830);
+            uint32_t _i834;
+            for (_i834 = 0; _i834 < _size830; ++_i834)
             {
-              std::string _elem827;
-              xfer += iprot->readString(_elem827);
-              this->tablesUsed.insert(_elem827);
+              std::string _elem835;
+              xfer += iprot->readString(_elem835);
+              this->tablesUsed.insert(_elem835);
             }
             xfer += iprot->readSetEnd();
           }
@@ -21072,10 +21338,10 @@ uint32_t CreationMetadata::write(::apache::thrift::protocol::TProtocol* oprot) c
   xfer += oprot->writeFieldBegin("tablesUsed", ::apache::thrift::protocol::T_SET, 4);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tablesUsed.size()));
-    std::set<std::string> ::const_iterator _iter828;
-    for (_iter828 = this->tablesUsed.begin(); _iter828 != this->tablesUsed.end(); ++_iter828)
+    std::set<std::string> ::const_iterator _iter836;
+    for (_iter836 = this->tablesUsed.begin(); _iter836 != this->tablesUsed.end(); ++_iter836)
     {
-      xfer += oprot->writeString((*_iter828));
+      xfer += oprot->writeString((*_iter836));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -21101,21 +21367,21 @@ void swap(CreationMetadata &a, CreationMetadata &b) {
   swap(a.__isset, b.__isset);
 }
 
-CreationMetadata::CreationMetadata(const CreationMetadata& other829) {
-  catName = other829.catName;
-  dbName = other829.dbName;
-  tblName = other829.tblName;
-  tablesUsed = other829.tablesUsed;
-  validTxnList = other829.validTxnList;
-  __isset = other829.__isset;
-}
-CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other830) {
-  catName = other830.catName;
-  dbName = other830.dbName;
-  tblName = other830.tblName;
-  tablesUsed = other830.tablesUsed;
-  validTxnList = other830.validTxnList;
-  __isset = other830.__isset;
+CreationMetadata::CreationMetadata(const CreationMetadata& other837) {
+  catName = other837.catName;
+  dbName = other837.dbName;
+  tblName = other837.tblName;
+  tablesUsed = other837.tablesUsed;
+  validTxnList = other837.validTxnList;
+  __isset = other837.__isset;
+}
+CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other838) {
+  catName = other838.catName;
+  dbName = other838.dbName;
+  tblName = other838.tblName;
+  tablesUsed = other838.tablesUsed;
+  validTxnList = other838.validTxnList;
+  __isset = other838.__isset;
   return *this;
 }
 void CreationMetadata::printTo(std::ostream& out) const {
@@ -21221,15 +21487,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other831) {
-  lastEvent = other831.lastEvent;
-  maxEvents = other831.maxEvents;
-  __isset = other831.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other839) {
+  lastEvent = other839.lastEvent;
+  maxEvents = other839.maxEvents;
+  __isset = other839.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other832) {
-  lastEvent = other832.lastEvent;
-  maxEvents = other832.maxEvents;
-  __isset = other832.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other840) {
+  lastEvent = other840.lastEvent;
+  maxEvents = other840.maxEvents;
+  __isset = other840.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -21449,27 +21715,27 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other833) {
-  eventId = other833.eventId;
-  eventTime = other833.eventTime;
-  eventType = other833.eventType;
-  dbName = other833.dbName;
-  tableName = other833.tableName;
-  message = other833.message;
-  messageFormat = other833.messageFormat;
-  catName = other833.catName;
-  __isset = other833.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other834) {
-  eventId = other834.eventId;
-  eventTime = other834.eventTime;
-  eventType = other834.eventType;
-  dbName = other834.dbName;
-  tableName = other834.tableName;
-  message = other834.message;
-  messageFormat = other834.messageFormat;
-  catName = other834.catName;
-  __isset = other834.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other841) {
+  eventId = other841.eventId;
+  eventTime = other841.eventTime;
+  eventType = other841.eventType;
+  dbName = other841.dbName;
+  tableName = other841.tableName;
+  message = other841.message;
+  messageFormat = other841.messageFormat;
+  catName = other841.catName;
+  __isset = other841.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other842) {
+  eventId = other842.eventId;
+  eventTime = other842.eventTime;
+  eventType = other842.eventType;
+  dbName = other842.dbName;
+  tableName = other842.tableName;
+  message = other842.message;
+  messageFormat = other842.messageFormat;
+  catName = other842.catName;
+  __isset = other842.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -21521,14 +21787,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size835;
-            ::apache::thrift::protocol::TType _etype838;
-            xfer += iprot->readListBegin(_etype838, _size835);
-            this->events.resize(_size835);
-            uint32_t _i839;
-            for (_i839 = 0; _i839 < _size835; ++_i839)
+            uint32_t _size843;
+            ::apache::thrift::protocol::TType _etype846;
+            xfer += iprot->readListBegin(_etype846, _size843);
+            this->events.resize(_size843);
+            uint32_t _i847;
+            for (_i847 = 0; _i847 < _size843; ++_i847)
             {
-              xfer += this->events[_i839].read(iprot);
+              xfer += this->events[_i847].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -21559,10 +21825,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter840;
-    for (_iter840 = this->events.begin(); _iter840 != this->events.end(); ++_iter840)
+    std::vector<NotificationEvent> ::const_iterator _iter848;
+    for (_iter848 = this->events.begin(); _iter848 != this->events.end(); ++_iter848)
     {
-      xfer += (*_iter840).write(oprot);
+      xfer += (*_iter848).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -21578,11 +21844,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other841) {
-  events = other841.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other849) {
+  events = other849.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other842) {
-  events = other842.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other850) {
+  events = other850.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -21664,11 +21930,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other843) {
-  eventId = other843.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other851) {
+  eventId = other851.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other844) {
-  eventId = other844.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other852) {
+  eventId = other852.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -21790,17 +22056,17 @@ void swap(NotificationEventsCountRequest &a, NotificationEventsCountRequest &b)
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other845) {
-  fromEventId = other845.fromEventId;
-  dbName = other845.dbName;
-  catName = other845.catName;
-  __isset = other845.__isset;
+NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other853) {
+  fromEventId = other853.fromEventId;
+  dbName = other853.dbName;
+  catName = other853.catName;
+  __isset = other853.__isset;
 }
-NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other846) {
-  fromEventId = other846.fromEventId;
-  dbName = other846.dbName;
-  catName = other846.catName;
-  __isset = other846.__isset;
+NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other854) {
+  fromEventId = other854.fromEventId;
+  dbName = other854.dbName;
+  catName = other854.catName;
+  __isset = other854.__isset;
   return *this;
 }
 void NotificationEventsCountRequest::printTo(std::ostream& out) const {
@@ -21884,11 +22150,11 @@ void swap(NotificationEventsCountResponse &a, NotificationEventsCountResponse &b
   swap(a.eventsCount, b.eventsCount);
 }
 
-NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other847) {
-  eventsCount = other847.eventsCount;
+NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other855) {
+  eventsCount = other855.eventsCount;
 }
-NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other848) {
-  eventsCount = other848.eventsCount;
+NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other856) {
+  eventsCount = other856.eventsCount;
   return *this;
 }
 void NotificationEventsCountResponse::printTo(std::ostream& out) const {
@@ -21917,6 +22183,11 @@ void InsertEventRequestData::__set_filesAddedChecksum(const std::vector<std::str
 __isset.filesAddedChecksum = true;
 }
 
+void InsertEventRequestData::__set_subDirectoryList(const std::vector<std::string> & val) {
+  this->subDirectoryList = val;
+__isset.subDirectoryList = true;
+}
+
 uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -21951,14 +22222,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size849;
-            ::apache::thrift::protocol::TType _etype852;
-            xfer += iprot->readListBegin(_etype852, _size849);
-            this->filesAdded.resize(_size849);
-            uint32_t _i853;
-            for (_i853 = 0; _i853 < _size849; ++_i853)
+            uint32_t _size857;
+            ::apache::thrift::protocol::TType _etype860;
+            xfer += iprot->readListBegin(_etype860, _size857);
+            this->filesAdded.resize(_size857);
+            uint32_t _i861;
+            for (_i861 = 0; _i861 < _size857; ++_i861)
             {
-              xfer += iprot->readString(this->filesAdded[_i853]);
+              xfer += iprot->readString(this->filesAdded[_i861]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21971,14 +22242,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAddedChecksum.clear();
-            uint32_t _size854;
-            ::apache::thrift::protocol::TType _etype857;
-            xfer += iprot->readListBegin(_etype857, _size854);
-            this->filesAddedChecksum.resize(_size854);
-            uint32_t _i858;
-            for (_i858 = 0; _i858 < _size854; ++_i858)
+            uint32_t _size862;
+            ::apache::thrift::protocol::TType _etype865;
+            xfer += iprot->readListBegin(_etype865, _size862);
+            this->filesAddedChecksum.resize(_size862);
+            uint32_t _i866;
+            for (_i866 = 0; _i866 < _size862; ++_i866)
             {
-              xfer += iprot->readString(this->filesAddedChecksum[_i858]);
+              xfer += iprot->readString(this->filesAddedChecksum[_i866]);
             }
             xfer += iprot->readListEnd();
           }
@@ -21987,6 +22258,26 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
           xfer += iprot->skip(ftype);
         }
         break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->subDirectoryList.clear();
+            uint32_t _size867;
+            ::apache::thrift::protocol::TType _etype870;
+            xfer += iprot->readListBegin(_etype870, _size867);
+            this->subDirectoryList.resize(_size867);
+            uint32_t _i871;
+            for (_i871 = 0; _i871 < _size867; ++_i871)
+            {
+              xfer += iprot->readString(this->subDirectoryList[_i871]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          this->__isset.subDirectoryList = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -22014,10 +22305,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter859;
-    for (_iter859 = this->filesAdded.begin(); _iter859 != this->filesAdded.end(); ++_iter859)
+    std::vector<std::string> ::const_iterator _iter872;
+    for (_iter872 = this->filesAdded.begin(); _iter872 != this->filesAdded.end(); ++_iter872)
     {
-      xfer += oprot->writeString((*_iter859));
+      xfer += oprot->writeString((*_iter872));
     }
     xfer += oprot->writeListEnd();
   }
@@ -22027,10 +22318,23 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
     xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAddedChecksum.size()));
-      std::vector<std::string> ::const_iterator _iter860;
-      for (_iter860 = this->filesAddedChecksum.begin(); _iter860 != this->filesAddedChecksum.end(); ++_iter860)
+      std::vector<std::string> ::const_iterator _iter873;
+      for (_iter873 = this->filesAddedChecksum.begin(); _iter873 != this->filesAddedChecksum.end(); ++_iter873)
+      {
+        xfer += oprot->writeString((*_iter873));
+      }
+      xfer += oprot->writeListEnd();
+    }
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.subDirectoryList) {
+    xfer += oprot->writeFieldBegin("subDirectoryList", ::apache::thrift::protocol::T_LIST, 4);
+    {
+      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->subDirectoryList.size()));
+      std::vector<std::string> ::const_iterator _iter874;
+      for (_iter874 = this->subDirectoryList.begin(); _iter874 != this->subDirectoryList.end(); ++_iter874)
       {
-        xfer += oprot->writeString((*_iter860));
+        xfer += oprot->writeString((*_iter874));
       }
       xfer += oprot->writeListEnd();
     }
@@ -22046,20 +22350,23 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.replace, b.replace);
   swap(a.filesAdded, b.filesAdded);
   swap(a.filesAddedChecksum, b.filesAddedChecksum);
+  swap(a.subDirectoryList, b.subDirectoryList);
   swap(a.__isset, b.__isset);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other861) {
-  replace = other861.replace;
-  filesAdded = other861.filesAdded;
-  filesAddedChecksum = other861.filesAddedChecksum;
-  __isset = other861.__isset;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other875) {
+  replace = other875.replace;
+  filesAdded = other875.filesAdded;
+  filesAddedChecksum = other875.filesAddedChecksum;
+  subDirectoryList = other875.subDirectoryList;
+  __isset = other875.__isset;
 }
-InsertEventRequest

<TRUNCATED>

[05/46] hive git commit: HIVE-19944: Investigate and fix version mismatch of GCP (Adam Szita, reviewed by Vihang Karajgaonkar)

Posted by se...@apache.org.
HIVE-19944: Investigate and fix version mismatch of GCP (Adam Szita, reviewed by Vihang Karajgaonkar)


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

Branch: refs/heads/master-txnstats
Commit: bb531be2c4ec0b514e3625f43a7f222b2259da4a
Parents: 9bc90f2
Author: Adam Szita <sz...@cloudera.com>
Authored: Mon Jun 25 16:35:06 2018 +0200
Committer: Adam Szita <sz...@cloudera.com>
Committed: Mon Jul 2 12:39:03 2018 +0200

----------------------------------------------------------------------
 testutils/ptest2/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/bb531be2/testutils/ptest2/pom.xml
----------------------------------------------------------------------
diff --git a/testutils/ptest2/pom.xml b/testutils/ptest2/pom.xml
index 5cd733b..b8e11f0 100644
--- a/testutils/ptest2/pom.xml
+++ b/testutils/ptest2/pom.xml
@@ -28,7 +28,7 @@ limitations under the License.
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <log4j2.version>2.10.0</log4j2.version>
     <spring.framework.version>3.2.16.RELEASE</spring.framework.version>
-    <jclouds.version>2.0.0</jclouds.version>
+    <jclouds.version>2.1.0</jclouds.version>
     <checkstyle.conf.dir>${basedir}/../../checkstyle/</checkstyle.conf.dir>
     <maven.checkstyle.plugin.version>2.12.1</maven.checkstyle.plugin.version>
     <jackson.version>2.9.4</jackson.version>


[15/46] hive git commit: HIVE-20021: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID (Gopal V, reviewed by Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-20021: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID (Gopal V, reviewed by Sergey Shelukhin)


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

Branch: refs/heads/master-txnstats
Commit: 1fc1221536671f28ac28a3e4d99969a704cf740a
Parents: 1cedb12
Author: Gopal V <go...@apache.org>
Authored: Mon Jul 2 15:54:24 2018 -0700
Committer: Gopal V <go...@apache.org>
Committed: Mon Jul 2 15:56:08 2018 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/1fc12215/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 5a77122..02490f1 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -739,6 +739,9 @@ public class Hadoop23Shims extends HadoopShimsSecure {
 
     @Override
     public Long getFileId() {
+      if (fileId == HdfsConstants.GRANDFATHER_INODE_ID) {
+        return null;
+      }
       return fileId;
     }
   }


[10/46] hive git commit: HIVE-19711 Refactor Hive Schema Tool (Miklos Gergely via Alan Gates).

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
new file mode 100644
index 0000000..314aa60
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaTool.java
@@ -0,0 +1,826 @@
+/*
+ * 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.beeline.schematool;
+
+import java.io.BufferedWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.net.URI;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.dbcp.DelegatingConnection;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.text.StrTokenizer;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
+import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.PostgresCommandParser;
+import org.apache.hadoop.hive.shims.ShimLoader;
+
+public class TestSchemaTool extends TestCase {
+  private static HiveSchemaTool schemaTool;
+  private Connection conn;
+  private HiveConf hiveConf;
+  private String testMetastoreDB;
+  private PrintStream errStream;
+  private PrintStream outStream;
+  private String argsBase;
+  private HiveSchemaToolTaskValidate validator;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    testMetastoreDB = System.getProperty("java.io.tmpdir") +
+        File.separator + "test_metastore-" + new Random().nextInt();
+    System.setProperty(MetastoreConf.ConfVars.CONNECT_URL_KEY.getVarname(),
+        "jdbc:derby:" + testMetastoreDB + ";create=true");
+    hiveConf = new HiveConf(this.getClass());
+    schemaTool = new HiveSchemaTool(
+        System.getProperty("test.tmp.dir", "target/tmp"), hiveConf, "derby", null);
+
+    String userName = hiveConf.get(MetastoreConf.ConfVars.CONNECTION_USER_NAME.getVarname());
+    String passWord = ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(),
+        MetastoreConf.ConfVars.PWD.getVarname());
+    schemaTool.setUserName(userName);
+    schemaTool.setPassWord(passWord);
+
+    argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " ";
+
+    System.setProperty("beeLine.system.exit", "true");
+    errStream = System.err;
+    outStream = System.out;
+    conn = schemaTool.getConnectionToMetastore(false);
+
+    validator = new HiveSchemaToolTaskValidate();
+    validator.setHiveSchemaTool(schemaTool);
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    File metaStoreDir = new File(testMetastoreDB);
+    if (metaStoreDir.exists()) {
+      FileUtils.forceDeleteOnExit(metaStoreDir);
+    }
+    System.setOut(outStream);
+    System.setErr(errStream);
+    if (conn != null) {
+      conn.close();
+    }
+  }
+
+  /**
+   * Test the sequence validation functionality
+   * @throws Exception
+   */
+  public void testValidateSequences() throws Exception {
+    execute(new HiveSchemaToolTaskInit(), "-initSchema");
+
+    // Test empty database
+    boolean isValid = validator.validateSequences(conn);
+    assertTrue(isValid);
+
+    // Test valid case
+    String[] scripts = new String[] {
+        "insert into CTLGS values(99, 'test_cat_1', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
+        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
+        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1')"
+    };
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSequences(conn);
+    assertTrue(isValid);
+
+    // Test invalid case
+    scripts = new String[] {
+        "delete from SEQUENCE_TABLE",
+        "delete from DBS",
+        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
+        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1')"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSequences(conn);
+    assertFalse(isValid);
+  }
+
+  /**
+   * Test to validate that all tables exist in the HMS metastore.
+   * @throws Exception
+   */
+  public void testValidateSchemaTables() throws Exception {
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 2.0.0");
+
+    boolean isValid = (boolean)validator.validateSchemaTables(conn);
+    assertTrue(isValid);
+
+    // upgrade from 2.0.0 schema and re-validate
+    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchemaFrom 2.0.0");
+    isValid = (boolean)validator.validateSchemaTables(conn);
+    assertTrue(isValid);
+
+    // Simulate a missing table scenario by renaming a couple of tables
+    String[] scripts = new String[] {
+        "RENAME TABLE SEQUENCE_TABLE to SEQUENCE_TABLE_RENAMED",
+        "RENAME TABLE NUCLEUS_TABLES to NUCLEUS_TABLES_RENAMED"
+    };
+
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSchemaTables(conn);
+    assertFalse(isValid);
+
+    // Restored the renamed tables
+    scripts = new String[] {
+        "RENAME TABLE SEQUENCE_TABLE_RENAMED to SEQUENCE_TABLE",
+        "RENAME TABLE NUCLEUS_TABLES_RENAMED to NUCLEUS_TABLES"
+    };
+
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSchemaTables(conn);
+    assertTrue(isValid);
+
+    // Check that an exception from getMetaData() is reported correctly
+    try {
+      // Make a Connection object that will throw an exception
+      BadMetaDataConnection bad = new BadMetaDataConnection(conn);
+      validator.validateSchemaTables(bad);
+      fail("did not get expected exception");
+    } catch (HiveMetaException hme) {
+      String message = hme.getMessage();
+      assertTrue("Bad HiveMetaException message :" + message,
+          message.contains("Failed to retrieve schema tables from Hive Metastore DB"));
+      Throwable cause = hme.getCause();
+      assertNotNull("HiveMetaException did not contain a cause", cause);
+      String causeMessage = cause.getMessage();
+      assertTrue("Bad SQLException message: " + causeMessage, causeMessage.contains(
+          BadMetaDataConnection.FAILURE_TEXT));
+    }
+  }
+
+  /*
+   * Test the validation of incorrect NULL values in the tables
+   * @throws Exception
+   */
+  public void testValidateNullValues() throws Exception {
+    execute(new HiveSchemaToolTaskInit(), "-initSchema");
+
+    // Test empty database
+    boolean isValid = validator.validateColumnNullValues(conn);
+    assertTrue(isValid);
+
+    // Test valid case
+    createTestHiveTableSchemas();
+    isValid = validator.validateColumnNullValues(conn);
+
+    // Test invalid case
+    String[] scripts = new String[] {
+        "update TBLS set SD_ID=null"
+    };
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateColumnNullValues(conn);
+    assertFalse(isValid);
+  }
+
+  /**
+   * Test dryrun of schema initialization
+   * @throws Exception
+   */
+  public void testSchemaInitDryRun() throws Exception {
+    schemaTool.setDryRun(true);
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 0.7.0");
+    schemaTool.setDryRun(false);
+    try {
+      schemaTool.verifySchemaVersion();
+    } catch (HiveMetaException e) {
+      // The connection should fail since it the dry run
+      return;
+    }
+    fail("Dry run shouldn't create actual metastore");
+  }
+
+  /**
+   * Test dryrun of schema upgrade
+   * @throws Exception
+   */
+  public void testSchemaUpgradeDryRun() throws Exception {
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 0.7.0");
+
+    schemaTool.setDryRun(true);
+    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchemaFrom 0.7.0");
+    schemaTool.setDryRun(false);
+    try {
+      schemaTool.verifySchemaVersion();
+    } catch (HiveMetaException e) {
+      // The connection should fail since it the dry run
+      return;
+    }
+    fail("Dry run shouldn't upgrade metastore schema");
+  }
+
+  /**
+   * Test schema initialization
+   * @throws Exception
+   */
+  public void testSchemaInit() throws Exception {
+    IMetaStoreSchemaInfo metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf,
+        System.getProperty("test.tmp.dir", "target/tmp"), "derby");
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo " + metastoreSchemaInfo.getHiveSchemaVersion());
+    schemaTool.verifySchemaVersion();
+  }
+
+  /**
+  * Test validation for schema versions
+  * @throws Exception
+  */
+  public void testValidateSchemaVersions() throws Exception {
+    execute(new HiveSchemaToolTaskInit(), "-initSchema");
+    boolean isValid = validator.validateSchemaVersions();
+    // Test an invalid case with multiple versions
+    String[] scripts = new String[] {
+        "insert into VERSION values(100, '2.2.0', 'Hive release version 2.2.0')"
+    };
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSchemaVersions();
+    assertFalse(isValid);
+
+    scripts = new String[] {
+        "delete from VERSION where VER_ID = 100"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSchemaVersions();
+    assertTrue(isValid);
+
+    // Test an invalid case without version
+    scripts = new String[] {
+        "delete from VERSION"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateSchemaVersions();
+    assertFalse(isValid);
+  }
+
+  /**
+   * Test schema upgrade
+   * @throws Exception
+   */
+  public void testSchemaUpgrade() throws Exception {
+    boolean foundException = false;
+    // Initialize 0.7.0 schema
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 0.7.0");
+    // verify that driver fails due to older version schema
+    try {
+      schemaTool.verifySchemaVersion();
+    } catch (HiveMetaException e) {
+      // Expected to fail due to old schema
+      foundException = true;
+    }
+    if (!foundException) {
+      throw new Exception(
+          "Hive operations shouldn't pass with older version schema");
+    }
+
+    // Generate dummy pre-upgrade script with errors
+    String invalidPreUpgradeScript = writeDummyPreUpgradeScript(
+        0, "upgrade-0.11.0-to-0.12.0.derby.sql", "foo bar;");
+    // Generate dummy pre-upgrade scripts with valid SQL
+    String validPreUpgradeScript0 = writeDummyPreUpgradeScript(
+        0, "upgrade-0.12.0-to-0.13.0.derby.sql",
+        "CREATE TABLE schema_test0 (id integer);");
+    String validPreUpgradeScript1 = writeDummyPreUpgradeScript(
+        1, "upgrade-0.12.0-to-0.13.0.derby.sql",
+        "CREATE TABLE schema_test1 (id integer);");
+
+    // Capture system out and err
+    schemaTool.setVerbose(true);
+    OutputStream stderr = new ByteArrayOutputStream();
+    PrintStream errPrintStream = new PrintStream(stderr);
+    System.setErr(errPrintStream);
+    OutputStream stdout = new ByteArrayOutputStream();
+    PrintStream outPrintStream = new PrintStream(stdout);
+    System.setOut(outPrintStream);
+
+    // Upgrade schema from 0.7.0 to latest
+    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchemaFrom 0.7.0");
+
+    // Verify that the schemaTool ran pre-upgrade scripts and ignored errors
+    assertTrue(stderr.toString().contains(invalidPreUpgradeScript));
+    assertTrue(stderr.toString().contains("foo"));
+    assertFalse(stderr.toString().contains(validPreUpgradeScript0));
+    assertFalse(stderr.toString().contains(validPreUpgradeScript1));
+    assertTrue(stdout.toString().contains(validPreUpgradeScript0));
+    assertTrue(stdout.toString().contains(validPreUpgradeScript1));
+
+    // Verify that driver works fine with latest schema
+    schemaTool.verifySchemaVersion();
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testScripts() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+      "DROP TABLE IF EXISTS fooTab;",
+      "/*!1234 this is comment code like mysql */;",
+      "CREATE TABLE fooTab(id INTEGER);",
+      "DROP TABLE footab;",
+      "-- ending comment"
+    };
+    String resultScript[] = {
+      "DROP TABLE IF EXISTS fooTab",
+      "/*!1234 this is comment code like mysql */",
+      "CREATE TABLE fooTab(id INTEGER)",
+      "DROP TABLE footab",
+    };
+    String expectedSQL = StringUtils.join(resultScript, System.getProperty("line.separator")) +
+        System.getProperty("line.separator");
+    File testScriptFile = generateTestScript(testScript);
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby", false)
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
+
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  /**
+   * Test nested script formatting
+   * @throws Exception
+   */
+  public void testNestedScriptsForDerby() throws Exception {
+    String childTab1 = "childTab1";
+    String childTab2 = "childTab2";
+    String parentTab = "fooTab";
+
+    String childTestScript1[] = {
+      "-- this is a comment ",
+      "DROP TABLE IF EXISTS " + childTab1 + ";",
+      "CREATE TABLE " + childTab1 + "(id INTEGER);",
+      "DROP TABLE " + childTab1 + ";"
+    };
+    String childTestScript2[] = {
+        "-- this is a comment",
+        "DROP TABLE IF EXISTS " + childTab2 + ";",
+        "CREATE TABLE " + childTab2 + "(id INTEGER);",
+        "-- this is also a comment",
+        "DROP TABLE " + childTab2 + ";"
+    };
+
+    String parentTestScript[] = {
+        " -- this is a comment",
+        "DROP TABLE IF EXISTS " + parentTab + ";",
+        " -- this is another comment ",
+        "CREATE TABLE " + parentTab + "(id INTEGER);",
+        "RUN '" + generateTestScript(childTestScript1).getName() + "';",
+        "DROP TABLE " + parentTab + ";",
+        "RUN '" + generateTestScript(childTestScript2).getName() + "';",
+        "--ending comment ",
+      };
+
+    File testScriptFile = generateTestScript(parentTestScript);
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby", false)
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
+    assertFalse(flattenedSql.contains("RUN"));
+    assertFalse(flattenedSql.contains("comment"));
+    assertTrue(flattenedSql.contains(childTab1));
+    assertTrue(flattenedSql.contains(childTab2));
+    assertTrue(flattenedSql.contains(parentTab));
+  }
+
+  /**
+   * Test nested script formatting
+   * @throws Exception
+   */
+  public void testNestedScriptsForMySQL() throws Exception {
+    String childTab1 = "childTab1";
+    String childTab2 = "childTab2";
+    String parentTab = "fooTab";
+
+    String childTestScript1[] = {
+      "/* this is a comment code */",
+      "DROP TABLE IF EXISTS " + childTab1 + ";",
+      "CREATE TABLE " + childTab1 + "(id INTEGER);",
+      "DROP TABLE " + childTab1 + ";"
+    };
+    String childTestScript2[] = {
+        "/* this is a special exec code */;",
+        "DROP TABLE IF EXISTS " + childTab2 + ";",
+        "CREATE TABLE " + childTab2 + "(id INTEGER);",
+        "-- this is a comment",
+        "DROP TABLE " + childTab2 + ";"
+    };
+
+    String parentTestScript[] = {
+        " -- this is a comment",
+        "DROP TABLE IF EXISTS " + parentTab + ";",
+        " /* this is special exec code */;",
+        "CREATE TABLE " + parentTab + "(id INTEGER);",
+        "SOURCE " + generateTestScript(childTestScript1).getName() + ";",
+        "DROP TABLE " + parentTab + ";",
+        "SOURCE " + generateTestScript(childTestScript2).getName() + ";",
+        "--ending comment ",
+      };
+
+    File testScriptFile = generateTestScript(parentTestScript);
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("mysql", false)
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
+    assertFalse(flattenedSql.contains("RUN"));
+    assertFalse(flattenedSql.contains("comment"));
+    assertTrue(flattenedSql.contains(childTab1));
+    assertTrue(flattenedSql.contains(childTab2));
+    assertTrue(flattenedSql.contains(parentTab));
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testScriptWithDelimiter() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+      "DROP TABLE IF EXISTS fooTab;",
+      "DELIMITER $$",
+      "/*!1234 this is comment code like mysql */$$",
+      "CREATE TABLE fooTab(id INTEGER)$$",
+      "CREATE PROCEDURE fooProc()",
+      "SELECT * FROM fooTab;",
+      "CALL barProc();",
+      "END PROCEDURE$$",
+      "DELIMITER ;",
+      "DROP TABLE footab;",
+      "-- ending comment"
+    };
+    String resultScript[] = {
+      "DROP TABLE IF EXISTS fooTab",
+      "/*!1234 this is comment code like mysql */",
+      "CREATE TABLE fooTab(id INTEGER)",
+      "CREATE PROCEDURE fooProc()" + " " +
+      "SELECT * FROM fooTab;" + " " +
+      "CALL barProc();" + " " +
+      "END PROCEDURE",
+      "DROP TABLE footab",
+    };
+    String expectedSQL = StringUtils.join(resultScript, System.getProperty("line.separator")) +
+        System.getProperty("line.separator");
+    File testScriptFile = generateTestScript(testScript);
+    NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql", false);
+    String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
+        testScriptFile.getName());
+
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testScriptMultiRowComment() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+      "DROP TABLE IF EXISTS fooTab;",
+      "DELIMITER $$",
+      "/*!1234 this is comment code like mysql */$$",
+      "CREATE TABLE fooTab(id INTEGER)$$",
+      "DELIMITER ;",
+      "/* multiline comment started ",
+      " * multiline comment continue",
+      " * multiline comment ended */",
+      "DROP TABLE footab;",
+      "-- ending comment"
+    };
+    String parsedScript[] = {
+      "DROP TABLE IF EXISTS fooTab",
+      "/*!1234 this is comment code like mysql */",
+      "CREATE TABLE fooTab(id INTEGER)",
+      "DROP TABLE footab",
+    };
+
+    String expectedSQL = StringUtils.join(parsedScript, System.getProperty("line.separator")) +
+        System.getProperty("line.separator");
+    File testScriptFile = generateTestScript(testScript);
+    NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql", false);
+    String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
+        testScriptFile.getName());
+
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  /**
+   * Test nested script formatting
+   * @throws Exception
+   */
+  public void testNestedScriptsForOracle() throws Exception {
+    String childTab1 = "childTab1";
+    String childTab2 = "childTab2";
+    String parentTab = "fooTab";
+
+    String childTestScript1[] = {
+      "-- this is a comment ",
+      "DROP TABLE IF EXISTS " + childTab1 + ";",
+      "CREATE TABLE " + childTab1 + "(id INTEGER);",
+      "DROP TABLE " + childTab1 + ";"
+    };
+    String childTestScript2[] = {
+        "-- this is a comment",
+        "DROP TABLE IF EXISTS " + childTab2 + ";",
+        "CREATE TABLE " + childTab2 + "(id INTEGER);",
+        "-- this is also a comment",
+        "DROP TABLE " + childTab2 + ";"
+    };
+
+    String parentTestScript[] = {
+        " -- this is a comment",
+        "DROP TABLE IF EXISTS " + parentTab + ";",
+        " -- this is another comment ",
+        "CREATE TABLE " + parentTab + "(id INTEGER);",
+        "@" + generateTestScript(childTestScript1).getName() + ";",
+        "DROP TABLE " + parentTab + ";",
+        "@" + generateTestScript(childTestScript2).getName() + ";",
+        "--ending comment ",
+      };
+
+    File testScriptFile = generateTestScript(parentTestScript);
+    String flattenedSql = HiveSchemaHelper.getDbCommandParser("oracle", false)
+        .buildCommand(testScriptFile.getParentFile().getPath(),
+            testScriptFile.getName());
+    assertFalse(flattenedSql.contains("@"));
+    assertFalse(flattenedSql.contains("comment"));
+    assertTrue(flattenedSql.contains(childTab1));
+    assertTrue(flattenedSql.contains(childTab2));
+    assertTrue(flattenedSql.contains(parentTab));
+  }
+
+  /**
+   * Test script formatting
+   * @throws Exception
+   */
+  public void testPostgresFilter() throws Exception {
+    String testScript[] = {
+        "-- this is a comment",
+        "DROP TABLE IF EXISTS fooTab;",
+        HiveSchemaHelper.PostgresCommandParser.POSTGRES_STANDARD_STRINGS_OPT + ";",
+        "CREATE TABLE fooTab(id INTEGER);",
+        "DROP TABLE footab;",
+        "-- ending comment"
+    };
+
+    String expectedScriptWithOptionPresent[] = {
+        "DROP TABLE IF EXISTS fooTab",
+        HiveSchemaHelper.PostgresCommandParser.POSTGRES_STANDARD_STRINGS_OPT,
+        "CREATE TABLE fooTab(id INTEGER)",
+        "DROP TABLE footab",
+    };
+
+    NestedScriptParser noDbOptParser = HiveSchemaHelper
+        .getDbCommandParser("postgres", false);
+    String expectedSQL = StringUtils.join(
+        expectedScriptWithOptionPresent, System.getProperty("line.separator")) +
+            System.getProperty("line.separator");
+    File testScriptFile = generateTestScript(testScript);
+    String flattenedSql = noDbOptParser.buildCommand(
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    assertEquals(expectedSQL, flattenedSql);
+
+    String expectedScriptWithOptionAbsent[] = {
+        "DROP TABLE IF EXISTS fooTab",
+        "CREATE TABLE fooTab(id INTEGER)",
+        "DROP TABLE footab",
+    };
+
+    NestedScriptParser dbOptParser = HiveSchemaHelper.getDbCommandParser(
+        "postgres",
+        PostgresCommandParser.POSTGRES_SKIP_STANDARD_STRINGS_DBOPT,
+        null, null, null, null, false);
+    expectedSQL = StringUtils.join(
+        expectedScriptWithOptionAbsent, System.getProperty("line.separator")) +
+            System.getProperty("line.separator");
+    testScriptFile = generateTestScript(testScript);
+    flattenedSql = dbOptParser.buildCommand(
+        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
+    assertEquals(expectedSQL, flattenedSql);
+  }
+
+  /**
+   * Test validate uri of locations
+   * @throws Exception
+   */
+  public void testValidateLocations() throws Exception {
+    execute(new HiveSchemaToolTaskInit(), "-initSchema");
+    URI defaultRoot = new URI("hdfs://myhost.com:8020");
+    URI defaultRoot2 = new URI("s3://myhost2.com:8888");
+    //check empty DB
+    boolean isValid = validator.validateLocations(conn, null);
+    assertTrue(isValid);
+    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
+    assertTrue(isValid);
+
+ // Test valid case
+    String[] scripts = new String[] {
+         "insert into CTLGS values(3, 'test_cat_2', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
+         "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2')",
+         "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role', 'test_cat_2')",
+         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n')",
+         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,7,0 ,'hive',0,4000,'mytal4012','MANAGED_TABLE',NULL,NULL,'n')",
+         "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
+         "insert into SKEWED_STRING_LIST values(1)",
+         "insert into SKEWED_STRING_LIST values(2)",
+         "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')",
+         "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'s3://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')"
+       };
+    File scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateLocations(conn, null);
+    assertTrue(isValid);
+    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
+    assertTrue(isValid);
+    scripts = new String[] {
+        "delete from SKEWED_COL_VALUE_LOC_MAP",
+        "delete from SKEWED_STRING_LIST",
+        "delete from PARTITIONS",
+        "delete from TBLS",
+        "delete from SDS",
+        "delete from DBS",
+        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2')",
+        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role', 'test_cat_2')",
+        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role', 'test_cat_2')",
+        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role', 'test_cat_2')",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4001,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4003,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4004,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4002,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (5000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2016_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL,'n')",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4011 ,1435255431,4,0 ,'hive',0,4001,'mytal4011','MANAGED_TABLE',NULL,NULL,'n')",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,4,0 ,'hive',0,4002,'','MANAGED_TABLE',NULL,NULL,'n')",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4013 ,1435255431,4,0 ,'hive',0,4003,'mytal4013','MANAGED_TABLE',NULL,NULL,'n')",
+        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4014 ,1435255431,2,0 ,'hive',0,4003,'','MANAGED_TABLE',NULL,NULL,'n')",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4001, 1441402388,0, 'd1=1/d2=4001',4001,4011)",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4002, 1441402388,0, 'd1=1/d2=4002',4002,4012)",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4003, 1441402388,0, 'd1=1/d2=4003',4003,4013)",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4004, 1441402388,0, 'd1=1/d2=4004',4004,4014)",
+        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(5000, 1441402388,0, 'd1=1/d2=5000',5000,2)",
+        "insert into SKEWED_STRING_LIST values(1)",
+        "insert into SKEWED_STRING_LIST values(2)",
+        "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://yourhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')",
+        "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'file:///user/admin/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')"
+    };
+    scriptFile = generateTestScript(scripts);
+    schemaTool.runBeeLine(scriptFile.getPath());
+    isValid = validator.validateLocations(conn, null);
+    assertFalse(isValid);
+    isValid = validator.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
+    assertFalse(isValid);
+  }
+
+  public void testHiveMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 3.0.0");
+    validateMetastoreDbPropertiesTable();
+  }
+
+  public void testMetastoreDbPropertiesAfterUpgrade() throws HiveMetaException, IOException {
+    execute(new HiveSchemaToolTaskInit(), "-initSchemaTo 2.0.0");
+    execute(new HiveSchemaToolTaskUpgrade(), "-upgradeSchema");
+    validateMetastoreDbPropertiesTable();
+  }
+
+  private File generateTestScript(String [] stmts) throws IOException {
+    File testScriptFile = File.createTempFile("schematest", ".sql");
+    testScriptFile.deleteOnExit();
+    FileWriter fstream = new FileWriter(testScriptFile.getPath());
+    BufferedWriter out = new BufferedWriter(fstream);
+    for (String line: stmts) {
+      out.write(line);
+      out.newLine();
+    }
+    out.close();
+    return testScriptFile;
+  }
+
+  private void validateMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
+    boolean isValid = (boolean) validator.validateSchemaTables(conn);
+    assertTrue(isValid);
+    // adding same property key twice should throw unique key constraint violation exception
+    String[] scripts = new String[] {
+        "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-1', 'dummy uuid 1')",
+        "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-2', 'dummy uuid 2')", };
+    File scriptFile = generateTestScript(scripts);
+    Exception ex = null;
+    try {
+      schemaTool.runBeeLine(scriptFile.getPath());
+    } catch (Exception iox) {
+      ex = iox;
+    }
+    assertTrue(ex != null && ex instanceof IOException);
+  }
+  /**
+   * Write out a dummy pre-upgrade script with given SQL statement.
+   */
+  private String writeDummyPreUpgradeScript(int index, String upgradeScriptName,
+      String sql) throws Exception {
+    String preUpgradeScript = "pre-" + index + "-" + upgradeScriptName;
+    String dummyPreScriptPath = System.getProperty("test.tmp.dir", "target/tmp") +
+        File.separatorChar + "scripts" + File.separatorChar + "metastore" +
+        File.separatorChar + "upgrade" + File.separatorChar + "derby" +
+        File.separatorChar + preUpgradeScript;
+    FileWriter fstream = new FileWriter(dummyPreScriptPath);
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write(sql + System.getProperty("line.separator") + ";");
+    out.close();
+    return preUpgradeScript;
+  }
+
+  /**
+   * Insert the records in DB to simulate a hive table
+   * @throws IOException
+   */
+  private void createTestHiveTableSchemas() throws IOException {
+     String[] scripts = new String[] {
+          "insert into CTLGS values(2, 'my_catalog', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
+          "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8021/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'my_catalog')",
+          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
+          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
+          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,2,'aTable','MANAGED_TABLE',NULL,NULL,'n')",
+          "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)"
+        };
+     File scriptFile = generateTestScript(scripts);
+     schemaTool.runBeeLine(scriptFile.getPath());
+  }
+
+  /**
+   * A mock Connection class that throws an exception out of getMetaData().
+   */
+  class BadMetaDataConnection extends DelegatingConnection {
+    static final String FAILURE_TEXT = "fault injected";
+
+    BadMetaDataConnection(Connection connection) {
+      super(connection);
+    }
+
+    @Override
+    public DatabaseMetaData getMetaData() throws SQLException {
+      throw new SQLException(FAILURE_TEXT);
+    }
+  }
+
+  private void execute(HiveSchemaToolTask task, String taskArgs) throws HiveMetaException {
+    try {
+      StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"');
+      HiveSchemaToolCommandLine cl = new HiveSchemaToolCommandLine(tokenizer.getTokenArray());
+      task.setCommandLineArguments(cl);
+    } catch (Exception e) {
+      throw new IllegalStateException("Could not parse comman line \n" + argsBase + taskArgs, e);
+    }
+
+    task.setHiveSchemaTool(schemaTool);
+    task.execute();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java
new file mode 100644
index 0000000..a13603d
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/schematool/TestSchemaToolCatalogOps.java
@@ -0,0 +1,478 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.beeline.schematool;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.text.StrTokenizer;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
+import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.thrift.TException;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+
+public class TestSchemaToolCatalogOps {
+  private static HiveSchemaTool schemaTool;
+  private static HiveConf conf;
+  private IMetaStoreClient client;
+  private static String testMetastoreDB;
+  private static String argsBase;
+
+  @BeforeClass
+  public static void initDb() throws HiveMetaException, IOException {
+    conf = new HiveConf();
+    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, false);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.HMS_HANDLER_ATTEMPTS, 1);
+    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES, 1);
+    testMetastoreDB = System.getProperty("java.io.tmpdir") +
+        File.separator + "testschematoolcatopsdb";
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY,
+        "jdbc:derby:" + testMetastoreDB + ";create=true");
+    schemaTool = new HiveSchemaTool(
+        System.getProperty("test.tmp.dir", "target/tmp"), conf, "derby", null);
+
+    String userName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_USER_NAME);
+    String passWord = MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD);
+    schemaTool.setUserName(userName);
+    schemaTool.setPassWord(passWord);
+
+    argsBase = "-dbType derby -userName " + userName + " -passWord " + passWord + " ";
+    execute(new HiveSchemaToolTaskInit(), "-initSchema"); // Pre-install the database so all the tables are there.
+  }
+
+  @AfterClass
+  public static void removeDb() throws Exception {
+    File metaStoreDir = new File(testMetastoreDB);
+    if (metaStoreDir.exists()) {
+      FileUtils.forceDeleteOnExit(metaStoreDir);
+    }
+  }
+
+  @Before
+  public void createClient() throws MetaException {
+    client = new HiveMetaStoreClient(conf);
+  }
+
+  @Test
+  public void createCatalog() throws HiveMetaException, TException {
+    String catName = "my_test_catalog";
+    String location = "file:///tmp/my_test_catalog";
+    String description = "very descriptive";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
+
+    Catalog cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void createExistingCatalog() throws HiveMetaException {
+    String catName = "hive";
+    String location = "somewhere";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\"",
+        catName, location);
+    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
+  }
+
+  @Test
+  public void createExistingCatalogWithIfNotExists() throws HiveMetaException {
+    String catName = "my_existing_test_catalog";
+    String location = "file:///tmp/my_test_catalog";
+    String description = "very descriptive";
+    String argsCreate1 = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate1);
+
+    String argsCreate2 =
+        String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\" -ifNotExists",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate2);
+  }
+
+  @Test
+  public void alterCatalog() throws HiveMetaException, TException {
+    String catName = "an_alterable_catalog";
+    String location = "file:///tmp/an_alterable_catalog";
+    String description = "description";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
+
+    location = "file:///tmp/somewhere_else";
+    String argsAlter1 = String.format("-alterCatalog %s -catalogLocation \"%s\"",
+        catName, location);
+    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter1);
+    Catalog cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+
+    description = "a better description";
+    String argsAlter2 = String.format("-alterCatalog %s -catalogDescription \"%s\"",
+        catName, description);
+    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter2);
+    cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+
+    location = "file:///tmp/a_third_location";
+    description = "best description yet";
+    String argsAlter3 = String.format("-alterCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter3);
+    cat = client.getCatalog(catName);
+    Assert.assertEquals(location, cat.getLocationUri());
+    Assert.assertEquals(description, cat.getDescription());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void alterBogusCatalog() throws HiveMetaException {
+    String catName = "nosuch";
+    String location = "file:///tmp/somewhere";
+    String description = "whatever";
+    String argsAlter = String.format("-alterCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter);
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void alterCatalogNoChange() throws HiveMetaException {
+    String catName = "alter_cat_no_change";
+    String location = "file:///tmp/alter_cat_no_change";
+    String description = "description";
+    String argsCreate = String.format("-createCatalog %s -catalogLocation \"%s\" -catalogDescription \"%s\"",
+        catName, location, description);
+    execute(new HiveSchemaToolTaskCreateCatalog(), argsCreate);
+
+    String argsAlter = String.format("-alterCatalog %s", catName);
+    execute(new HiveSchemaToolTaskAlterCatalog(), argsAlter);
+  }
+
+  @Test
+  public void moveDatabase() throws HiveMetaException, TException {
+    String toCatName = "moveDbCat";
+    String dbName = "moveDbDb";
+    String tableName = "moveDbTable";
+    String funcName = "movedbfunc";
+    String partVal = "moveDbKey";
+
+    new CatalogBuilder()
+        .setName(toCatName)
+        .setLocation("file:///tmp")
+        .create(client);
+
+    Database db = new DatabaseBuilder()
+        .setCatalogName(DEFAULT_CATALOG_NAME)
+        .setName(dbName)
+        .create(client, conf);
+
+    new FunctionBuilder()
+        .inDb(db)
+        .setName(funcName)
+        .setClass("org.apache.hive.myudf")
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .inDb(db)
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog %s", dbName,
+        DEFAULT_CATALOG_NAME, toCatName);
+    execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
+
+    Database fetchedDb = client.getDatabase(toCatName, dbName);
+    Assert.assertNotNull(fetchedDb);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedDb.getCatalogName());
+
+    Function fetchedFunction = client.getFunction(toCatName, dbName, funcName);
+    Assert.assertNotNull(fetchedFunction);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedFunction.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedFunction.getDbName());
+
+    Table fetchedTable = client.getTable(toCatName, dbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(toCatName, dbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
+    Assert.assertEquals(dbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveDatabaseWithExistingDbOfSameNameAlreadyInTargetCatalog()
+      throws TException, HiveMetaException {
+    String catName = "clobberCatalog";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation("file:///tmp")
+        .create(client);
+    try {
+      String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog %s",
+          DEFAULT_DATABASE_NAME, catName, DEFAULT_CATALOG_NAME);
+      execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
+      Assert.fail("Attempt to move default database should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
+    Assert.assertTrue(dbNames.contains(DEFAULT_DATABASE_NAME));
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void moveNonExistentDatabase() throws TException, HiveMetaException {
+    String catName = "moveNonExistentDb";
+    new CatalogBuilder()
+        .setName(catName)
+        .setLocation("file:///tmp")
+        .create(client);
+    String argsMoveDB = String.format("-moveDatabase nosuch -fromCatalog %s -toCatalog %s",
+        catName, DEFAULT_CATALOG_NAME);
+    execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
+  }
+
+  @Test
+  public void moveDbToNonExistentCatalog() throws TException, HiveMetaException {
+    String dbName = "doomedToHomelessness";
+    new DatabaseBuilder()
+        .setName(dbName)
+        .create(client, conf);
+    try {
+      String argsMoveDB = String.format("-moveDatabase %s -fromCatalog %s -toCatalog nosuch",
+          dbName, DEFAULT_CATALOG_NAME);
+      execute(new HiveSchemaToolTaskMoveDatabase(), argsMoveDB);
+      Assert.fail("Attempt to move database to non-existent catalog should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
+    Assert.assertTrue(dbNames.contains(dbName.toLowerCase()));
+  }
+
+  @Test
+  public void moveTable() throws TException, HiveMetaException {
+    String toCatName = "moveTableCat";
+    String toDbName = "moveTableDb";
+    String tableName = "moveTableTable";
+    String partVal = "moveTableKey";
+
+    new CatalogBuilder()
+        .setName(toCatName)
+        .setLocation("file:///tmp")
+        .create(client);
+
+    new DatabaseBuilder()
+        .setCatalogName(toCatName)
+        .setName(toDbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    String argsMoveTable = String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+        tableName, DEFAULT_CATALOG_NAME, toCatName, DEFAULT_DATABASE_NAME, toDbName);
+    execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
+
+    Table fetchedTable = client.getTable(toCatName, toDbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(toCatName, toDbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveTableWithinCatalog() throws TException, HiveMetaException {
+    String toDbName = "moveTableWithinCatalogDb";
+    String tableName = "moveTableWithinCatalogTable";
+    String partVal = "moveTableWithinCatalogKey";
+
+    new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+
+    Table table = new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .addPartCol("p", "string")
+        .create(client, conf);
+
+    new PartitionBuilder()
+        .inTable(table)
+        .addValue(partVal)
+        .addToTable(client, conf);
+
+    String argsMoveTable = String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+        tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
+    execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
+
+    Table fetchedTable = client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
+    Assert.assertNotNull(fetchedTable);
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedTable.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
+
+    Partition fetchedPart =
+        client.getPartition(DEFAULT_CATALOG_NAME, toDbName, tableName, Collections.singletonList(partVal));
+    Assert.assertNotNull(fetchedPart);
+    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedPart.getCatName());
+    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
+    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
+  }
+
+  @Test
+  public void moveTableWithExistingTableOfSameNameAlreadyInTargetDatabase()
+      throws TException, HiveMetaException {
+    String toDbName = "clobberTableDb";
+    String tableName = "clobberTableTable";
+
+    Database toDb = new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+
+    new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .create(client, conf);
+
+    new TableBuilder()
+        .inDb(toDb)
+        .setTableName(tableName)
+        .addCol("b", "varchar(32)")
+        .create(client, conf);
+
+    try {
+      String argsMoveTable =
+          String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+          tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
+      execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
+      Assert.fail("Attempt to move table should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
+    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
+
+    // Make sure the table in the target database didn't get clobbered
+    Table fetchedTable =  client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
+    Assert.assertEquals("b", fetchedTable.getSd().getCols().get(0).getName());
+  }
+
+  @Test(expected = HiveMetaException.class)
+  public void moveNonExistentTable() throws TException, HiveMetaException {
+    String toDbName = "moveNonExistentTable";
+    new DatabaseBuilder()
+        .setName(toDbName)
+        .create(client, conf);
+    String argsMoveTable =
+        String.format("-moveTable nosuch -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase %s",
+        DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName);
+    execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
+  }
+
+  @Test
+  public void moveTableToNonExistentDb() throws TException, HiveMetaException {
+    String tableName = "doomedToWander";
+    new TableBuilder()
+        .setTableName(tableName)
+        .addCol("a", "int")
+        .create(client, conf);
+
+    try {
+      String argsMoveTable =
+          String.format("-moveTable %s -fromCatalog %s -toCatalog %s -fromDatabase %s -toDatabase nosuch",
+          tableName, DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
+      execute(new HiveSchemaToolTaskMoveTable(), argsMoveTable);
+      Assert.fail("Attempt to move table to non-existent table should have failed.");
+    } catch (HiveMetaException e) {
+      // good
+    }
+
+    // Make sure nothing really moved
+    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
+    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
+  }
+
+  private static void execute(HiveSchemaToolTask task, String taskArgs) throws HiveMetaException {
+    try {
+      StrTokenizer tokenizer = new StrTokenizer(argsBase + taskArgs, ' ', '\"');
+      HiveSchemaToolCommandLine cl = new HiveSchemaToolCommandLine(tokenizer.getTokenArray());
+      task.setCommandLineArguments(cl);
+    } catch (Exception e) {
+      throw new IllegalStateException("Could not parse comman line \n" + argsBase + taskArgs, e);
+    }
+
+    task.setHiveSchemaTool(schemaTool);
+    task.execute();
+  }
+}


[04/46] hive git commit: HIVE-19812: Disable external table replication by default via a configuration property (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
HIVE-19812: Disable external table replication by default via a configuration property (Mahesh Kumar Behera, reviewed by Sankar Hariappan)


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

Branch: refs/heads/master-txnstats
Commit: 9bc90f28cf4406d74f9597837c2b72c809b582e6
Parents: b9bac8e
Author: Sankar Hariappan <sa...@apache.org>
Authored: Mon Jul 2 16:02:45 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Mon Jul 2 16:02:45 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  8 +-
 .../hadoop/hive/ql/parse/TestExportImport.java  | 55 ++++++++++++++
 ...TestReplicationScenariosAcrossInstances.java | 77 ++++++++++++++++++++
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |  4 +-
 .../apache/hadoop/hive/ql/exec/ExportTask.java  |  2 +-
 .../hadoop/hive/ql/parse/repl/dump/Utils.java   |  6 ++
 .../queries/clientpositive/repl_2_exim_basic.q  |  1 +
 .../exim_03_nonpart_noncompat_colschema.q.out   |  2 +-
 .../exim_04_nonpart_noncompat_colnumber.q.out   |  2 +-
 .../exim_05_nonpart_noncompat_coltype.q.out     |  2 +-
 .../exim_06_nonpart_noncompat_storage.q.out     |  2 +-
 .../exim_07_nonpart_noncompat_ifof.q.out        |  2 +-
 .../exim_08_nonpart_noncompat_serde.q.out       |  2 +-
 .../exim_09_nonpart_noncompat_serdeparam.q.out  |  2 +-
 .../exim_10_nonpart_noncompat_bucketing.q.out   |  2 +-
 .../exim_11_nonpart_noncompat_sorting.q.out     |  2 +-
 .../exim_12_nonnative_export.q.out              |  2 +-
 .../exim_13_nonnative_import.q.out              |  2 +-
 .../clientnegative/exim_14_nonpart_part.q.out   |  2 +-
 .../clientnegative/exim_15_part_nonpart.q.out   |  2 +-
 .../exim_16_part_noncompat_schema.q.out         |  2 +-
 .../exim_19_external_over_existing.q.out        |  2 +-
 .../exim_21_part_managed_external.q.out         |  2 +-
 23 files changed, 165 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/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 35aee20..a3dd53e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -466,7 +466,8 @@ public class HiveConf extends Configuration {
         new TimeValidator(TimeUnit.DAYS),
         "TTL of dump dirs before cleanup."),
     REPL_DUMP_METADATA_ONLY("hive.repl.dump.metadata.only", false,
-        "Indicates whether replication dump only metadata information or data + metadata."),
+        "Indicates whether replication dump only metadata information or data + metadata. \n"
+          + "This config makes hive.repl.include.external.tables config ineffective."),
     REPL_DUMP_INCLUDE_ACID_TABLES("hive.repl.dump.include.acid.tables", false,
         "Indicates if repl dump should include information about ACID tables. It should be \n"
             + "used in conjunction with 'hive.repl.dump.metadata.only' to enable copying of \n"
@@ -482,6 +483,11 @@ public class HiveConf extends Configuration {
     REPL_ADD_RAW_RESERVED_NAMESPACE("hive.repl.add.raw.reserved.namespace", false,
         "For TDE with same encryption keys on source and target, allow Distcp super user to access \n"
             + "the raw bytes from filesystem without decrypting on source and then encrypting on target."),
+    REPL_INCLUDE_EXTERNAL_TABLES("hive.repl.include.external.tables", false,
+        "Indicates if repl dump should include information about external tables. It should be \n"
+          + "used in conjunction with 'hive.repl.dump.metadata.only' set to false. if 'hive.repl.dump.metadata.only' \n"
+          + " is set to true then this config parameter has no effect as external table meta data is flushed \n"
+          + " always by default."),
     LOCALSCRATCHDIR("hive.exec.local.scratchdir",
         "${system:java.io.tmpdir}" + File.separator + "${system:user.name}",
         "Local scratch space for Hive jobs"),

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
index 53d13d8..e442528 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
@@ -42,6 +42,7 @@ public class TestExportImport {
   protected static final Logger LOG = LoggerFactory.getLogger(TestExportImport.class);
   private static WarehouseInstance srcHiveWarehouse;
   private static WarehouseInstance destHiveWarehouse;
+  private static WarehouseInstance dumpExternalWarehouse;
 
   @Rule
   public final TestName testName = new TestName();
@@ -58,9 +59,15 @@ public class TestExportImport {
     HashMap<String, String> overridesForHiveConf = new HashMap<String, String>() {{
       put(HiveConf.ConfVars.HIVE_IN_TEST.varname, "false");
     }};
+    HashMap<String, String> overridesForHiveConfDump = new HashMap<String, String>() {{
+        put(HiveConf.ConfVars.HIVE_IN_TEST.varname, "false");
+        put(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES.varname, "true");
+    }};
     srcHiveWarehouse =
         new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
     destHiveWarehouse = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
+    dumpExternalWarehouse =
+            new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConfDump);
   }
 
   @AfterClass
@@ -110,6 +117,54 @@ public class TestExportImport {
   }
 
   @Test
+  public void testExportExternalTableSetFalse() throws Throwable {
+    String path = "hdfs:///tmp/" + dbName + "/";
+    String exportMDPath = "'" + path + "1/'";
+    String exportDataPath = "'" + path + "2/'";
+    String exportDataPathRepl = "'" + path + "3/'";
+    srcHiveWarehouse.run("create external table " + dbName + ".t1 (i int)")
+            .run("insert into table " + dbName + ".t1 values (1),(2)")
+            .run("export table " + dbName + ".t1 to " + exportMDPath + " for metadata replication('1')")
+            .run("export table " + dbName + ".t1 to " + exportDataPath)
+            .runFailure("export table " + dbName + ".t1 to " + exportDataPathRepl + " for replication('2')");
+
+    destHiveWarehouse.run("use " + replDbName)
+            .run("import table " + replDbName + ".t1 from " + exportMDPath)
+            .run("show tables like 't1'")
+            .verifyResult("t1")
+            .run("import table " + replDbName + ".t2 from " + exportDataPath)
+            .run("select * from " + replDbName + ".t2")
+            .verifyResults(new String[] {"1", "2" })
+            .runFailure("import table " + replDbName + ".t3 from " + exportDataPathRepl)
+            .run("show tables like 't3'")
+            .verifyFailure(new String[] {"t3"});
+  }
+
+  @Test
+  public void testExportExternalTableSetTrue() throws Throwable {
+    String path = "hdfs:///tmp/" + dbName + "/";
+    String exportMDPath = "'" + path + "1/'";
+    String exportDataPath = "'" + path + "2/'";
+    String exportDataPathRepl = "'" + path + "3/'";
+    dumpExternalWarehouse.run("create external table " + dbName + ".t1 (i int)")
+            .run("insert into table " + dbName + ".t1 values (1),(2)")
+            .run("export table " + dbName + ".t1 to " + exportDataPathRepl + " for replication('2')")
+            .run("export table " + dbName + ".t1 to " + exportMDPath + " for metadata replication('1')")
+            .run("export table " + dbName + ".t1 to " + exportDataPath);
+
+    destHiveWarehouse.run("use " + replDbName)
+            .run("import table " + replDbName + ".t1 from " + exportMDPath)
+            .run("show tables like 't1'")
+            .verifyResult("t1")
+            .run("import table " + replDbName + ".t2 from " + exportDataPath)
+            .run("select * from " + replDbName + ".t2")
+            .verifyResults(new String[] {"1", "2" })
+            .run("import table " + replDbName + ".t3 from " + exportDataPathRepl)
+            .run("select * from " + replDbName + ".t3")
+            .verifyResults(new String[] {"1", "2" });
+  }
+
+  @Test
   public void databaseTheTableIsImportedIntoShouldBeParsedFromCommandLine() throws Throwable {
     String path = "hdfs:///tmp/" + dbName + "/";
     String exportPath = "'" + path + "1/'";

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
index 08f0130..ff7f9bc 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
@@ -1313,4 +1313,81 @@ public class TestReplicationScenariosAcrossInstances {
             .run("show functions like '" + replicatedDbName + "*'")
             .verifyResult(replicatedDbName + ".testFunctionOne");
   }
+
+  @Test
+  public void testDumpExternalTableSetFalse() throws Throwable {
+    WarehouseInstance.Tuple tuple = primary
+            .run("use " + primaryDbName)
+            .run("create external table t1 (id int)")
+            .run("insert into table t1 values (1)")
+            .run("insert into table t1 values (2)")
+            .run("create external table t2 (place string) partitioned by (country string)")
+            .run("insert into table t2 partition(country='india') values ('bangalore')")
+            .run("insert into table t2 partition(country='us') values ('austin')")
+            .run("insert into table t2 partition(country='france') values ('paris')")
+            .dump(primaryDbName, null);
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("repl status " + replicatedDbName)
+            .verifyResult(tuple.lastReplicationId)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't1'")
+            .verifyFailure(new String[] {"t1"})
+            .run("show tables like 't2'")
+            .verifyFailure(new String[] {"t2"});
+
+    tuple = primary.run("use " + primaryDbName)
+            .run("create external table t3 (id int)")
+            .run("insert into table t3 values (10)")
+            .run("insert into table t3 values (20)")
+            .dump("repl dump " + primaryDbName + " from " + tuple.lastReplicationId
+                    + " with ('hive.repl.dump.metadata.only'='true')");
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't3'")
+            .verifyResult("t3")
+            .run("select id from t3 where id = 10")
+            .verifyFailure(new String[] {"10"});
+  }
+
+  @Test
+  public void testDumpExternalTableSetTrue() throws Throwable {
+    WarehouseInstance.Tuple tuple = primary
+            .run("use " + primaryDbName)
+            .run("create external table t1 (id int)")
+            .run("insert into table t1 values (1)")
+            .run("insert into table t1 values (2)")
+            .run("create external table t2 (place string) partitioned by (country string)")
+            .run("insert into table t2 partition(country='india') values ('bangalore')")
+            .run("insert into table t2 partition(country='us') values ('austin')")
+            .run("insert into table t2 partition(country='france') values ('paris')")
+            .dump("repl dump " + primaryDbName + " with ('hive.repl.include.external.tables'='true')");
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't1'")
+            .verifyResult("t1")
+            .run("show tables like 't2'")
+            .verifyResult("t2")
+            .run("repl status " + replicatedDbName)
+            .verifyResult(tuple.lastReplicationId)
+            .run("select country from t2 where country = 'us'")
+            .verifyResult("us")
+            .run("select country from t2 where country = 'france'")
+            .verifyResult("france");
+
+    tuple = primary.run("use " + primaryDbName)
+            .run("create external table t3 (id int)")
+            .run("insert into table t3 values (10)")
+            .dump("repl dump " + primaryDbName + " from " + tuple.lastReplicationId
+                    + " with ('hive.repl.include.external.tables'='true')");
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't3'")
+            .verifyResult("t3")
+            .run("select id from t3")
+            .verifyResult("10");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index b2c9daa..06d0ed3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -199,8 +199,8 @@ public enum ErrorMsg {
   NEED_TABLE_SPECIFICATION(10117, "Table name could be determined; It should be specified "),
   PARTITION_EXISTS(10118, "Partition already exists"),
   TABLE_DATA_EXISTS(10119, "Table exists and contains data files"),
-  INCOMPATIBLE_SCHEMA(10120, "The existing table is not compatible with the import spec. "),
-  EXIM_FOR_NON_NATIVE(10121, "Export/Import cannot be done for a non-native table. "),
+  INCOMPATIBLE_SCHEMA(10120, "The existing table is not compatible with the Export/Import spec. "),
+  EXIM_FOR_NON_NATIVE(10121, "Export/Import cannot be done for a non-native table."),
   INSERT_INTO_BUCKETIZED_TABLE(10122, "Bucketized tables do not support INSERT INTO:"),
   PARTSPEC_DIFFER_FROM_SCHEMA(10125, "Partition columns in partition specification are "
       + "not the same as that defined in the table schema. "

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
index 3c6a606..078691c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
@@ -55,7 +55,7 @@ public class ExportTask extends Task<ExportWork> implements Serializable {
       TableExport tableExport = new TableExport(exportPaths, work.getTableSpec(),
           work.getReplicationSpec(), db, null, conf, work.getMmContext());
       if (!tableExport.write()) {
-        throw new SemanticException(ErrorMsg.EXIM_FOR_NON_NATIVE.getMsg());
+        throw new SemanticException(ErrorMsg.INCOMPATIBLE_SCHEMA.getMsg());
       }
     } catch (Exception e) {
       LOG.error("failed", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
index 976104c..c0701c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -180,6 +181,11 @@ public class Utils {
     }
 
     if (replicationSpec.isInReplicationScope()) {
+      if (!hiveConf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES) &&
+              MetaStoreUtils.isExternalTable(tableHandle.getTTable()) && !replicationSpec.isMetadataOnly()) {
+        return false;
+      }
+
       boolean isAcidTable = AcidUtils.isTransactionalTable(tableHandle);
       if (isAcidTable) {
         return hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_INCLUDE_ACID_TABLES);

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/repl_2_exim_basic.q b/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
index 0a2a53d..5b75ca8 100644
--- a/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
+++ b/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
@@ -2,6 +2,7 @@ set hive.mapred.mode=nonstrict;
 set hive.test.mode=true;
 set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=managed_t,ext_t,managed_t_imported,managed_t_r_imported,ext_t_imported,ext_t_r_imported;
+set hive.repl.include.external.tables=true;
 
 drop table if exists managed_t;
 drop table if exists ext_t;

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out b/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
index b61c38c..10ce3e9 100644
--- a/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
+++ b/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_key int comment "department
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Column Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Column Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out b/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
index f8f5ea7..c53407b 100644
--- a/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
+++ b/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Column Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Column Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out b/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
index f118aa7..4806861 100644
--- a/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
+++ b/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id bigint comment "departmen
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Column Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Column Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out b/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
index 4b0fbdb..748d596 100644
--- a/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
+++ b/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table inputformat/outputformats do not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table inputformat/outputformats do not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out b/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
index 2c63d1b..7c3ba02 100644
--- a/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
+++ b/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
@@ -65,4 +65,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table inputformat/outputformats do not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table inputformat/outputformats do not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out b/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
index 6098ad5..eb3715f 100644
--- a/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
+++ b/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
@@ -61,4 +61,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table Serde class does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table Serde class does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out b/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
index e3cc024..d076829 100644
--- a/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
+++ b/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
@@ -69,4 +69,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table Serde format does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table Serde format does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out b/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
index 8dc3df0..0ad8410 100644
--- a/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
+++ b/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
@@ -61,4 +61,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table bucketing spec does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table bucketing spec does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out b/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
index 872d5e6..abd0d82 100644
--- a/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
+++ b/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
@@ -63,4 +63,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table sorting spec does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table sorting spec does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out b/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
index bd73536..4546490 100644
--- a/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
+++ b/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
@@ -16,4 +16,4 @@ PREHOOK: query: export table exim_department to 'ql/test/data/exports/exim_depar
 PREHOOK: type: EXPORT
 PREHOOK: Input: default@exim_department
 #### A masked pattern was here ####
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExportTask. Export/Import cannot be done for a non-native table. 
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExportTask. The existing table is not compatible with the Export/Import spec. 

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out b/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
index d820643..ab45089 100644
--- a/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
+++ b/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10121]: Export/Import cannot be done for a non-native table. 
+FAILED: SemanticException [Error 10121]: Export/Import cannot be done for a non-native table.

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out b/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
index da116ef..f5d19a8 100644
--- a/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
+++ b/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
@@ -61,4 +61,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Partition Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Partition Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out b/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
index dfbf025..3bb0fce 100644
--- a/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
+++ b/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
@@ -62,4 +62,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Partition Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Partition Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out b/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
index 4cb6ca7..302e511 100644
--- a/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
+++ b/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
@@ -64,4 +64,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Partition Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Partition Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out b/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
index 763efb2..db01d4f 100644
--- a/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
+++ b/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create  table exim_department ( dep_id int comment "department
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   External table cannot overwrite existing table. Drop existing table first.
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   External table cannot overwrite existing table. Drop existing table first.

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out b/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
index fd27f29..ebcc8db 100644
--- a/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
+++ b/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
@@ -109,4 +109,4 @@ POSTHOOK: query: create table exim_employee ( emp_id int comment "employee id")
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_employee
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   External table cannot overwrite existing table. Drop existing table first.
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   External table cannot overwrite existing table. Drop existing table first.


[21/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
index 5402372..add9197 100755
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
@@ -189,6 +189,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  NotificationEventsCountResponse get_notification_events_count(NotificationEventsCountRequest rqst)')
   print('  FireEventResponse fire_listener_event(FireEventRequest rqst)')
   print('  void flushCache()')
+  print('  WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)')
   print('  CmRecycleResponse cm_recycle(CmRecycleRequest request)')
   print('  GetFileMetadataByExprResult get_file_metadata_by_expr(GetFileMetadataByExprRequest req)')
   print('  GetFileMetadataResult get_file_metadata(GetFileMetadataRequest req)')
@@ -1290,6 +1291,12 @@ elif cmd == 'flushCache':
     sys.exit(1)
   pp.pprint(client.flushCache())
 
+elif cmd == 'add_write_notification_log':
+  if len(args) != 1:
+    print('add_write_notification_log requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.add_write_notification_log(eval(args[0]),))
+
 elif cmd == 'cm_recycle':
   if len(args) != 1:
     print('cm_recycle requires 1 args')

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index 5a3f2c1..a5bcc10 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@ -1309,6 +1309,13 @@ class Iface(fb303.FacebookService.Iface):
   def flushCache(self):
     pass
 
+  def add_write_notification_log(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    pass
+
   def cm_recycle(self, request):
     """
     Parameters:
@@ -7546,6 +7553,37 @@ class Client(fb303.FacebookService.Client, Iface):
     iprot.readMessageEnd()
     return
 
+  def add_write_notification_log(self, rqst):
+    """
+    Parameters:
+     - rqst
+    """
+    self.send_add_write_notification_log(rqst)
+    return self.recv_add_write_notification_log()
+
+  def send_add_write_notification_log(self, rqst):
+    self._oprot.writeMessageBegin('add_write_notification_log', TMessageType.CALL, self._seqid)
+    args = add_write_notification_log_args()
+    args.rqst = rqst
+    args.write(self._oprot)
+    self._oprot.writeMessageEnd()
+    self._oprot.trans.flush()
+
+  def recv_add_write_notification_log(self):
+    iprot = self._iprot
+    (fname, mtype, rseqid) = iprot.readMessageBegin()
+    if mtype == TMessageType.EXCEPTION:
+      x = TApplicationException()
+      x.read(iprot)
+      iprot.readMessageEnd()
+      raise x
+    result = add_write_notification_log_result()
+    result.read(iprot)
+    iprot.readMessageEnd()
+    if result.success is not None:
+      return result.success
+    raise TApplicationException(TApplicationException.MISSING_RESULT, "add_write_notification_log failed: unknown result")
+
   def cm_recycle(self, request):
     """
     Parameters:
@@ -9170,6 +9208,7 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     self._processMap["get_notification_events_count"] = Processor.process_get_notification_events_count
     self._processMap["fire_listener_event"] = Processor.process_fire_listener_event
     self._processMap["flushCache"] = Processor.process_flushCache
+    self._processMap["add_write_notification_log"] = Processor.process_add_write_notification_log
     self._processMap["cm_recycle"] = Processor.process_cm_recycle
     self._processMap["get_file_metadata_by_expr"] = Processor.process_get_file_metadata_by_expr
     self._processMap["get_file_metadata"] = Processor.process_get_file_metadata
@@ -13287,6 +13326,25 @@ class Processor(fb303.FacebookService.Processor, Iface, TProcessor):
     oprot.writeMessageEnd()
     oprot.trans.flush()
 
+  def process_add_write_notification_log(self, seqid, iprot, oprot):
+    args = add_write_notification_log_args()
+    args.read(iprot)
+    iprot.readMessageEnd()
+    result = add_write_notification_log_result()
+    try:
+      result.success = self._handler.add_write_notification_log(args.rqst)
+      msg_type = TMessageType.REPLY
+    except (TTransport.TTransportException, KeyboardInterrupt, SystemExit):
+      raise
+    except Exception as ex:
+      msg_type = TMessageType.EXCEPTION
+      logging.exception(ex)
+      result = TApplicationException(TApplicationException.INTERNAL_ERROR, 'Internal error')
+    oprot.writeMessageBegin("add_write_notification_log", msg_type, seqid)
+    result.write(oprot)
+    oprot.writeMessageEnd()
+    oprot.trans.flush()
+
   def process_cm_recycle(self, seqid, iprot, oprot):
     args = cm_recycle_args()
     args.read(iprot)
@@ -15987,10 +16045,10 @@ class get_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype819, _size816) = iprot.readListBegin()
-          for _i820 in xrange(_size816):
-            _elem821 = iprot.readString()
-            self.success.append(_elem821)
+          (_etype840, _size837) = iprot.readListBegin()
+          for _i841 in xrange(_size837):
+            _elem842 = iprot.readString()
+            self.success.append(_elem842)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16013,8 +16071,8 @@ class get_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter822 in self.success:
-        oprot.writeString(iter822)
+      for iter843 in self.success:
+        oprot.writeString(iter843)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16119,10 +16177,10 @@ class get_all_databases_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype826, _size823) = iprot.readListBegin()
-          for _i827 in xrange(_size823):
-            _elem828 = iprot.readString()
-            self.success.append(_elem828)
+          (_etype847, _size844) = iprot.readListBegin()
+          for _i848 in xrange(_size844):
+            _elem849 = iprot.readString()
+            self.success.append(_elem849)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16145,8 +16203,8 @@ class get_all_databases_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter829 in self.success:
-        oprot.writeString(iter829)
+      for iter850 in self.success:
+        oprot.writeString(iter850)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -16916,12 +16974,12 @@ class get_type_all_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype831, _vtype832, _size830 ) = iprot.readMapBegin()
-          for _i834 in xrange(_size830):
-            _key835 = iprot.readString()
-            _val836 = Type()
-            _val836.read(iprot)
-            self.success[_key835] = _val836
+          (_ktype852, _vtype853, _size851 ) = iprot.readMapBegin()
+          for _i855 in xrange(_size851):
+            _key856 = iprot.readString()
+            _val857 = Type()
+            _val857.read(iprot)
+            self.success[_key856] = _val857
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -16944,9 +17002,9 @@ class get_type_all_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter837,viter838 in self.success.items():
-        oprot.writeString(kiter837)
-        viter838.write(oprot)
+      for kiter858,viter859 in self.success.items():
+        oprot.writeString(kiter858)
+        viter859.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o2 is not None:
@@ -17089,11 +17147,11 @@ class get_fields_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype842, _size839) = iprot.readListBegin()
-          for _i843 in xrange(_size839):
-            _elem844 = FieldSchema()
-            _elem844.read(iprot)
-            self.success.append(_elem844)
+          (_etype863, _size860) = iprot.readListBegin()
+          for _i864 in xrange(_size860):
+            _elem865 = FieldSchema()
+            _elem865.read(iprot)
+            self.success.append(_elem865)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17128,8 +17186,8 @@ class get_fields_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter845 in self.success:
-        iter845.write(oprot)
+      for iter866 in self.success:
+        iter866.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17296,11 +17354,11 @@ class get_fields_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype849, _size846) = iprot.readListBegin()
-          for _i850 in xrange(_size846):
-            _elem851 = FieldSchema()
-            _elem851.read(iprot)
-            self.success.append(_elem851)
+          (_etype870, _size867) = iprot.readListBegin()
+          for _i871 in xrange(_size867):
+            _elem872 = FieldSchema()
+            _elem872.read(iprot)
+            self.success.append(_elem872)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17335,8 +17393,8 @@ class get_fields_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter852 in self.success:
-        iter852.write(oprot)
+      for iter873 in self.success:
+        iter873.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17489,11 +17547,11 @@ class get_schema_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype856, _size853) = iprot.readListBegin()
-          for _i857 in xrange(_size853):
-            _elem858 = FieldSchema()
-            _elem858.read(iprot)
-            self.success.append(_elem858)
+          (_etype877, _size874) = iprot.readListBegin()
+          for _i878 in xrange(_size874):
+            _elem879 = FieldSchema()
+            _elem879.read(iprot)
+            self.success.append(_elem879)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17528,8 +17586,8 @@ class get_schema_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter859 in self.success:
-        iter859.write(oprot)
+      for iter880 in self.success:
+        iter880.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -17696,11 +17754,11 @@ class get_schema_with_environment_context_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype863, _size860) = iprot.readListBegin()
-          for _i864 in xrange(_size860):
-            _elem865 = FieldSchema()
-            _elem865.read(iprot)
-            self.success.append(_elem865)
+          (_etype884, _size881) = iprot.readListBegin()
+          for _i885 in xrange(_size881):
+            _elem886 = FieldSchema()
+            _elem886.read(iprot)
+            self.success.append(_elem886)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -17735,8 +17793,8 @@ class get_schema_with_environment_context_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter866 in self.success:
-        iter866.write(oprot)
+      for iter887 in self.success:
+        iter887.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -18189,66 +18247,66 @@ class create_table_with_constraints_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.primaryKeys = []
-          (_etype870, _size867) = iprot.readListBegin()
-          for _i871 in xrange(_size867):
-            _elem872 = SQLPrimaryKey()
-            _elem872.read(iprot)
-            self.primaryKeys.append(_elem872)
+          (_etype891, _size888) = iprot.readListBegin()
+          for _i892 in xrange(_size888):
+            _elem893 = SQLPrimaryKey()
+            _elem893.read(iprot)
+            self.primaryKeys.append(_elem893)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.foreignKeys = []
-          (_etype876, _size873) = iprot.readListBegin()
-          for _i877 in xrange(_size873):
-            _elem878 = SQLForeignKey()
-            _elem878.read(iprot)
-            self.foreignKeys.append(_elem878)
+          (_etype897, _size894) = iprot.readListBegin()
+          for _i898 in xrange(_size894):
+            _elem899 = SQLForeignKey()
+            _elem899.read(iprot)
+            self.foreignKeys.append(_elem899)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.uniqueConstraints = []
-          (_etype882, _size879) = iprot.readListBegin()
-          for _i883 in xrange(_size879):
-            _elem884 = SQLUniqueConstraint()
-            _elem884.read(iprot)
-            self.uniqueConstraints.append(_elem884)
+          (_etype903, _size900) = iprot.readListBegin()
+          for _i904 in xrange(_size900):
+            _elem905 = SQLUniqueConstraint()
+            _elem905.read(iprot)
+            self.uniqueConstraints.append(_elem905)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.notNullConstraints = []
-          (_etype888, _size885) = iprot.readListBegin()
-          for _i889 in xrange(_size885):
-            _elem890 = SQLNotNullConstraint()
-            _elem890.read(iprot)
-            self.notNullConstraints.append(_elem890)
+          (_etype909, _size906) = iprot.readListBegin()
+          for _i910 in xrange(_size906):
+            _elem911 = SQLNotNullConstraint()
+            _elem911.read(iprot)
+            self.notNullConstraints.append(_elem911)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 6:
         if ftype == TType.LIST:
           self.defaultConstraints = []
-          (_etype894, _size891) = iprot.readListBegin()
-          for _i895 in xrange(_size891):
-            _elem896 = SQLDefaultConstraint()
-            _elem896.read(iprot)
-            self.defaultConstraints.append(_elem896)
+          (_etype915, _size912) = iprot.readListBegin()
+          for _i916 in xrange(_size912):
+            _elem917 = SQLDefaultConstraint()
+            _elem917.read(iprot)
+            self.defaultConstraints.append(_elem917)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.LIST:
           self.checkConstraints = []
-          (_etype900, _size897) = iprot.readListBegin()
-          for _i901 in xrange(_size897):
-            _elem902 = SQLCheckConstraint()
-            _elem902.read(iprot)
-            self.checkConstraints.append(_elem902)
+          (_etype921, _size918) = iprot.readListBegin()
+          for _i922 in xrange(_size918):
+            _elem923 = SQLCheckConstraint()
+            _elem923.read(iprot)
+            self.checkConstraints.append(_elem923)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18269,43 +18327,43 @@ class create_table_with_constraints_args:
     if self.primaryKeys is not None:
       oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-      for iter903 in self.primaryKeys:
-        iter903.write(oprot)
+      for iter924 in self.primaryKeys:
+        iter924.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.foreignKeys is not None:
       oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-      for iter904 in self.foreignKeys:
-        iter904.write(oprot)
+      for iter925 in self.foreignKeys:
+        iter925.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.uniqueConstraints is not None:
       oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-      for iter905 in self.uniqueConstraints:
-        iter905.write(oprot)
+      for iter926 in self.uniqueConstraints:
+        iter926.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.notNullConstraints is not None:
       oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-      for iter906 in self.notNullConstraints:
-        iter906.write(oprot)
+      for iter927 in self.notNullConstraints:
+        iter927.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.defaultConstraints is not None:
       oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
       oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-      for iter907 in self.defaultConstraints:
-        iter907.write(oprot)
+      for iter928 in self.defaultConstraints:
+        iter928.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.checkConstraints is not None:
       oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
       oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-      for iter908 in self.checkConstraints:
-        iter908.write(oprot)
+      for iter929 in self.checkConstraints:
+        iter929.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19865,10 +19923,10 @@ class truncate_table_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype912, _size909) = iprot.readListBegin()
-          for _i913 in xrange(_size909):
-            _elem914 = iprot.readString()
-            self.partNames.append(_elem914)
+          (_etype933, _size930) = iprot.readListBegin()
+          for _i934 in xrange(_size930):
+            _elem935 = iprot.readString()
+            self.partNames.append(_elem935)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19893,8 +19951,8 @@ class truncate_table_args:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter915 in self.partNames:
-        oprot.writeString(iter915)
+      for iter936 in self.partNames:
+        oprot.writeString(iter936)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20094,10 +20152,10 @@ class get_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype919, _size916) = iprot.readListBegin()
-          for _i920 in xrange(_size916):
-            _elem921 = iprot.readString()
-            self.success.append(_elem921)
+          (_etype940, _size937) = iprot.readListBegin()
+          for _i941 in xrange(_size937):
+            _elem942 = iprot.readString()
+            self.success.append(_elem942)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20120,8 +20178,8 @@ class get_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter922 in self.success:
-        oprot.writeString(iter922)
+      for iter943 in self.success:
+        oprot.writeString(iter943)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20271,10 +20329,10 @@ class get_tables_by_type_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype926, _size923) = iprot.readListBegin()
-          for _i927 in xrange(_size923):
-            _elem928 = iprot.readString()
-            self.success.append(_elem928)
+          (_etype947, _size944) = iprot.readListBegin()
+          for _i948 in xrange(_size944):
+            _elem949 = iprot.readString()
+            self.success.append(_elem949)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20297,8 +20355,8 @@ class get_tables_by_type_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter929 in self.success:
-        oprot.writeString(iter929)
+      for iter950 in self.success:
+        oprot.writeString(iter950)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20422,10 +20480,10 @@ class get_materialized_views_for_rewriting_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype933, _size930) = iprot.readListBegin()
-          for _i934 in xrange(_size930):
-            _elem935 = iprot.readString()
-            self.success.append(_elem935)
+          (_etype954, _size951) = iprot.readListBegin()
+          for _i955 in xrange(_size951):
+            _elem956 = iprot.readString()
+            self.success.append(_elem956)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20448,8 +20506,8 @@ class get_materialized_views_for_rewriting_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter936 in self.success:
-        oprot.writeString(iter936)
+      for iter957 in self.success:
+        oprot.writeString(iter957)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20522,10 +20580,10 @@ class get_table_meta_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.tbl_types = []
-          (_etype940, _size937) = iprot.readListBegin()
-          for _i941 in xrange(_size937):
-            _elem942 = iprot.readString()
-            self.tbl_types.append(_elem942)
+          (_etype961, _size958) = iprot.readListBegin()
+          for _i962 in xrange(_size958):
+            _elem963 = iprot.readString()
+            self.tbl_types.append(_elem963)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20550,8 +20608,8 @@ class get_table_meta_args:
     if self.tbl_types is not None:
       oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-      for iter943 in self.tbl_types:
-        oprot.writeString(iter943)
+      for iter964 in self.tbl_types:
+        oprot.writeString(iter964)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20607,11 +20665,11 @@ class get_table_meta_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype947, _size944) = iprot.readListBegin()
-          for _i948 in xrange(_size944):
-            _elem949 = TableMeta()
-            _elem949.read(iprot)
-            self.success.append(_elem949)
+          (_etype968, _size965) = iprot.readListBegin()
+          for _i969 in xrange(_size965):
+            _elem970 = TableMeta()
+            _elem970.read(iprot)
+            self.success.append(_elem970)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20634,8 +20692,8 @@ class get_table_meta_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter950 in self.success:
-        iter950.write(oprot)
+      for iter971 in self.success:
+        iter971.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -20759,10 +20817,10 @@ class get_all_tables_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype954, _size951) = iprot.readListBegin()
-          for _i955 in xrange(_size951):
-            _elem956 = iprot.readString()
-            self.success.append(_elem956)
+          (_etype975, _size972) = iprot.readListBegin()
+          for _i976 in xrange(_size972):
+            _elem977 = iprot.readString()
+            self.success.append(_elem977)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20785,8 +20843,8 @@ class get_all_tables_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter957 in self.success:
-        oprot.writeString(iter957)
+      for iter978 in self.success:
+        oprot.writeString(iter978)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -21022,10 +21080,10 @@ class get_table_objects_by_name_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype961, _size958) = iprot.readListBegin()
-          for _i962 in xrange(_size958):
-            _elem963 = iprot.readString()
-            self.tbl_names.append(_elem963)
+          (_etype982, _size979) = iprot.readListBegin()
+          for _i983 in xrange(_size979):
+            _elem984 = iprot.readString()
+            self.tbl_names.append(_elem984)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21046,8 +21104,8 @@ class get_table_objects_by_name_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter964 in self.tbl_names:
-        oprot.writeString(iter964)
+      for iter985 in self.tbl_names:
+        oprot.writeString(iter985)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21099,11 +21157,11 @@ class get_table_objects_by_name_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype968, _size965) = iprot.readListBegin()
-          for _i969 in xrange(_size965):
-            _elem970 = Table()
-            _elem970.read(iprot)
-            self.success.append(_elem970)
+          (_etype989, _size986) = iprot.readListBegin()
+          for _i990 in xrange(_size986):
+            _elem991 = Table()
+            _elem991.read(iprot)
+            self.success.append(_elem991)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21120,8 +21178,8 @@ class get_table_objects_by_name_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter971 in self.success:
-        iter971.write(oprot)
+      for iter992 in self.success:
+        iter992.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21513,10 +21571,10 @@ class get_materialization_invalidation_info_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tbl_names = []
-          (_etype975, _size972) = iprot.readListBegin()
-          for _i976 in xrange(_size972):
-            _elem977 = iprot.readString()
-            self.tbl_names.append(_elem977)
+          (_etype996, _size993) = iprot.readListBegin()
+          for _i997 in xrange(_size993):
+            _elem998 = iprot.readString()
+            self.tbl_names.append(_elem998)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21537,8 +21595,8 @@ class get_materialization_invalidation_info_args:
     if self.tbl_names is not None:
       oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-      for iter978 in self.tbl_names:
-        oprot.writeString(iter978)
+      for iter999 in self.tbl_names:
+        oprot.writeString(iter999)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -21599,12 +21657,12 @@ class get_materialization_invalidation_info_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype980, _vtype981, _size979 ) = iprot.readMapBegin()
-          for _i983 in xrange(_size979):
-            _key984 = iprot.readString()
-            _val985 = Materialization()
-            _val985.read(iprot)
-            self.success[_key984] = _val985
+          (_ktype1001, _vtype1002, _size1000 ) = iprot.readMapBegin()
+          for _i1004 in xrange(_size1000):
+            _key1005 = iprot.readString()
+            _val1006 = Materialization()
+            _val1006.read(iprot)
+            self.success[_key1005] = _val1006
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -21639,9 +21697,9 @@ class get_materialization_invalidation_info_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-      for kiter986,viter987 in self.success.items():
-        oprot.writeString(kiter986)
-        viter987.write(oprot)
+      for kiter1007,viter1008 in self.success.items():
+        oprot.writeString(kiter1007)
+        viter1008.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -22006,10 +22064,10 @@ class get_table_names_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype991, _size988) = iprot.readListBegin()
-          for _i992 in xrange(_size988):
-            _elem993 = iprot.readString()
-            self.success.append(_elem993)
+          (_etype1012, _size1009) = iprot.readListBegin()
+          for _i1013 in xrange(_size1009):
+            _elem1014 = iprot.readString()
+            self.success.append(_elem1014)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -22044,8 +22102,8 @@ class get_table_names_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter994 in self.success:
-        oprot.writeString(iter994)
+      for iter1015 in self.success:
+        oprot.writeString(iter1015)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -23015,11 +23073,11 @@ class add_partitions_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype998, _size995) = iprot.readListBegin()
-          for _i999 in xrange(_size995):
-            _elem1000 = Partition()
-            _elem1000.read(iprot)
-            self.new_parts.append(_elem1000)
+          (_etype1019, _size1016) = iprot.readListBegin()
+          for _i1020 in xrange(_size1016):
+            _elem1021 = Partition()
+            _elem1021.read(iprot)
+            self.new_parts.append(_elem1021)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23036,8 +23094,8 @@ class add_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1001 in self.new_parts:
-        iter1001.write(oprot)
+      for iter1022 in self.new_parts:
+        iter1022.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23195,11 +23253,11 @@ class add_partitions_pspec_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1005, _size1002) = iprot.readListBegin()
-          for _i1006 in xrange(_size1002):
-            _elem1007 = PartitionSpec()
-            _elem1007.read(iprot)
-            self.new_parts.append(_elem1007)
+          (_etype1026, _size1023) = iprot.readListBegin()
+          for _i1027 in xrange(_size1023):
+            _elem1028 = PartitionSpec()
+            _elem1028.read(iprot)
+            self.new_parts.append(_elem1028)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23216,8 +23274,8 @@ class add_partitions_pspec_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1008 in self.new_parts:
-        iter1008.write(oprot)
+      for iter1029 in self.new_parts:
+        iter1029.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23391,10 +23449,10 @@ class append_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1012, _size1009) = iprot.readListBegin()
-          for _i1013 in xrange(_size1009):
-            _elem1014 = iprot.readString()
-            self.part_vals.append(_elem1014)
+          (_etype1033, _size1030) = iprot.readListBegin()
+          for _i1034 in xrange(_size1030):
+            _elem1035 = iprot.readString()
+            self.part_vals.append(_elem1035)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23419,8 +23477,8 @@ class append_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1015 in self.part_vals:
-        oprot.writeString(iter1015)
+      for iter1036 in self.part_vals:
+        oprot.writeString(iter1036)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -23773,10 +23831,10 @@ class append_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1019, _size1016) = iprot.readListBegin()
-          for _i1020 in xrange(_size1016):
-            _elem1021 = iprot.readString()
-            self.part_vals.append(_elem1021)
+          (_etype1040, _size1037) = iprot.readListBegin()
+          for _i1041 in xrange(_size1037):
+            _elem1042 = iprot.readString()
+            self.part_vals.append(_elem1042)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -23807,8 +23865,8 @@ class append_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1022 in self.part_vals:
-        oprot.writeString(iter1022)
+      for iter1043 in self.part_vals:
+        oprot.writeString(iter1043)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -24403,10 +24461,10 @@ class drop_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1026, _size1023) = iprot.readListBegin()
-          for _i1027 in xrange(_size1023):
-            _elem1028 = iprot.readString()
-            self.part_vals.append(_elem1028)
+          (_etype1047, _size1044) = iprot.readListBegin()
+          for _i1048 in xrange(_size1044):
+            _elem1049 = iprot.readString()
+            self.part_vals.append(_elem1049)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24436,8 +24494,8 @@ class drop_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1029 in self.part_vals:
-        oprot.writeString(iter1029)
+      for iter1050 in self.part_vals:
+        oprot.writeString(iter1050)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -24610,10 +24668,10 @@ class drop_partition_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1033, _size1030) = iprot.readListBegin()
-          for _i1034 in xrange(_size1030):
-            _elem1035 = iprot.readString()
-            self.part_vals.append(_elem1035)
+          (_etype1054, _size1051) = iprot.readListBegin()
+          for _i1055 in xrange(_size1051):
+            _elem1056 = iprot.readString()
+            self.part_vals.append(_elem1056)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -24649,8 +24707,8 @@ class drop_partition_with_environment_context_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1036 in self.part_vals:
-        oprot.writeString(iter1036)
+      for iter1057 in self.part_vals:
+        oprot.writeString(iter1057)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.deleteData is not None:
@@ -25387,10 +25445,10 @@ class get_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1040, _size1037) = iprot.readListBegin()
-          for _i1041 in xrange(_size1037):
-            _elem1042 = iprot.readString()
-            self.part_vals.append(_elem1042)
+          (_etype1061, _size1058) = iprot.readListBegin()
+          for _i1062 in xrange(_size1058):
+            _elem1063 = iprot.readString()
+            self.part_vals.append(_elem1063)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25415,8 +25473,8 @@ class get_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1043 in self.part_vals:
-        oprot.writeString(iter1043)
+      for iter1064 in self.part_vals:
+        oprot.writeString(iter1064)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -25575,11 +25633,11 @@ class exchange_partition_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1045, _vtype1046, _size1044 ) = iprot.readMapBegin()
-          for _i1048 in xrange(_size1044):
-            _key1049 = iprot.readString()
-            _val1050 = iprot.readString()
-            self.partitionSpecs[_key1049] = _val1050
+          (_ktype1066, _vtype1067, _size1065 ) = iprot.readMapBegin()
+          for _i1069 in xrange(_size1065):
+            _key1070 = iprot.readString()
+            _val1071 = iprot.readString()
+            self.partitionSpecs[_key1070] = _val1071
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25616,9 +25674,9 @@ class exchange_partition_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter1051,viter1052 in self.partitionSpecs.items():
-        oprot.writeString(kiter1051)
-        oprot.writeString(viter1052)
+      for kiter1072,viter1073 in self.partitionSpecs.items():
+        oprot.writeString(kiter1072)
+        oprot.writeString(viter1073)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25823,11 +25881,11 @@ class exchange_partitions_args:
       if fid == 1:
         if ftype == TType.MAP:
           self.partitionSpecs = {}
-          (_ktype1054, _vtype1055, _size1053 ) = iprot.readMapBegin()
-          for _i1057 in xrange(_size1053):
-            _key1058 = iprot.readString()
-            _val1059 = iprot.readString()
-            self.partitionSpecs[_key1058] = _val1059
+          (_ktype1075, _vtype1076, _size1074 ) = iprot.readMapBegin()
+          for _i1078 in xrange(_size1074):
+            _key1079 = iprot.readString()
+            _val1080 = iprot.readString()
+            self.partitionSpecs[_key1079] = _val1080
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -25864,9 +25922,9 @@ class exchange_partitions_args:
     if self.partitionSpecs is not None:
       oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-      for kiter1060,viter1061 in self.partitionSpecs.items():
-        oprot.writeString(kiter1060)
-        oprot.writeString(viter1061)
+      for kiter1081,viter1082 in self.partitionSpecs.items():
+        oprot.writeString(kiter1081)
+        oprot.writeString(viter1082)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.source_db is not None:
@@ -25949,11 +26007,11 @@ class exchange_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1065, _size1062) = iprot.readListBegin()
-          for _i1066 in xrange(_size1062):
-            _elem1067 = Partition()
-            _elem1067.read(iprot)
-            self.success.append(_elem1067)
+          (_etype1086, _size1083) = iprot.readListBegin()
+          for _i1087 in xrange(_size1083):
+            _elem1088 = Partition()
+            _elem1088.read(iprot)
+            self.success.append(_elem1088)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -25994,8 +26052,8 @@ class exchange_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1068 in self.success:
-        iter1068.write(oprot)
+      for iter1089 in self.success:
+        iter1089.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26089,10 +26147,10 @@ class get_partition_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1072, _size1069) = iprot.readListBegin()
-          for _i1073 in xrange(_size1069):
-            _elem1074 = iprot.readString()
-            self.part_vals.append(_elem1074)
+          (_etype1093, _size1090) = iprot.readListBegin()
+          for _i1094 in xrange(_size1090):
+            _elem1095 = iprot.readString()
+            self.part_vals.append(_elem1095)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26104,10 +26162,10 @@ class get_partition_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1078, _size1075) = iprot.readListBegin()
-          for _i1079 in xrange(_size1075):
-            _elem1080 = iprot.readString()
-            self.group_names.append(_elem1080)
+          (_etype1099, _size1096) = iprot.readListBegin()
+          for _i1100 in xrange(_size1096):
+            _elem1101 = iprot.readString()
+            self.group_names.append(_elem1101)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26132,8 +26190,8 @@ class get_partition_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1081 in self.part_vals:
-        oprot.writeString(iter1081)
+      for iter1102 in self.part_vals:
+        oprot.writeString(iter1102)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.user_name is not None:
@@ -26143,8 +26201,8 @@ class get_partition_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1082 in self.group_names:
-        oprot.writeString(iter1082)
+      for iter1103 in self.group_names:
+        oprot.writeString(iter1103)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26573,11 +26631,11 @@ class get_partitions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1086, _size1083) = iprot.readListBegin()
-          for _i1087 in xrange(_size1083):
-            _elem1088 = Partition()
-            _elem1088.read(iprot)
-            self.success.append(_elem1088)
+          (_etype1107, _size1104) = iprot.readListBegin()
+          for _i1108 in xrange(_size1104):
+            _elem1109 = Partition()
+            _elem1109.read(iprot)
+            self.success.append(_elem1109)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26606,8 +26664,8 @@ class get_partitions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1089 in self.success:
-        iter1089.write(oprot)
+      for iter1110 in self.success:
+        iter1110.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26701,10 +26759,10 @@ class get_partitions_with_auth_args:
       elif fid == 5:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1093, _size1090) = iprot.readListBegin()
-          for _i1094 in xrange(_size1090):
-            _elem1095 = iprot.readString()
-            self.group_names.append(_elem1095)
+          (_etype1114, _size1111) = iprot.readListBegin()
+          for _i1115 in xrange(_size1111):
+            _elem1116 = iprot.readString()
+            self.group_names.append(_elem1116)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26737,8 +26795,8 @@ class get_partitions_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1096 in self.group_names:
-        oprot.writeString(iter1096)
+      for iter1117 in self.group_names:
+        oprot.writeString(iter1117)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -26799,11 +26857,11 @@ class get_partitions_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1100, _size1097) = iprot.readListBegin()
-          for _i1101 in xrange(_size1097):
-            _elem1102 = Partition()
-            _elem1102.read(iprot)
-            self.success.append(_elem1102)
+          (_etype1121, _size1118) = iprot.readListBegin()
+          for _i1122 in xrange(_size1118):
+            _elem1123 = Partition()
+            _elem1123.read(iprot)
+            self.success.append(_elem1123)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -26832,8 +26890,8 @@ class get_partitions_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1103 in self.success:
-        iter1103.write(oprot)
+      for iter1124 in self.success:
+        iter1124.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -26991,11 +27049,11 @@ class get_partitions_pspec_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1107, _size1104) = iprot.readListBegin()
-          for _i1108 in xrange(_size1104):
-            _elem1109 = PartitionSpec()
-            _elem1109.read(iprot)
-            self.success.append(_elem1109)
+          (_etype1128, _size1125) = iprot.readListBegin()
+          for _i1129 in xrange(_size1125):
+            _elem1130 = PartitionSpec()
+            _elem1130.read(iprot)
+            self.success.append(_elem1130)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27024,8 +27082,8 @@ class get_partitions_pspec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1110 in self.success:
-        iter1110.write(oprot)
+      for iter1131 in self.success:
+        iter1131.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27183,10 +27241,10 @@ class get_partition_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1114, _size1111) = iprot.readListBegin()
-          for _i1115 in xrange(_size1111):
-            _elem1116 = iprot.readString()
-            self.success.append(_elem1116)
+          (_etype1135, _size1132) = iprot.readListBegin()
+          for _i1136 in xrange(_size1132):
+            _elem1137 = iprot.readString()
+            self.success.append(_elem1137)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27215,8 +27273,8 @@ class get_partition_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1117 in self.success:
-        oprot.writeString(iter1117)
+      for iter1138 in self.success:
+        oprot.writeString(iter1138)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27456,10 +27514,10 @@ class get_partitions_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1121, _size1118) = iprot.readListBegin()
-          for _i1122 in xrange(_size1118):
-            _elem1123 = iprot.readString()
-            self.part_vals.append(_elem1123)
+          (_etype1142, _size1139) = iprot.readListBegin()
+          for _i1143 in xrange(_size1139):
+            _elem1144 = iprot.readString()
+            self.part_vals.append(_elem1144)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27489,8 +27547,8 @@ class get_partitions_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1124 in self.part_vals:
-        oprot.writeString(iter1124)
+      for iter1145 in self.part_vals:
+        oprot.writeString(iter1145)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27554,11 +27612,11 @@ class get_partitions_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1128, _size1125) = iprot.readListBegin()
-          for _i1129 in xrange(_size1125):
-            _elem1130 = Partition()
-            _elem1130.read(iprot)
-            self.success.append(_elem1130)
+          (_etype1149, _size1146) = iprot.readListBegin()
+          for _i1150 in xrange(_size1146):
+            _elem1151 = Partition()
+            _elem1151.read(iprot)
+            self.success.append(_elem1151)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27587,8 +27645,8 @@ class get_partitions_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1131 in self.success:
-        iter1131.write(oprot)
+      for iter1152 in self.success:
+        iter1152.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27675,10 +27733,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1135, _size1132) = iprot.readListBegin()
-          for _i1136 in xrange(_size1132):
-            _elem1137 = iprot.readString()
-            self.part_vals.append(_elem1137)
+          (_etype1156, _size1153) = iprot.readListBegin()
+          for _i1157 in xrange(_size1153):
+            _elem1158 = iprot.readString()
+            self.part_vals.append(_elem1158)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27695,10 +27753,10 @@ class get_partitions_ps_with_auth_args:
       elif fid == 6:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1141, _size1138) = iprot.readListBegin()
-          for _i1142 in xrange(_size1138):
-            _elem1143 = iprot.readString()
-            self.group_names.append(_elem1143)
+          (_etype1162, _size1159) = iprot.readListBegin()
+          for _i1163 in xrange(_size1159):
+            _elem1164 = iprot.readString()
+            self.group_names.append(_elem1164)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27723,8 +27781,8 @@ class get_partitions_ps_with_auth_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1144 in self.part_vals:
-        oprot.writeString(iter1144)
+      for iter1165 in self.part_vals:
+        oprot.writeString(iter1165)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -27738,8 +27796,8 @@ class get_partitions_ps_with_auth_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1145 in self.group_names:
-        oprot.writeString(iter1145)
+      for iter1166 in self.group_names:
+        oprot.writeString(iter1166)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -27801,11 +27859,11 @@ class get_partitions_ps_with_auth_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1149, _size1146) = iprot.readListBegin()
-          for _i1150 in xrange(_size1146):
-            _elem1151 = Partition()
-            _elem1151.read(iprot)
-            self.success.append(_elem1151)
+          (_etype1170, _size1167) = iprot.readListBegin()
+          for _i1171 in xrange(_size1167):
+            _elem1172 = Partition()
+            _elem1172.read(iprot)
+            self.success.append(_elem1172)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27834,8 +27892,8 @@ class get_partitions_ps_with_auth_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1152 in self.success:
-        iter1152.write(oprot)
+      for iter1173 in self.success:
+        iter1173.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -27916,10 +27974,10 @@ class get_partition_names_ps_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1156, _size1153) = iprot.readListBegin()
-          for _i1157 in xrange(_size1153):
-            _elem1158 = iprot.readString()
-            self.part_vals.append(_elem1158)
+          (_etype1177, _size1174) = iprot.readListBegin()
+          for _i1178 in xrange(_size1174):
+            _elem1179 = iprot.readString()
+            self.part_vals.append(_elem1179)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -27949,8 +28007,8 @@ class get_partition_names_ps_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1159 in self.part_vals:
-        oprot.writeString(iter1159)
+      for iter1180 in self.part_vals:
+        oprot.writeString(iter1180)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.max_parts is not None:
@@ -28014,10 +28072,10 @@ class get_partition_names_ps_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1163, _size1160) = iprot.readListBegin()
-          for _i1164 in xrange(_size1160):
-            _elem1165 = iprot.readString()
-            self.success.append(_elem1165)
+          (_etype1184, _size1181) = iprot.readListBegin()
+          for _i1185 in xrange(_size1181):
+            _elem1186 = iprot.readString()
+            self.success.append(_elem1186)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28046,8 +28104,8 @@ class get_partition_names_ps_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1166 in self.success:
-        oprot.writeString(iter1166)
+      for iter1187 in self.success:
+        oprot.writeString(iter1187)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28218,11 +28276,11 @@ class get_partitions_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1170, _size1167) = iprot.readListBegin()
-          for _i1171 in xrange(_size1167):
-            _elem1172 = Partition()
-            _elem1172.read(iprot)
-            self.success.append(_elem1172)
+          (_etype1191, _size1188) = iprot.readListBegin()
+          for _i1192 in xrange(_size1188):
+            _elem1193 = Partition()
+            _elem1193.read(iprot)
+            self.success.append(_elem1193)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28251,8 +28309,8 @@ class get_partitions_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1173 in self.success:
-        iter1173.write(oprot)
+      for iter1194 in self.success:
+        iter1194.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28423,11 +28481,11 @@ class get_part_specs_by_filter_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1177, _size1174) = iprot.readListBegin()
-          for _i1178 in xrange(_size1174):
-            _elem1179 = PartitionSpec()
-            _elem1179.read(iprot)
-            self.success.append(_elem1179)
+          (_etype1198, _size1195) = iprot.readListBegin()
+          for _i1199 in xrange(_size1195):
+            _elem1200 = PartitionSpec()
+            _elem1200.read(iprot)
+            self.success.append(_elem1200)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28456,8 +28514,8 @@ class get_part_specs_by_filter_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1180 in self.success:
-        iter1180.write(oprot)
+      for iter1201 in self.success:
+        iter1201.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -28877,10 +28935,10 @@ class get_partitions_by_names_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.names = []
-          (_etype1184, _size1181) = iprot.readListBegin()
-          for _i1185 in xrange(_size1181):
-            _elem1186 = iprot.readString()
-            self.names.append(_elem1186)
+          (_etype1205, _size1202) = iprot.readListBegin()
+          for _i1206 in xrange(_size1202):
+            _elem1207 = iprot.readString()
+            self.names.append(_elem1207)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28905,8 +28963,8 @@ class get_partitions_by_names_args:
     if self.names is not None:
       oprot.writeFieldBegin('names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.names))
-      for iter1187 in self.names:
-        oprot.writeString(iter1187)
+      for iter1208 in self.names:
+        oprot.writeString(iter1208)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -28965,11 +29023,11 @@ class get_partitions_by_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1191, _size1188) = iprot.readListBegin()
-          for _i1192 in xrange(_size1188):
-            _elem1193 = Partition()
-            _elem1193.read(iprot)
-            self.success.append(_elem1193)
+          (_etype1212, _size1209) = iprot.readListBegin()
+          for _i1213 in xrange(_size1209):
+            _elem1214 = Partition()
+            _elem1214.read(iprot)
+            self.success.append(_elem1214)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -28998,8 +29056,8 @@ class get_partitions_by_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1194 in self.success:
-        iter1194.write(oprot)
+      for iter1215 in self.success:
+        iter1215.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -29249,11 +29307,11 @@ class alter_partitions_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1198, _size1195) = iprot.readListBegin()
-          for _i1199 in xrange(_size1195):
-            _elem1200 = Partition()
-            _elem1200.read(iprot)
-            self.new_parts.append(_elem1200)
+          (_etype1219, _size1216) = iprot.readListBegin()
+          for _i1220 in xrange(_size1216):
+            _elem1221 = Partition()
+            _elem1221.read(iprot)
+            self.new_parts.append(_elem1221)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29278,8 +29336,8 @@ class alter_partitions_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1201 in self.new_parts:
-        iter1201.write(oprot)
+      for iter1222 in self.new_parts:
+        iter1222.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -29432,11 +29490,11 @@ class alter_partitions_with_environment_context_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.new_parts = []
-          (_etype1205, _size1202) = iprot.readListBegin()
-          for _i1206 in xrange(_size1202):
-            _elem1207 = Partition()
-            _elem1207.read(iprot)
-            self.new_parts.append(_elem1207)
+          (_etype1226, _size1223) = iprot.readListBegin()
+          for _i1227 in xrange(_size1223):
+            _elem1228 = Partition()
+            _elem1228.read(iprot)
+            self.new_parts.append(_elem1228)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29467,8 +29525,8 @@ class alter_partitions_with_environment_context_args:
     if self.new_parts is not None:
       oprot.writeFieldBegin('new_parts', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-      for iter1208 in self.new_parts:
-        iter1208.write(oprot)
+      for iter1229 in self.new_parts:
+        iter1229.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.environment_context is not None:
@@ -29812,10 +29870,10 @@ class rename_partition_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1212, _size1209) = iprot.readListBegin()
-          for _i1213 in xrange(_size1209):
-            _elem1214 = iprot.readString()
-            self.part_vals.append(_elem1214)
+          (_etype1233, _size1230) = iprot.readListBegin()
+          for _i1234 in xrange(_size1230):
+            _elem1235 = iprot.readString()
+            self.part_vals.append(_elem1235)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -29846,8 +29904,8 @@ class rename_partition_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1215 in self.part_vals:
-        oprot.writeString(iter1215)
+      for iter1236 in self.part_vals:
+        oprot.writeString(iter1236)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.new_part is not None:
@@ -29989,10 +30047,10 @@ class partition_name_has_valid_characters_args:
       if fid == 1:
         if ftype == TType.LIST:
           self.part_vals = []
-          (_etype1219, _size1216) = iprot.readListBegin()
-          for _i1220 in xrange(_size1216):
-            _elem1221 = iprot.readString()
-            self.part_vals.append(_elem1221)
+          (_etype1240, _size1237) = iprot.readListBegin()
+          for _i1241 in xrange(_size1237):
+            _elem1242 = iprot.readString()
+            self.part_vals.append(_elem1242)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30014,8 +30072,8 @@ class partition_name_has_valid_characters_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.part_vals))
-      for iter1222 in self.part_vals:
-        oprot.writeString(iter1222)
+      for iter1243 in self.part_vals:
+        oprot.writeString(iter1243)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.throw_exception is not None:
@@ -30373,10 +30431,10 @@ class partition_name_to_vals_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1226, _size1223) = iprot.readListBegin()
-          for _i1227 in xrange(_size1223):
-            _elem1228 = iprot.readString()
-            self.success.append(_elem1228)
+          (_etype1247, _size1244) = iprot.readListBegin()
+          for _i1248 in xrange(_size1244):
+            _elem1249 = iprot.readString()
+            self.success.append(_elem1249)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -30399,8 +30457,8 @@ class partition_name_to_vals_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1229 in self.success:
-        oprot.writeString(iter1229)
+      for iter1250 in self.success:
+        oprot.writeString(iter1250)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30524,11 +30582,11 @@ class partition_name_to_spec_result:
       if fid == 0:
         if ftype == TType.MAP:
           self.success = {}
-          (_ktype1231, _vtype1232, _size1230 ) = iprot.readMapBegin()
-          for _i1234 in xrange(_size1230):
-            _key1235 = iprot.readString()
-            _val1236 = iprot.readString()
-            self.success[_key1235] = _val1236
+          (_ktype1252, _vtype1253, _size1251 ) = iprot.readMapBegin()
+          for _i1255 in xrange(_size1251):
+            _key1256 = iprot.readString()
+            _val1257 = iprot.readString()
+            self.success[_key1256] = _val1257
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30551,9 +30609,9 @@ class partition_name_to_spec_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.MAP, 0)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.success))
-      for kiter1237,viter1238 in self.success.items():
-        oprot.writeString(kiter1237)
-        oprot.writeString(viter1238)
+      for kiter1258,viter1259 in self.success.items():
+        oprot.writeString(kiter1258)
+        oprot.writeString(viter1259)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -30629,11 +30687,11 @@ class markPartitionForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1240, _vtype1241, _size1239 ) = iprot.readMapBegin()
-          for _i1243 in xrange(_size1239):
-            _key1244 = iprot.readString()
-            _val1245 = iprot.readString()
-            self.part_vals[_key1244] = _val1245
+          (_ktype1261, _vtype1262, _size1260 ) = iprot.readMapBegin()
+          for _i1264 in xrange(_size1260):
+            _key1265 = iprot.readString()
+            _val1266 = iprot.readString()
+            self.part_vals[_key1265] = _val1266
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30663,9 +30721,9 @@ class markPartitionForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1246,viter1247 in self.part_vals.items():
-        oprot.writeString(kiter1246)
-        oprot.writeString(viter1247)
+      for kiter1267,viter1268 in self.part_vals.items():
+        oprot.writeString(kiter1267)
+        oprot.writeString(viter1268)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -30879,11 +30937,11 @@ class isPartitionMarkedForEvent_args:
       elif fid == 3:
         if ftype == TType.MAP:
           self.part_vals = {}
-          (_ktype1249, _vtype1250, _size1248 ) = iprot.readMapBegin()
-          for _i1252 in xrange(_size1248):
-            _key1253 = iprot.readString()
-            _val1254 = iprot.readString()
-            self.part_vals[_key1253] = _val1254
+          (_ktype1270, _vtype1271, _size1269 ) = iprot.readMapBegin()
+          for _i1273 in xrange(_size1269):
+            _key1274 = iprot.readString()
+            _val1275 = iprot.readString()
+            self.part_vals[_key1274] = _val1275
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -30913,9 +30971,9 @@ class isPartitionMarkedForEvent_args:
     if self.part_vals is not None:
       oprot.writeFieldBegin('part_vals', TType.MAP, 3)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.part_vals))
-      for kiter1255,viter1256 in self.part_vals.items():
-        oprot.writeString(kiter1255)
-        oprot.writeString(viter1256)
+      for kiter1276,viter1277 in self.part_vals.items():
+        oprot.writeString(kiter1276)
+        oprot.writeString(viter1277)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.eventType is not None:
@@ -34567,10 +34625,10 @@ class get_functions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1260, _size1257) = iprot.readListBegin()
-          for _i1261 in xrange(_size1257):
-            _elem1262 = iprot.readString()
-            self.success.append(_elem1262)
+          (_etype1281, _size1278) = iprot.readListBegin()
+          for _i1282 in xrange(_size1278):
+            _elem1283 = iprot.readString()
+            self.success.append(_elem1283)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -34593,8 +34651,8 @@ class get_functions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1263 in self.success:
-        oprot.writeString(iter1263)
+      for iter1284 in self.success:
+        oprot.writeString(iter1284)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35282,10 +35340,10 @@ class get_role_names_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1267, _size1264) = iprot.readListBegin()
-          for _i1268 in xrange(_size1264):
-            _elem1269 = iprot.readString()
-            self.success.append(_elem1269)
+          (_etype1288, _size1285) = iprot.readListBegin()
+          for _i1289 in xrange(_size1285):
+            _elem1290 = iprot.readString()
+            self.success.append(_elem1290)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35308,8 +35366,8 @@ class get_role_names_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1270 in self.success:
-        oprot.writeString(iter1270)
+      for iter1291 in self.success:
+        oprot.writeString(iter1291)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -35823,11 +35881,11 @@ class list_roles_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1274, _size1271) = iprot.readListBegin()
-          for _i1275 in xrange(_size1271):
-            _elem1276 = Role()
-            _elem1276.read(iprot)
-            self.success.append(_elem1276)
+          (_etype1295, _size1292) = iprot.readListBegin()
+          for _i1296 in xrange(_size1292):
+            _elem1297 = Role()
+            _elem1297.read(iprot)
+            self.success.append(_elem1297)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -35850,8 +35908,8 @@ class list_roles_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1277 in self.success:
-        iter1277.write(oprot)
+      for iter1298 in self.success:
+        iter1298.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -36360,10 +36418,10 @@ class get_privilege_set_args:
       elif fid == 3:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1281, _size1278) = iprot.readListBegin()
-          for _i1282 in xrange(_size1278):
-            _elem1283 = iprot.readString()
-            self.group_names.append(_elem1283)
+          (_etype1302, _size1299) = iprot.readListBegin()
+          for _i1303 in xrange(_size1299):
+            _elem1304 = iprot.readString()
+            self.group_names.append(_elem1304)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36388,8 +36446,8 @@ class get_privilege_set_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1284 in self.group_names:
-        oprot.writeString(iter1284)
+      for iter1305 in self.group_names:
+        oprot.writeString(iter1305)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -36616,11 +36674,11 @@ class list_privileges_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1288, _size1285) = iprot.readListBegin()
-          for _i1289 in xrange(_size1285):
-            _elem1290 = HiveObjectPrivilege()
-            _elem1290.read(iprot)
-            self.success.append(_elem1290)
+          (_etype1309, _size1306) = iprot.readListBegin()
+          for _i1310 in xrange(_size1306):
+            _elem1311 = HiveObjectPrivilege()
+            _elem1311.read(iprot)
+            self.success.append(_elem1311)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -36643,8 +36701,8 @@ class list_privileges_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1291 in self.success:
-        iter1291.write(oprot)
+      for iter1312 in self.success:
+        iter1312.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -37314,10 +37372,10 @@ class set_ugi_args:
       elif fid == 2:
         if ftype == TType.LIST:
           self.group_names = []
-          (_etype1295, _size1292) = iprot.readListBegin()
-          for _i1296 in xrange(_size1292):
-            _elem1297 = iprot.readString()
-            self.group_names.append(_elem1297)
+          (_etype1316, _size1313) = iprot.readListBegin()
+          for _i1317 in xrange(_size1313):
+            _elem1318 = iprot.readString()
+            self.group_names.append(_elem1318)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37338,8 +37396,8 @@ class set_ugi_args:
     if self.group_names is not None:
       oprot.writeFieldBegin('group_names', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.group_names))
-      for iter1298 in self.group_names:
-        oprot.writeString(iter1298)
+      for iter1319 in self.group_names:
+        oprot.writeString(iter1319)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -37394,10 +37452,10 @@ class set_ugi_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1302, _size1299) = iprot.readListBegin()
-          for _i1303 in xrange(_size1299):
-            _elem1304 = iprot.readString()
-            self.success.append(_elem1304)
+          (_etype1323, _size1320) = iprot.readListBegin()
+          for _i1324 in xrange(_size1320):
+            _elem1325 = iprot.readString()
+            self.success.append(_elem1325)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -37420,8 +37478,8 @@ class set_ugi_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1305 in self.success:
-        oprot.writeString(iter1305)
+      for iter1326 in self.success:
+        oprot.writeString(iter1326)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -38353,10 +38411,10 @@ class get_all_token_identifiers_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1309, _size1306) = iprot.readListBegin()
-          for _i1310 in xrange(_size1306):
-            _elem1311 = iprot.readString()
-            self.success.append(_elem1311)
+          (_etype1330, _size1327) = iprot.readListBegin()
+          for _i1331 in xrange(_size1327):
+            _elem1332 = iprot.readString()
+            self.success.append(_elem1332)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38373,8 +38431,8 @@ class get_all_token_identifiers_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1312 in self.success:
-        oprot.writeString(iter1312)
+      for iter1333 in self.success:
+        oprot.writeString(iter1333)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -38901,10 +38959,10 @@ class get_master_keys_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1316, _size1313) = iprot.readListBegin()
-          for _i1317 in xrange(_size1313):
-            _elem1318 = iprot.readString()
-            self.success.append(_elem1318)
+          (_etype1337, _size1334) = iprot.readListBegin()
+          for _i1338 in xrange(_size1334):
+            _elem1339 = iprot.readString()
+            self.success.append(_elem1339)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -38921,8 +38979,8 @@ class get_master_keys_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRING, len(self.success))
-      for iter1319 in self.success:
-        oprot.writeString(iter1319)
+      for iter1340 in self.success:
+        oprot.writeString(iter1340)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -42171,6 +42229,133 @@ class flushCache_result:
   def __ne__(self, other):
     return not (self == other)
 
+class add_write_notification_log_args:
+  """
+  Attributes:
+   - rqst
+  """
+
+  thrift_spec = None
+  def __init__(self, rqst=None,):
+    self.rqst = rqst
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == -1:
+        if ftype == TType.STRUCT:
+          self.rqst = WriteNotificationLogRequest()
+          self.rqst.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('add_write_notification_log_args')
+    if self.rqst is not None:
+      oprot.writeFieldBegin('rqst', TType.STRUCT, -1)
+      self.rqst.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.rqst)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class add_write_notification_log_result:
+  """
+  Attributes:
+   - success
+  """
+
+  thrift_spec = (
+    (0, TType.STRUCT, 'success', (WriteNotificationLogResponse, WriteNotificationLogResponse.thrift_spec), None, ), # 0
+  )
+
+  def __init__(self, success=None,):
+    self.success = success
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 0:
+        if ftype == TType.STRUCT:
+          self.success = WriteNotificationLogResponse()
+          self.success.read(iprot)
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('add_write_notification_log_result')
+    if self.success is not None:
+      oprot.writeFieldBegin('success', TType.STRUCT, 0)
+      self.success.write(oprot)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.success)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class cm_recycle_args:
   """
   Attributes:
@@ -47202,11 +47387,11 @@ class get_schema_all_versions_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1323, _size1320) = iprot.readListBegin()
-          for _i1324 in xrange(_size1320):
-            _elem1325 = SchemaVersion()
-            _elem1325.read(iprot)
-            self.success.append(_elem1325)
+          (_etype1344, _size1341) = iprot.readListBegin()
+          for _i1345 in xrange(_size1341):
+            _elem1346 = SchemaVersion()
+            _elem1346.read(iprot)
+            self.success.append(_elem1346)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -47235,8 +47420,8 @@ class get_schema_all_versions_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1326 in self.success:
-        iter1326.write(oprot)
+      for iter1347 in self.success:
+        iter1347.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:
@@ -48711,11 +48896,11 @@ class get_runtime_stats_result:
       if fid == 0:
         if ftype == TType.LIST:
           self.success = []
-          (_etype1330, _size1327) = iprot.readListBegin()
-          for _i1331 in xrange(_size1327):
-            _elem1332 = RuntimeStat()
-            _elem1332.read(iprot)
-            self.success.append(_elem1332)
+          (_etype1351, _size1348) = iprot.readListBegin()
+          for _i1352 in xrange(_size1348):
+            _elem1353 = RuntimeStat()
+            _elem1353.read(iprot)
+            self.success.append(_elem1353)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -48738,8 +48923,8 @@ class get_runtime_stats_result:
     if self.success is not None:
       oprot.writeFieldBegin('success', TType.LIST, 0)
       oprot.writeListBegin(TType.STRUCT, len(self.success))
-      for iter1333 in self.success:
-        iter1333.write(oprot)
+      for iter1354 in self.success:
+        iter1354.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.o1 is not None:


[20/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 11affe3..031e72b 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -11562,17 +11562,20 @@ class CommitTxnRequest:
   Attributes:
    - txnid
    - replPolicy
+   - writeEventInfos
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.I64, 'txnid', None, None, ), # 1
     (2, TType.STRING, 'replPolicy', None, None, ), # 2
+    (3, TType.LIST, 'writeEventInfos', (TType.STRUCT,(WriteEventInfo, WriteEventInfo.thrift_spec)), None, ), # 3
   )
 
-  def __init__(self, txnid=None, replPolicy=None,):
+  def __init__(self, txnid=None, replPolicy=None, writeEventInfos=None,):
     self.txnid = txnid
     self.replPolicy = replPolicy
+    self.writeEventInfos = writeEventInfos
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -11593,6 +11596,17 @@ class CommitTxnRequest:
           self.replPolicy = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.LIST:
+          self.writeEventInfos = []
+          (_etype526, _size523) = iprot.readListBegin()
+          for _i527 in xrange(_size523):
+            _elem528 = WriteEventInfo()
+            _elem528.read(iprot)
+            self.writeEventInfos.append(_elem528)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -11611,6 +11625,13 @@ class CommitTxnRequest:
       oprot.writeFieldBegin('replPolicy', TType.STRING, 2)
       oprot.writeString(self.replPolicy)
       oprot.writeFieldEnd()
+    if self.writeEventInfos is not None:
+      oprot.writeFieldBegin('writeEventInfos', TType.LIST, 3)
+      oprot.writeListBegin(TType.STRUCT, len(self.writeEventInfos))
+      for iter529 in self.writeEventInfos:
+        iter529.write(oprot)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -11624,6 +11645,158 @@ class CommitTxnRequest:
     value = 17
     value = (value * 31) ^ hash(self.txnid)
     value = (value * 31) ^ hash(self.replPolicy)
+    value = (value * 31) ^ hash(self.writeEventInfos)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class WriteEventInfo:
+  """
+  Attributes:
+   - writeId
+   - database
+   - table
+   - files
+   - partition
+   - tableObj
+   - partitionObj
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'writeId', None, None, ), # 1
+    (2, TType.STRING, 'database', None, None, ), # 2
+    (3, TType.STRING, 'table', None, None, ), # 3
+    (4, TType.STRING, 'files', None, None, ), # 4
+    (5, TType.STRING, 'partition', None, None, ), # 5
+    (6, TType.STRING, 'tableObj', None, None, ), # 6
+    (7, TType.STRING, 'partitionObj', None, None, ), # 7
+  )
+
+  def __init__(self, writeId=None, database=None, table=None, files=None, partition=None, tableObj=None, partitionObj=None,):
+    self.writeId = writeId
+    self.database = database
+    self.table = table
+    self.files = files
+    self.partition = partition
+    self.tableObj = tableObj
+    self.partitionObj = partitionObj
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I64:
+          self.writeId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.STRING:
+          self.database = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.table = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.files = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRING:
+          self.partition = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.STRING:
+          self.tableObj = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.STRING:
+          self.partitionObj = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WriteEventInfo')
+    if self.writeId is not None:
+      oprot.writeFieldBegin('writeId', TType.I64, 1)
+      oprot.writeI64(self.writeId)
+      oprot.writeFieldEnd()
+    if self.database is not None:
+      oprot.writeFieldBegin('database', TType.STRING, 2)
+      oprot.writeString(self.database)
+      oprot.writeFieldEnd()
+    if self.table is not None:
+      oprot.writeFieldBegin('table', TType.STRING, 3)
+      oprot.writeString(self.table)
+      oprot.writeFieldEnd()
+    if self.files is not None:
+      oprot.writeFieldBegin('files', TType.STRING, 4)
+      oprot.writeString(self.files)
+      oprot.writeFieldEnd()
+    if self.partition is not None:
+      oprot.writeFieldBegin('partition', TType.STRING, 5)
+      oprot.writeString(self.partition)
+      oprot.writeFieldEnd()
+    if self.tableObj is not None:
+      oprot.writeFieldBegin('tableObj', TType.STRING, 6)
+      oprot.writeString(self.tableObj)
+      oprot.writeFieldEnd()
+    if self.partitionObj is not None:
+      oprot.writeFieldBegin('partitionObj', TType.STRING, 7)
+      oprot.writeString(self.partitionObj)
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.writeId is None:
+      raise TProtocol.TProtocolException(message='Required field writeId is unset!')
+    if self.database is None:
+      raise TProtocol.TProtocolException(message='Required field database is unset!')
+    if self.table is None:
+      raise TProtocol.TProtocolException(message='Required field table is unset!')
+    if self.files is None:
+      raise TProtocol.TProtocolException(message='Required field files is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.writeId)
+    value = (value * 31) ^ hash(self.database)
+    value = (value * 31) ^ hash(self.table)
+    value = (value * 31) ^ hash(self.files)
+    value = (value * 31) ^ hash(self.partition)
+    value = (value * 31) ^ hash(self.tableObj)
+    value = (value * 31) ^ hash(self.partitionObj)
     return value
 
   def __repr__(self):
@@ -11703,10 +11876,10 @@ class ReplTblWriteIdStateRequest:
       elif fid == 6:
         if ftype == TType.LIST:
           self.partNames = []
-          (_etype526, _size523) = iprot.readListBegin()
-          for _i527 in xrange(_size523):
-            _elem528 = iprot.readString()
-            self.partNames.append(_elem528)
+          (_etype533, _size530) = iprot.readListBegin()
+          for _i534 in xrange(_size530):
+            _elem535 = iprot.readString()
+            self.partNames.append(_elem535)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11743,8 +11916,8 @@ class ReplTblWriteIdStateRequest:
     if self.partNames is not None:
       oprot.writeFieldBegin('partNames', TType.LIST, 6)
       oprot.writeListBegin(TType.STRING, len(self.partNames))
-      for iter529 in self.partNames:
-        oprot.writeString(iter529)
+      for iter536 in self.partNames:
+        oprot.writeString(iter536)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -11814,10 +11987,10 @@ class GetValidWriteIdsRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fullTableNames = []
-          (_etype533, _size530) = iprot.readListBegin()
-          for _i534 in xrange(_size530):
-            _elem535 = iprot.readString()
-            self.fullTableNames.append(_elem535)
+          (_etype540, _size537) = iprot.readListBegin()
+          for _i541 in xrange(_size537):
+            _elem542 = iprot.readString()
+            self.fullTableNames.append(_elem542)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11839,8 +12012,8 @@ class GetValidWriteIdsRequest:
     if self.fullTableNames is not None:
       oprot.writeFieldBegin('fullTableNames', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.fullTableNames))
-      for iter536 in self.fullTableNames:
-        oprot.writeString(iter536)
+      for iter543 in self.fullTableNames:
+        oprot.writeString(iter543)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -11923,10 +12096,10 @@ class TableValidWriteIds:
       elif fid == 3:
         if ftype == TType.LIST:
           self.invalidWriteIds = []
-          (_etype540, _size537) = iprot.readListBegin()
-          for _i541 in xrange(_size537):
-            _elem542 = iprot.readI64()
-            self.invalidWriteIds.append(_elem542)
+          (_etype547, _size544) = iprot.readListBegin()
+          for _i548 in xrange(_size544):
+            _elem549 = iprot.readI64()
+            self.invalidWriteIds.append(_elem549)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -11961,8 +12134,8 @@ class TableValidWriteIds:
     if self.invalidWriteIds is not None:
       oprot.writeFieldBegin('invalidWriteIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.invalidWriteIds))
-      for iter543 in self.invalidWriteIds:
-        oprot.writeI64(iter543)
+      for iter550 in self.invalidWriteIds:
+        oprot.writeI64(iter550)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.minOpenWriteId is not None:
@@ -12034,11 +12207,11 @@ class GetValidWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.tblValidWriteIds = []
-          (_etype547, _size544) = iprot.readListBegin()
-          for _i548 in xrange(_size544):
-            _elem549 = TableValidWriteIds()
-            _elem549.read(iprot)
-            self.tblValidWriteIds.append(_elem549)
+          (_etype554, _size551) = iprot.readListBegin()
+          for _i555 in xrange(_size551):
+            _elem556 = TableValidWriteIds()
+            _elem556.read(iprot)
+            self.tblValidWriteIds.append(_elem556)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12055,8 +12228,8 @@ class GetValidWriteIdsResponse:
     if self.tblValidWriteIds is not None:
       oprot.writeFieldBegin('tblValidWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tblValidWriteIds))
-      for iter550 in self.tblValidWriteIds:
-        iter550.write(oprot)
+      for iter557 in self.tblValidWriteIds:
+        iter557.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12132,10 +12305,10 @@ class AllocateTableWriteIdsRequest:
       elif fid == 3:
         if ftype == TType.LIST:
           self.txnIds = []
-          (_etype554, _size551) = iprot.readListBegin()
-          for _i555 in xrange(_size551):
-            _elem556 = iprot.readI64()
-            self.txnIds.append(_elem556)
+          (_etype561, _size558) = iprot.readListBegin()
+          for _i562 in xrange(_size558):
+            _elem563 = iprot.readI64()
+            self.txnIds.append(_elem563)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12147,11 +12320,11 @@ class AllocateTableWriteIdsRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.srcTxnToWriteIdList = []
-          (_etype560, _size557) = iprot.readListBegin()
-          for _i561 in xrange(_size557):
-            _elem562 = TxnToWriteId()
-            _elem562.read(iprot)
-            self.srcTxnToWriteIdList.append(_elem562)
+          (_etype567, _size564) = iprot.readListBegin()
+          for _i568 in xrange(_size564):
+            _elem569 = TxnToWriteId()
+            _elem569.read(iprot)
+            self.srcTxnToWriteIdList.append(_elem569)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12176,8 +12349,8 @@ class AllocateTableWriteIdsRequest:
     if self.txnIds is not None:
       oprot.writeFieldBegin('txnIds', TType.LIST, 3)
       oprot.writeListBegin(TType.I64, len(self.txnIds))
-      for iter563 in self.txnIds:
-        oprot.writeI64(iter563)
+      for iter570 in self.txnIds:
+        oprot.writeI64(iter570)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.replPolicy is not None:
@@ -12187,8 +12360,8 @@ class AllocateTableWriteIdsRequest:
     if self.srcTxnToWriteIdList is not None:
       oprot.writeFieldBegin('srcTxnToWriteIdList', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.srcTxnToWriteIdList))
-      for iter564 in self.srcTxnToWriteIdList:
-        iter564.write(oprot)
+      for iter571 in self.srcTxnToWriteIdList:
+        iter571.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12330,11 +12503,11 @@ class AllocateTableWriteIdsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.txnToWriteIds = []
-          (_etype568, _size565) = iprot.readListBegin()
-          for _i569 in xrange(_size565):
-            _elem570 = TxnToWriteId()
-            _elem570.read(iprot)
-            self.txnToWriteIds.append(_elem570)
+          (_etype575, _size572) = iprot.readListBegin()
+          for _i576 in xrange(_size572):
+            _elem577 = TxnToWriteId()
+            _elem577.read(iprot)
+            self.txnToWriteIds.append(_elem577)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12351,8 +12524,8 @@ class AllocateTableWriteIdsResponse:
     if self.txnToWriteIds is not None:
       oprot.writeFieldBegin('txnToWriteIds', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.txnToWriteIds))
-      for iter571 in self.txnToWriteIds:
-        iter571.write(oprot)
+      for iter578 in self.txnToWriteIds:
+        iter578.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -12580,11 +12753,11 @@ class LockRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.component = []
-          (_etype575, _size572) = iprot.readListBegin()
-          for _i576 in xrange(_size572):
-            _elem577 = LockComponent()
-            _elem577.read(iprot)
-            self.component.append(_elem577)
+          (_etype582, _size579) = iprot.readListBegin()
+          for _i583 in xrange(_size579):
+            _elem584 = LockComponent()
+            _elem584.read(iprot)
+            self.component.append(_elem584)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -12621,8 +12794,8 @@ class LockRequest:
     if self.component is not None:
       oprot.writeFieldBegin('component', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.component))
-      for iter578 in self.component:
-        iter578.write(oprot)
+      for iter585 in self.component:
+        iter585.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.txnid is not None:
@@ -13320,11 +13493,11 @@ class ShowLocksResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.locks = []
-          (_etype582, _size579) = iprot.readListBegin()
-          for _i583 in xrange(_size579):
-            _elem584 = ShowLocksResponseElement()
-            _elem584.read(iprot)
-            self.locks.append(_elem584)
+          (_etype589, _size586) = iprot.readListBegin()
+          for _i590 in xrange(_size586):
+            _elem591 = ShowLocksResponseElement()
+            _elem591.read(iprot)
+            self.locks.append(_elem591)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -13341,8 +13514,8 @@ class ShowLocksResponse:
     if self.locks is not None:
       oprot.writeFieldBegin('locks', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.locks))
-      for iter585 in self.locks:
-        iter585.write(oprot)
+      for iter592 in self.locks:
+        iter592.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13557,20 +13730,20 @@ class HeartbeatTxnRangeResponse:
       if fid == 1:
         if ftype == TType.SET:
           self.aborted = set()
-          (_etype589, _size586) = iprot.readSetBegin()
-          for _i590 in xrange(_size586):
-            _elem591 = iprot.readI64()
-            self.aborted.add(_elem591)
+          (_etype596, _size593) = iprot.readSetBegin()
+          for _i597 in xrange(_size593):
+            _elem598 = iprot.readI64()
+            self.aborted.add(_elem598)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.SET:
           self.nosuch = set()
-          (_etype595, _size592) = iprot.readSetBegin()
-          for _i596 in xrange(_size592):
-            _elem597 = iprot.readI64()
-            self.nosuch.add(_elem597)
+          (_etype602, _size599) = iprot.readSetBegin()
+          for _i603 in xrange(_size599):
+            _elem604 = iprot.readI64()
+            self.nosuch.add(_elem604)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -13587,15 +13760,15 @@ class HeartbeatTxnRangeResponse:
     if self.aborted is not None:
       oprot.writeFieldBegin('aborted', TType.SET, 1)
       oprot.writeSetBegin(TType.I64, len(self.aborted))
-      for iter598 in self.aborted:
-        oprot.writeI64(iter598)
+      for iter605 in self.aborted:
+        oprot.writeI64(iter605)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.nosuch is not None:
       oprot.writeFieldBegin('nosuch', TType.SET, 2)
       oprot.writeSetBegin(TType.I64, len(self.nosuch))
-      for iter599 in self.nosuch:
-        oprot.writeI64(iter599)
+      for iter606 in self.nosuch:
+        oprot.writeI64(iter606)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -13692,11 +13865,11 @@ class CompactionRequest:
       elif fid == 6:
         if ftype == TType.MAP:
           self.properties = {}
-          (_ktype601, _vtype602, _size600 ) = iprot.readMapBegin()
-          for _i604 in xrange(_size600):
-            _key605 = iprot.readString()
-            _val606 = iprot.readString()
-            self.properties[_key605] = _val606
+          (_ktype608, _vtype609, _size607 ) = iprot.readMapBegin()
+          for _i611 in xrange(_size607):
+            _key612 = iprot.readString()
+            _val613 = iprot.readString()
+            self.properties[_key612] = _val613
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -13733,9 +13906,9 @@ class CompactionRequest:
     if self.properties is not None:
       oprot.writeFieldBegin('properties', TType.MAP, 6)
       oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.properties))
-      for kiter607,viter608 in self.properties.items():
-        oprot.writeString(kiter607)
-        oprot.writeString(viter608)
+      for kiter614,viter615 in self.properties.items():
+        oprot.writeString(kiter614)
+        oprot.writeString(viter615)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14170,11 +14343,11 @@ class ShowCompactResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.compacts = []
-          (_etype612, _size609) = iprot.readListBegin()
-          for _i613 in xrange(_size609):
-            _elem614 = ShowCompactResponseElement()
-            _elem614.read(iprot)
-            self.compacts.append(_elem614)
+          (_etype619, _size616) = iprot.readListBegin()
+          for _i620 in xrange(_size616):
+            _elem621 = ShowCompactResponseElement()
+            _elem621.read(iprot)
+            self.compacts.append(_elem621)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14191,8 +14364,8 @@ class ShowCompactResponse:
     if self.compacts is not None:
       oprot.writeFieldBegin('compacts', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.compacts))
-      for iter615 in self.compacts:
-        iter615.write(oprot)
+      for iter622 in self.compacts:
+        iter622.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -14281,10 +14454,10 @@ class AddDynamicPartitions:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionnames = []
-          (_etype619, _size616) = iprot.readListBegin()
-          for _i620 in xrange(_size616):
-            _elem621 = iprot.readString()
-            self.partitionnames.append(_elem621)
+          (_etype626, _size623) = iprot.readListBegin()
+          for _i627 in xrange(_size623):
+            _elem628 = iprot.readString()
+            self.partitionnames.append(_elem628)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14322,8 +14495,8 @@ class AddDynamicPartitions:
     if self.partitionnames is not None:
       oprot.writeFieldBegin('partitionnames', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionnames))
-      for iter622 in self.partitionnames:
-        oprot.writeString(iter622)
+      for iter629 in self.partitionnames:
+        oprot.writeString(iter629)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.operationType is not None:
@@ -14553,10 +14726,10 @@ class CreationMetadata:
       elif fid == 4:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype626, _size623) = iprot.readSetBegin()
-          for _i627 in xrange(_size623):
-            _elem628 = iprot.readString()
-            self.tablesUsed.add(_elem628)
+          (_etype633, _size630) = iprot.readSetBegin()
+          for _i634 in xrange(_size630):
+            _elem635 = iprot.readString()
+            self.tablesUsed.add(_elem635)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -14590,8 +14763,8 @@ class CreationMetadata:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 4)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter629 in self.tablesUsed:
-        oprot.writeString(iter629)
+      for iter636 in self.tablesUsed:
+        oprot.writeString(iter636)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -14903,11 +15076,11 @@ class NotificationEventResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.events = []
-          (_etype633, _size630) = iprot.readListBegin()
-          for _i634 in xrange(_size630):
-            _elem635 = NotificationEvent()
-            _elem635.read(iprot)
-            self.events.append(_elem635)
+          (_etype640, _size637) = iprot.readListBegin()
+          for _i641 in xrange(_size637):
+            _elem642 = NotificationEvent()
+            _elem642.read(iprot)
+            self.events.append(_elem642)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -14924,8 +15097,8 @@ class NotificationEventResponse:
     if self.events is not None:
       oprot.writeFieldBegin('events', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.events))
-      for iter636 in self.events:
-        iter636.write(oprot)
+      for iter643 in self.events:
+        iter643.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15188,6 +15361,7 @@ class InsertEventRequestData:
    - replace
    - filesAdded
    - filesAddedChecksum
+   - subDirectoryList
   """
 
   thrift_spec = (
@@ -15195,12 +15369,14 @@ class InsertEventRequestData:
     (1, TType.BOOL, 'replace', None, None, ), # 1
     (2, TType.LIST, 'filesAdded', (TType.STRING,None), None, ), # 2
     (3, TType.LIST, 'filesAddedChecksum', (TType.STRING,None), None, ), # 3
+    (4, TType.LIST, 'subDirectoryList', (TType.STRING,None), None, ), # 4
   )
 
-  def __init__(self, replace=None, filesAdded=None, filesAddedChecksum=None,):
+  def __init__(self, replace=None, filesAdded=None, filesAddedChecksum=None, subDirectoryList=None,):
     self.replace = replace
     self.filesAdded = filesAdded
     self.filesAddedChecksum = filesAddedChecksum
+    self.subDirectoryList = subDirectoryList
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -15219,20 +15395,30 @@ class InsertEventRequestData:
       elif fid == 2:
         if ftype == TType.LIST:
           self.filesAdded = []
-          (_etype640, _size637) = iprot.readListBegin()
-          for _i641 in xrange(_size637):
-            _elem642 = iprot.readString()
-            self.filesAdded.append(_elem642)
+          (_etype647, _size644) = iprot.readListBegin()
+          for _i648 in xrange(_size644):
+            _elem649 = iprot.readString()
+            self.filesAdded.append(_elem649)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.filesAddedChecksum = []
-          (_etype646, _size643) = iprot.readListBegin()
-          for _i647 in xrange(_size643):
-            _elem648 = iprot.readString()
-            self.filesAddedChecksum.append(_elem648)
+          (_etype653, _size650) = iprot.readListBegin()
+          for _i654 in xrange(_size650):
+            _elem655 = iprot.readString()
+            self.filesAddedChecksum.append(_elem655)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.LIST:
+          self.subDirectoryList = []
+          (_etype659, _size656) = iprot.readListBegin()
+          for _i660 in xrange(_size656):
+            _elem661 = iprot.readString()
+            self.subDirectoryList.append(_elem661)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15253,15 +15439,22 @@ class InsertEventRequestData:
     if self.filesAdded is not None:
       oprot.writeFieldBegin('filesAdded', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.filesAdded))
-      for iter649 in self.filesAdded:
-        oprot.writeString(iter649)
+      for iter662 in self.filesAdded:
+        oprot.writeString(iter662)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.filesAddedChecksum is not None:
       oprot.writeFieldBegin('filesAddedChecksum', TType.LIST, 3)
       oprot.writeListBegin(TType.STRING, len(self.filesAddedChecksum))
-      for iter650 in self.filesAddedChecksum:
-        oprot.writeString(iter650)
+      for iter663 in self.filesAddedChecksum:
+        oprot.writeString(iter663)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    if self.subDirectoryList is not None:
+      oprot.writeFieldBegin('subDirectoryList', TType.LIST, 4)
+      oprot.writeListBegin(TType.STRING, len(self.subDirectoryList))
+      for iter664 in self.subDirectoryList:
+        oprot.writeString(iter664)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -15278,6 +15471,7 @@ class InsertEventRequestData:
     value = (value * 31) ^ hash(self.replace)
     value = (value * 31) ^ hash(self.filesAdded)
     value = (value * 31) ^ hash(self.filesAddedChecksum)
+    value = (value * 31) ^ hash(self.subDirectoryList)
     return value
 
   def __repr__(self):
@@ -15419,10 +15613,10 @@ class FireEventRequest:
       elif fid == 5:
         if ftype == TType.LIST:
           self.partitionVals = []
-          (_etype654, _size651) = iprot.readListBegin()
-          for _i655 in xrange(_size651):
-            _elem656 = iprot.readString()
-            self.partitionVals.append(_elem656)
+          (_etype668, _size665) = iprot.readListBegin()
+          for _i669 in xrange(_size665):
+            _elem670 = iprot.readString()
+            self.partitionVals.append(_elem670)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15460,8 +15654,8 @@ class FireEventRequest:
     if self.partitionVals is not None:
       oprot.writeFieldBegin('partitionVals', TType.LIST, 5)
       oprot.writeListBegin(TType.STRING, len(self.partitionVals))
-      for iter657 in self.partitionVals:
-        oprot.writeString(iter657)
+      for iter671 in self.partitionVals:
+        oprot.writeString(iter671)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.catName is not None:
@@ -15546,6 +15740,201 @@ class FireEventResponse:
   def __ne__(self, other):
     return not (self == other)
 
+class WriteNotificationLogRequest:
+  """
+  Attributes:
+   - txnId
+   - writeId
+   - db
+   - table
+   - fileInfo
+   - partitionVals
+  """
+
+  thrift_spec = (
+    None, # 0
+    (1, TType.I64, 'txnId', None, None, ), # 1
+    (2, TType.I64, 'writeId', None, None, ), # 2
+    (3, TType.STRING, 'db', None, None, ), # 3
+    (4, TType.STRING, 'table', None, None, ), # 4
+    (5, TType.STRUCT, 'fileInfo', (InsertEventRequestData, InsertEventRequestData.thrift_spec), None, ), # 5
+    (6, TType.LIST, 'partitionVals', (TType.STRING,None), None, ), # 6
+  )
+
+  def __init__(self, txnId=None, writeId=None, db=None, table=None, fileInfo=None, partitionVals=None,):
+    self.txnId = txnId
+    self.writeId = writeId
+    self.db = db
+    self.table = table
+    self.fileInfo = fileInfo
+    self.partitionVals = partitionVals
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      if fid == 1:
+        if ftype == TType.I64:
+          self.txnId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 2:
+        if ftype == TType.I64:
+          self.writeId = iprot.readI64()
+        else:
+          iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.STRING:
+          self.db = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 4:
+        if ftype == TType.STRING:
+          self.table = iprot.readString()
+        else:
+          iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.STRUCT:
+          self.fileInfo = InsertEventRequestData()
+          self.fileInfo.read(iprot)
+        else:
+          iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.LIST:
+          self.partitionVals = []
+          (_etype675, _size672) = iprot.readListBegin()
+          for _i676 in xrange(_size672):
+            _elem677 = iprot.readString()
+            self.partitionVals.append(_elem677)
+          iprot.readListEnd()
+        else:
+          iprot.skip(ftype)
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WriteNotificationLogRequest')
+    if self.txnId is not None:
+      oprot.writeFieldBegin('txnId', TType.I64, 1)
+      oprot.writeI64(self.txnId)
+      oprot.writeFieldEnd()
+    if self.writeId is not None:
+      oprot.writeFieldBegin('writeId', TType.I64, 2)
+      oprot.writeI64(self.writeId)
+      oprot.writeFieldEnd()
+    if self.db is not None:
+      oprot.writeFieldBegin('db', TType.STRING, 3)
+      oprot.writeString(self.db)
+      oprot.writeFieldEnd()
+    if self.table is not None:
+      oprot.writeFieldBegin('table', TType.STRING, 4)
+      oprot.writeString(self.table)
+      oprot.writeFieldEnd()
+    if self.fileInfo is not None:
+      oprot.writeFieldBegin('fileInfo', TType.STRUCT, 5)
+      self.fileInfo.write(oprot)
+      oprot.writeFieldEnd()
+    if self.partitionVals is not None:
+      oprot.writeFieldBegin('partitionVals', TType.LIST, 6)
+      oprot.writeListBegin(TType.STRING, len(self.partitionVals))
+      for iter678 in self.partitionVals:
+        oprot.writeString(iter678)
+      oprot.writeListEnd()
+      oprot.writeFieldEnd()
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    if self.txnId is None:
+      raise TProtocol.TProtocolException(message='Required field txnId is unset!')
+    if self.writeId is None:
+      raise TProtocol.TProtocolException(message='Required field writeId is unset!')
+    if self.db is None:
+      raise TProtocol.TProtocolException(message='Required field db is unset!')
+    if self.table is None:
+      raise TProtocol.TProtocolException(message='Required field table is unset!')
+    if self.fileInfo is None:
+      raise TProtocol.TProtocolException(message='Required field fileInfo is unset!')
+    return
+
+
+  def __hash__(self):
+    value = 17
+    value = (value * 31) ^ hash(self.txnId)
+    value = (value * 31) ^ hash(self.writeId)
+    value = (value * 31) ^ hash(self.db)
+    value = (value * 31) ^ hash(self.table)
+    value = (value * 31) ^ hash(self.fileInfo)
+    value = (value * 31) ^ hash(self.partitionVals)
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
+class WriteNotificationLogResponse:
+
+  thrift_spec = (
+  )
+
+  def read(self, iprot):
+    if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
+      fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec))
+      return
+    iprot.readStructBegin()
+    while True:
+      (fname, ftype, fid) = iprot.readFieldBegin()
+      if ftype == TType.STOP:
+        break
+      else:
+        iprot.skip(ftype)
+      iprot.readFieldEnd()
+    iprot.readStructEnd()
+
+  def write(self, oprot):
+    if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None:
+      oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec)))
+      return
+    oprot.writeStructBegin('WriteNotificationLogResponse')
+    oprot.writeFieldStop()
+    oprot.writeStructEnd()
+
+  def validate(self):
+    return
+
+
+  def __hash__(self):
+    value = 17
+    return value
+
+  def __repr__(self):
+    L = ['%s=%r' % (key, value)
+      for key, value in self.__dict__.iteritems()]
+    return '%s(%s)' % (self.__class__.__name__, ', '.join(L))
+
+  def __eq__(self, other):
+    return isinstance(other, self.__class__) and self.__dict__ == other.__dict__
+
+  def __ne__(self, other):
+    return not (self == other)
+
 class MetadataPpdResult:
   """
   Attributes:
@@ -15653,12 +16042,12 @@ class GetFileMetadataByExprResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype659, _vtype660, _size658 ) = iprot.readMapBegin()
-          for _i662 in xrange(_size658):
-            _key663 = iprot.readI64()
-            _val664 = MetadataPpdResult()
-            _val664.read(iprot)
-            self.metadata[_key663] = _val664
+          (_ktype680, _vtype681, _size679 ) = iprot.readMapBegin()
+          for _i683 in xrange(_size679):
+            _key684 = iprot.readI64()
+            _val685 = MetadataPpdResult()
+            _val685.read(iprot)
+            self.metadata[_key684] = _val685
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15680,9 +16069,9 @@ class GetFileMetadataByExprResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRUCT, len(self.metadata))
-      for kiter665,viter666 in self.metadata.items():
-        oprot.writeI64(kiter665)
-        viter666.write(oprot)
+      for kiter686,viter687 in self.metadata.items():
+        oprot.writeI64(kiter686)
+        viter687.write(oprot)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -15752,10 +16141,10 @@ class GetFileMetadataByExprRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype670, _size667) = iprot.readListBegin()
-          for _i671 in xrange(_size667):
-            _elem672 = iprot.readI64()
-            self.fileIds.append(_elem672)
+          (_etype691, _size688) = iprot.readListBegin()
+          for _i692 in xrange(_size688):
+            _elem693 = iprot.readI64()
+            self.fileIds.append(_elem693)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15787,8 +16176,8 @@ class GetFileMetadataByExprRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter673 in self.fileIds:
-        oprot.writeI64(iter673)
+      for iter694 in self.fileIds:
+        oprot.writeI64(iter694)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.expr is not None:
@@ -15862,11 +16251,11 @@ class GetFileMetadataResult:
       if fid == 1:
         if ftype == TType.MAP:
           self.metadata = {}
-          (_ktype675, _vtype676, _size674 ) = iprot.readMapBegin()
-          for _i678 in xrange(_size674):
-            _key679 = iprot.readI64()
-            _val680 = iprot.readString()
-            self.metadata[_key679] = _val680
+          (_ktype696, _vtype697, _size695 ) = iprot.readMapBegin()
+          for _i699 in xrange(_size695):
+            _key700 = iprot.readI64()
+            _val701 = iprot.readString()
+            self.metadata[_key700] = _val701
           iprot.readMapEnd()
         else:
           iprot.skip(ftype)
@@ -15888,9 +16277,9 @@ class GetFileMetadataResult:
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.MAP, 1)
       oprot.writeMapBegin(TType.I64, TType.STRING, len(self.metadata))
-      for kiter681,viter682 in self.metadata.items():
-        oprot.writeI64(kiter681)
-        oprot.writeString(viter682)
+      for kiter702,viter703 in self.metadata.items():
+        oprot.writeI64(kiter702)
+        oprot.writeString(viter703)
       oprot.writeMapEnd()
       oprot.writeFieldEnd()
     if self.isSupported is not None:
@@ -15951,10 +16340,10 @@ class GetFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype686, _size683) = iprot.readListBegin()
-          for _i687 in xrange(_size683):
-            _elem688 = iprot.readI64()
-            self.fileIds.append(_elem688)
+          (_etype707, _size704) = iprot.readListBegin()
+          for _i708 in xrange(_size704):
+            _elem709 = iprot.readI64()
+            self.fileIds.append(_elem709)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -15971,8 +16360,8 @@ class GetFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter689 in self.fileIds:
-        oprot.writeI64(iter689)
+      for iter710 in self.fileIds:
+        oprot.writeI64(iter710)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16078,20 +16467,20 @@ class PutFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype693, _size690) = iprot.readListBegin()
-          for _i694 in xrange(_size690):
-            _elem695 = iprot.readI64()
-            self.fileIds.append(_elem695)
+          (_etype714, _size711) = iprot.readListBegin()
+          for _i715 in xrange(_size711):
+            _elem716 = iprot.readI64()
+            self.fileIds.append(_elem716)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.metadata = []
-          (_etype699, _size696) = iprot.readListBegin()
-          for _i700 in xrange(_size696):
-            _elem701 = iprot.readString()
-            self.metadata.append(_elem701)
+          (_etype720, _size717) = iprot.readListBegin()
+          for _i721 in xrange(_size717):
+            _elem722 = iprot.readString()
+            self.metadata.append(_elem722)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16113,15 +16502,15 @@ class PutFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter702 in self.fileIds:
-        oprot.writeI64(iter702)
+      for iter723 in self.fileIds:
+        oprot.writeI64(iter723)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.metadata is not None:
       oprot.writeFieldBegin('metadata', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.metadata))
-      for iter703 in self.metadata:
-        oprot.writeString(iter703)
+      for iter724 in self.metadata:
+        oprot.writeString(iter724)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.type is not None:
@@ -16229,10 +16618,10 @@ class ClearFileMetadataRequest:
       if fid == 1:
         if ftype == TType.LIST:
           self.fileIds = []
-          (_etype707, _size704) = iprot.readListBegin()
-          for _i708 in xrange(_size704):
-            _elem709 = iprot.readI64()
-            self.fileIds.append(_elem709)
+          (_etype728, _size725) = iprot.readListBegin()
+          for _i729 in xrange(_size725):
+            _elem730 = iprot.readI64()
+            self.fileIds.append(_elem730)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16249,8 +16638,8 @@ class ClearFileMetadataRequest:
     if self.fileIds is not None:
       oprot.writeFieldBegin('fileIds', TType.LIST, 1)
       oprot.writeListBegin(TType.I64, len(self.fileIds))
-      for iter710 in self.fileIds:
-        oprot.writeI64(iter710)
+      for iter731 in self.fileIds:
+        oprot.writeI64(iter731)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16479,11 +16868,11 @@ class GetAllFunctionsResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.functions = []
-          (_etype714, _size711) = iprot.readListBegin()
-          for _i715 in xrange(_size711):
-            _elem716 = Function()
-            _elem716.read(iprot)
-            self.functions.append(_elem716)
+          (_etype735, _size732) = iprot.readListBegin()
+          for _i736 in xrange(_size732):
+            _elem737 = Function()
+            _elem737.read(iprot)
+            self.functions.append(_elem737)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16500,8 +16889,8 @@ class GetAllFunctionsResponse:
     if self.functions is not None:
       oprot.writeFieldBegin('functions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.functions))
-      for iter717 in self.functions:
-        iter717.write(oprot)
+      for iter738 in self.functions:
+        iter738.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16553,10 +16942,10 @@ class ClientCapabilities:
       if fid == 1:
         if ftype == TType.LIST:
           self.values = []
-          (_etype721, _size718) = iprot.readListBegin()
-          for _i722 in xrange(_size718):
-            _elem723 = iprot.readI32()
-            self.values.append(_elem723)
+          (_etype742, _size739) = iprot.readListBegin()
+          for _i743 in xrange(_size739):
+            _elem744 = iprot.readI32()
+            self.values.append(_elem744)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16573,8 +16962,8 @@ class ClientCapabilities:
     if self.values is not None:
       oprot.writeFieldBegin('values', TType.LIST, 1)
       oprot.writeListBegin(TType.I32, len(self.values))
-      for iter724 in self.values:
-        oprot.writeI32(iter724)
+      for iter745 in self.values:
+        oprot.writeI32(iter745)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -16819,10 +17208,10 @@ class GetTablesRequest:
       elif fid == 2:
         if ftype == TType.LIST:
           self.tblNames = []
-          (_etype728, _size725) = iprot.readListBegin()
-          for _i729 in xrange(_size725):
-            _elem730 = iprot.readString()
-            self.tblNames.append(_elem730)
+          (_etype749, _size746) = iprot.readListBegin()
+          for _i750 in xrange(_size746):
+            _elem751 = iprot.readString()
+            self.tblNames.append(_elem751)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16854,8 +17243,8 @@ class GetTablesRequest:
     if self.tblNames is not None:
       oprot.writeFieldBegin('tblNames', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.tblNames))
-      for iter731 in self.tblNames:
-        oprot.writeString(iter731)
+      for iter752 in self.tblNames:
+        oprot.writeString(iter752)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.capabilities is not None:
@@ -16920,11 +17309,11 @@ class GetTablesResult:
       if fid == 1:
         if ftype == TType.LIST:
           self.tables = []
-          (_etype735, _size732) = iprot.readListBegin()
-          for _i736 in xrange(_size732):
-            _elem737 = Table()
-            _elem737.read(iprot)
-            self.tables.append(_elem737)
+          (_etype756, _size753) = iprot.readListBegin()
+          for _i757 in xrange(_size753):
+            _elem758 = Table()
+            _elem758.read(iprot)
+            self.tables.append(_elem758)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -16941,8 +17330,8 @@ class GetTablesResult:
     if self.tables is not None:
       oprot.writeFieldBegin('tables', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.tables))
-      for iter738 in self.tables:
-        iter738.write(oprot)
+      for iter759 in self.tables:
+        iter759.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -17256,10 +17645,10 @@ class Materialization:
       if fid == 1:
         if ftype == TType.SET:
           self.tablesUsed = set()
-          (_etype742, _size739) = iprot.readSetBegin()
-          for _i743 in xrange(_size739):
-            _elem744 = iprot.readString()
-            self.tablesUsed.add(_elem744)
+          (_etype763, _size760) = iprot.readSetBegin()
+          for _i764 in xrange(_size760):
+            _elem765 = iprot.readString()
+            self.tablesUsed.add(_elem765)
           iprot.readSetEnd()
         else:
           iprot.skip(ftype)
@@ -17291,8 +17680,8 @@ class Materialization:
     if self.tablesUsed is not None:
       oprot.writeFieldBegin('tablesUsed', TType.SET, 1)
       oprot.writeSetBegin(TType.STRING, len(self.tablesUsed))
-      for iter745 in self.tablesUsed:
-        oprot.writeString(iter745)
+      for iter766 in self.tablesUsed:
+        oprot.writeString(iter766)
       oprot.writeSetEnd()
       oprot.writeFieldEnd()
     if self.validTxnList is not None:
@@ -18197,44 +18586,44 @@ class WMFullResourcePlan:
       elif fid == 2:
         if ftype == TType.LIST:
           self.pools = []
-          (_etype749, _size746) = iprot.readListBegin()
-          for _i750 in xrange(_size746):
-            _elem751 = WMPool()
-            _elem751.read(iprot)
-            self.pools.append(_elem751)
+          (_etype770, _size767) = iprot.readListBegin()
+          for _i771 in xrange(_size767):
+            _elem772 = WMPool()
+            _elem772.read(iprot)
+            self.pools.append(_elem772)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 3:
         if ftype == TType.LIST:
           self.mappings = []
-          (_etype755, _size752) = iprot.readListBegin()
-          for _i756 in xrange(_size752):
-            _elem757 = WMMapping()
-            _elem757.read(iprot)
-            self.mappings.append(_elem757)
+          (_etype776, _size773) = iprot.readListBegin()
+          for _i777 in xrange(_size773):
+            _elem778 = WMMapping()
+            _elem778.read(iprot)
+            self.mappings.append(_elem778)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 4:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype761, _size758) = iprot.readListBegin()
-          for _i762 in xrange(_size758):
-            _elem763 = WMTrigger()
-            _elem763.read(iprot)
-            self.triggers.append(_elem763)
+          (_etype782, _size779) = iprot.readListBegin()
+          for _i783 in xrange(_size779):
+            _elem784 = WMTrigger()
+            _elem784.read(iprot)
+            self.triggers.append(_elem784)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 5:
         if ftype == TType.LIST:
           self.poolTriggers = []
-          (_etype767, _size764) = iprot.readListBegin()
-          for _i768 in xrange(_size764):
-            _elem769 = WMPoolTrigger()
-            _elem769.read(iprot)
-            self.poolTriggers.append(_elem769)
+          (_etype788, _size785) = iprot.readListBegin()
+          for _i789 in xrange(_size785):
+            _elem790 = WMPoolTrigger()
+            _elem790.read(iprot)
+            self.poolTriggers.append(_elem790)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18255,29 +18644,29 @@ class WMFullResourcePlan:
     if self.pools is not None:
       oprot.writeFieldBegin('pools', TType.LIST, 2)
       oprot.writeListBegin(TType.STRUCT, len(self.pools))
-      for iter770 in self.pools:
-        iter770.write(oprot)
+      for iter791 in self.pools:
+        iter791.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.mappings is not None:
       oprot.writeFieldBegin('mappings', TType.LIST, 3)
       oprot.writeListBegin(TType.STRUCT, len(self.mappings))
-      for iter771 in self.mappings:
-        iter771.write(oprot)
+      for iter792 in self.mappings:
+        iter792.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter772 in self.triggers:
-        iter772.write(oprot)
+      for iter793 in self.triggers:
+        iter793.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.poolTriggers is not None:
       oprot.writeFieldBegin('poolTriggers', TType.LIST, 5)
       oprot.writeListBegin(TType.STRUCT, len(self.poolTriggers))
-      for iter773 in self.poolTriggers:
-        iter773.write(oprot)
+      for iter794 in self.poolTriggers:
+        iter794.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -18751,11 +19140,11 @@ class WMGetAllResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.resourcePlans = []
-          (_etype777, _size774) = iprot.readListBegin()
-          for _i778 in xrange(_size774):
-            _elem779 = WMResourcePlan()
-            _elem779.read(iprot)
-            self.resourcePlans.append(_elem779)
+          (_etype798, _size795) = iprot.readListBegin()
+          for _i799 in xrange(_size795):
+            _elem800 = WMResourcePlan()
+            _elem800.read(iprot)
+            self.resourcePlans.append(_elem800)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -18772,8 +19161,8 @@ class WMGetAllResourcePlanResponse:
     if self.resourcePlans is not None:
       oprot.writeFieldBegin('resourcePlans', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.resourcePlans))
-      for iter780 in self.resourcePlans:
-        iter780.write(oprot)
+      for iter801 in self.resourcePlans:
+        iter801.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19077,20 +19466,20 @@ class WMValidateResourcePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.errors = []
-          (_etype784, _size781) = iprot.readListBegin()
-          for _i785 in xrange(_size781):
-            _elem786 = iprot.readString()
-            self.errors.append(_elem786)
+          (_etype805, _size802) = iprot.readListBegin()
+          for _i806 in xrange(_size802):
+            _elem807 = iprot.readString()
+            self.errors.append(_elem807)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
       elif fid == 2:
         if ftype == TType.LIST:
           self.warnings = []
-          (_etype790, _size787) = iprot.readListBegin()
-          for _i791 in xrange(_size787):
-            _elem792 = iprot.readString()
-            self.warnings.append(_elem792)
+          (_etype811, _size808) = iprot.readListBegin()
+          for _i812 in xrange(_size808):
+            _elem813 = iprot.readString()
+            self.warnings.append(_elem813)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19107,15 +19496,15 @@ class WMValidateResourcePlanResponse:
     if self.errors is not None:
       oprot.writeFieldBegin('errors', TType.LIST, 1)
       oprot.writeListBegin(TType.STRING, len(self.errors))
-      for iter793 in self.errors:
-        oprot.writeString(iter793)
+      for iter814 in self.errors:
+        oprot.writeString(iter814)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.warnings is not None:
       oprot.writeFieldBegin('warnings', TType.LIST, 2)
       oprot.writeListBegin(TType.STRING, len(self.warnings))
-      for iter794 in self.warnings:
-        oprot.writeString(iter794)
+      for iter815 in self.warnings:
+        oprot.writeString(iter815)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -19692,11 +20081,11 @@ class WMGetTriggersForResourePlanResponse:
       if fid == 1:
         if ftype == TType.LIST:
           self.triggers = []
-          (_etype798, _size795) = iprot.readListBegin()
-          for _i799 in xrange(_size795):
-            _elem800 = WMTrigger()
-            _elem800.read(iprot)
-            self.triggers.append(_elem800)
+          (_etype819, _size816) = iprot.readListBegin()
+          for _i820 in xrange(_size816):
+            _elem821 = WMTrigger()
+            _elem821.read(iprot)
+            self.triggers.append(_elem821)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -19713,8 +20102,8 @@ class WMGetTriggersForResourePlanResponse:
     if self.triggers is not None:
       oprot.writeFieldBegin('triggers', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.triggers))
-      for iter801 in self.triggers:
-        iter801.write(oprot)
+      for iter822 in self.triggers:
+        iter822.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()
@@ -20898,11 +21287,11 @@ class SchemaVersion:
       elif fid == 4:
         if ftype == TType.LIST:
           self.cols = []
-          (_etype805, _size802) = iprot.readListBegin()
-          for _i806 in xrange(_size802):
-            _elem807 = FieldSchema()
-            _elem807.read(iprot)
-            self.cols.append(_elem807)
+          (_etype826, _size823) = iprot.readListBegin()
+          for _i827 in xrange(_size823):
+            _elem828 = FieldSchema()
+            _elem828.read(iprot)
+            self.cols.append(_elem828)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -20962,8 +21351,8 @@ class SchemaVersion:
     if self.cols is not None:
       oprot.writeFieldBegin('cols', TType.LIST, 4)
       oprot.writeListBegin(TType.STRUCT, len(self.cols))
-      for iter808 in self.cols:
-        iter808.write(oprot)
+      for iter829 in self.cols:
+        iter829.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     if self.state is not None:
@@ -21218,11 +21607,11 @@ class FindSchemasByColsResp:
       if fid == 1:
         if ftype == TType.LIST:
           self.schemaVersions = []
-          (_etype812, _size809) = iprot.readListBegin()
-          for _i813 in xrange(_size809):
-            _elem814 = SchemaVersionDescriptor()
-            _elem814.read(iprot)
-            self.schemaVersions.append(_elem814)
+          (_etype833, _size830) = iprot.readListBegin()
+          for _i834 in xrange(_size830):
+            _elem835 = SchemaVersionDescriptor()
+            _elem835.read(iprot)
+            self.schemaVersions.append(_elem835)
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
@@ -21239,8 +21628,8 @@ class FindSchemasByColsResp:
     if self.schemaVersions is not None:
       oprot.writeFieldBegin('schemaVersions', TType.LIST, 1)
       oprot.writeListBegin(TType.STRUCT, len(self.schemaVersions))
-      for iter815 in self.schemaVersions:
-        iter815.write(oprot)
+      for iter836 in self.schemaVersions:
+        iter836.write(oprot)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
     oprot.writeFieldStop()

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index fc640d0..0348ff2 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2582,10 +2582,12 @@ class CommitTxnRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   TXNID = 1
   REPLPOLICY = 2
+  WRITEEVENTINFOS = 3
 
   FIELDS = {
     TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid'},
-    REPLPOLICY => {:type => ::Thrift::Types::STRING, :name => 'replPolicy', :optional => true}
+    REPLPOLICY => {:type => ::Thrift::Types::STRING, :name => 'replPolicy', :optional => true},
+    WRITEEVENTINFOS => {:type => ::Thrift::Types::LIST, :name => 'writeEventInfos', :element => {:type => ::Thrift::Types::STRUCT, :class => ::WriteEventInfo}, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -2597,6 +2599,38 @@ class CommitTxnRequest
   ::Thrift::Struct.generate_accessors self
 end
 
+class WriteEventInfo
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  WRITEID = 1
+  DATABASE = 2
+  TABLE = 3
+  FILES = 4
+  PARTITION = 5
+  TABLEOBJ = 6
+  PARTITIONOBJ = 7
+
+  FIELDS = {
+    WRITEID => {:type => ::Thrift::Types::I64, :name => 'writeId'},
+    DATABASE => {:type => ::Thrift::Types::STRING, :name => 'database'},
+    TABLE => {:type => ::Thrift::Types::STRING, :name => 'table'},
+    FILES => {:type => ::Thrift::Types::STRING, :name => 'files'},
+    PARTITION => {:type => ::Thrift::Types::STRING, :name => 'partition', :optional => true},
+    TABLEOBJ => {:type => ::Thrift::Types::STRING, :name => 'tableObj', :optional => true},
+    PARTITIONOBJ => {:type => ::Thrift::Types::STRING, :name => 'partitionObj', :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field writeId is unset!') unless @writeId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field database is unset!') unless @database
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field table is unset!') unless @table
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field files is unset!') unless @files
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class ReplTblWriteIdStateRequest
   include ::Thrift::Struct, ::Thrift::Struct_Union
   VALIDWRITEIDLIST = 1
@@ -3395,11 +3429,13 @@ class InsertEventRequestData
   REPLACE = 1
   FILESADDED = 2
   FILESADDEDCHECKSUM = 3
+  SUBDIRECTORYLIST = 4
 
   FIELDS = {
     REPLACE => {:type => ::Thrift::Types::BOOL, :name => 'replace', :optional => true},
     FILESADDED => {:type => ::Thrift::Types::LIST, :name => 'filesAdded', :element => {:type => ::Thrift::Types::STRING}},
-    FILESADDEDCHECKSUM => {:type => ::Thrift::Types::LIST, :name => 'filesAddedChecksum', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+    FILESADDEDCHECKSUM => {:type => ::Thrift::Types::LIST, :name => 'filesAddedChecksum', :element => {:type => ::Thrift::Types::STRING}, :optional => true},
+    SUBDIRECTORYLIST => {:type => ::Thrift::Types::LIST, :name => 'subDirectoryList', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -3477,6 +3513,52 @@ class FireEventResponse
   ::Thrift::Struct.generate_accessors self
 end
 
+class WriteNotificationLogRequest
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+  TXNID = 1
+  WRITEID = 2
+  DB = 3
+  TABLE = 4
+  FILEINFO = 5
+  PARTITIONVALS = 6
+
+  FIELDS = {
+    TXNID => {:type => ::Thrift::Types::I64, :name => 'txnId'},
+    WRITEID => {:type => ::Thrift::Types::I64, :name => 'writeId'},
+    DB => {:type => ::Thrift::Types::STRING, :name => 'db'},
+    TABLE => {:type => ::Thrift::Types::STRING, :name => 'table'},
+    FILEINFO => {:type => ::Thrift::Types::STRUCT, :name => 'fileInfo', :class => ::InsertEventRequestData},
+    PARTITIONVALS => {:type => ::Thrift::Types::LIST, :name => 'partitionVals', :element => {:type => ::Thrift::Types::STRING}, :optional => true}
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field txnId is unset!') unless @txnId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field writeId is unset!') unless @writeId
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field db is unset!') unless @db
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field table is unset!') unless @table
+    raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field fileInfo is unset!') unless @fileInfo
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
+class WriteNotificationLogResponse
+  include ::Thrift::Struct, ::Thrift::Struct_Union
+
+  FIELDS = {
+
+  }
+
+  def struct_fields; FIELDS; end
+
+  def validate
+  end
+
+  ::Thrift::Struct.generate_accessors self
+end
+
 class MetadataPpdResult
   include ::Thrift::Struct, ::Thrift::Struct_Union
   METADATA = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
index bbf3f12..2bd958e 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/thrift_hive_metastore.rb
@@ -2751,6 +2751,21 @@ module ThriftHiveMetastore
       return
     end
 
+    def add_write_notification_log(rqst)
+      send_add_write_notification_log(rqst)
+      return recv_add_write_notification_log()
+    end
+
+    def send_add_write_notification_log(rqst)
+      send_message('add_write_notification_log', Add_write_notification_log_args, :rqst => rqst)
+    end
+
+    def recv_add_write_notification_log()
+      result = receive_message(Add_write_notification_log_result)
+      return result.success unless result.success.nil?
+      raise ::Thrift::ApplicationException.new(::Thrift::ApplicationException::MISSING_RESULT, 'add_write_notification_log failed: unknown result')
+    end
+
     def cm_recycle(request)
       send_cm_recycle(request)
       return recv_cm_recycle()
@@ -5520,6 +5535,13 @@ module ThriftHiveMetastore
       write_result(result, oprot, 'flushCache', seqid)
     end
 
+    def process_add_write_notification_log(seqid, iprot, oprot)
+      args = read_args(iprot, Add_write_notification_log_args)
+      result = Add_write_notification_log_result.new()
+      result.success = @handler.add_write_notification_log(args.rqst)
+      write_result(result, oprot, 'add_write_notification_log', seqid)
+    end
+
     def process_cm_recycle(seqid, iprot, oprot)
       args = read_args(iprot, Cm_recycle_args)
       result = Cm_recycle_result.new()
@@ -12220,6 +12242,38 @@ module ThriftHiveMetastore
     ::Thrift::Struct.generate_accessors self
   end
 
+  class Add_write_notification_log_args
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    RQST = -1
+
+    FIELDS = {
+      RQST => {:type => ::Thrift::Types::STRUCT, :name => 'rqst', :class => ::WriteNotificationLogRequest}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
+  class Add_write_notification_log_result
+    include ::Thrift::Struct, ::Thrift::Struct_Union
+    SUCCESS = 0
+
+    FIELDS = {
+      SUCCESS => {:type => ::Thrift::Types::STRUCT, :name => 'success', :class => ::WriteNotificationLogResponse}
+    }
+
+    def struct_fields; FIELDS; end
+
+    def validate
+    end
+
+    ::Thrift::Struct.generate_accessors self
+  end
+
   class Cm_recycle_args
     include ::Thrift::Struct, ::Thrift::Struct_Union
     REQUEST = 1

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
index 1327fa2..c6c04b7 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
@@ -81,6 +81,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hadoop.hive.metastore.events.AddForeignKeyEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.StatsUpdateMode;
@@ -7169,6 +7170,55 @@ public class HiveMetaStore extends ThriftHiveMetastore {
 
     @Override
     public void commit_txn(CommitTxnRequest rqst) throws TException {
+      // in replication flow, the write notification log table will be updated here.
+      if (rqst.isSetWriteEventInfos()) {
+        long targetTxnId = getTxnHandler().getTargetTxnId(rqst.getReplPolicy(), rqst.getTxnid());
+        if (targetTxnId < 0) {
+          //looks like a retry
+          return;
+        }
+        for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) {
+          String[] filesAdded = ReplChangeManager.getListFromSeparatedString(writeEventInfo.getFiles());
+          List<String> partitionValue = null;
+          Partition ptnObj = null;
+          String root;
+          Table tbl = getTblObject(writeEventInfo.getDatabase(), writeEventInfo.getTable());
+
+          if (writeEventInfo.getPartition() != null && !writeEventInfo.getPartition().isEmpty()) {
+            partitionValue = Warehouse.getPartValuesFromPartName(writeEventInfo.getPartition());
+            ptnObj = getPartitionObj(writeEventInfo.getDatabase(), writeEventInfo.getTable(), partitionValue, tbl);
+            root = ptnObj.getSd().getLocation();
+          } else {
+            root = tbl.getSd().getLocation();
+          }
+
+          InsertEventRequestData insertData = new InsertEventRequestData();
+          insertData.setReplace(true);
+
+          // The files in the commit txn message during load will have files with path corresponding to source
+          // warehouse. Need to transform them to target warehouse using table or partition object location.
+          for (String file : filesAdded) {
+            String[] decodedPath = ReplChangeManager.decodeFileUri(file);
+            String name = (new Path(decodedPath[0])).getName();
+            Path newPath = FileUtils.getTransformedPath(name, decodedPath[3], root);
+            insertData.addToFilesAdded(newPath.toUri().toString());
+            insertData.addToSubDirectoryList(decodedPath[3]);
+            try {
+              insertData.addToFilesAddedChecksum(ReplChangeManager.checksumFor(newPath, newPath.getFileSystem(conf)));
+            } catch (IOException e) {
+              LOG.error("failed to get checksum for the file " + newPath + " with error: " + e.getMessage());
+              throw new TException(e.getMessage());
+            }
+          }
+
+          WriteNotificationLogRequest wnRqst = new WriteNotificationLogRequest(targetTxnId,
+                  writeEventInfo.getWriteId(), writeEventInfo.getDatabase(), writeEventInfo.getTable(), insertData);
+          if (partitionValue != null) {
+            wnRqst.setPartitionVals(partitionValue);
+          }
+          addTxnWriteNotificationLog(tbl, ptnObj, wnRqst);
+        }
+      }
       getTxnHandler().commitTxn(rqst);
       if (listeners != null && !listeners.isEmpty()) {
         MetaStoreListenerNotifier.notifyEvent(listeners, EventType.COMMIT_TXN,
@@ -7198,6 +7248,42 @@ public class HiveMetaStore extends ThriftHiveMetastore {
       return response;
     }
 
+    private void addTxnWriteNotificationLog(Table tableObj, Partition ptnObj, WriteNotificationLogRequest rqst)
+            throws MetaException {
+      String partition = ""; //Empty string is an invalid partition name. Can be used for non partitioned table.
+      if (ptnObj != null) {
+        partition = Warehouse.makePartName(tableObj.getPartitionKeys(), rqst.getPartitionVals());
+      }
+      AcidWriteEvent event = new AcidWriteEvent(partition, tableObj, ptnObj, rqst);
+      getTxnHandler().addWriteNotificationLog(event);
+      if (listeners != null && !listeners.isEmpty()) {
+        MetaStoreListenerNotifier.notifyEvent(listeners, EventType.ACID_WRITE, event);
+      }
+    }
+
+    private Table getTblObject(String db, String table) throws MetaException, NoSuchObjectException {
+      GetTableRequest req = new GetTableRequest(db, table);
+      req.setCapabilities(new ClientCapabilities(Lists.newArrayList(ClientCapability.TEST_CAPABILITY)));
+      return get_table_req(req).getTable();
+    }
+
+    private Partition getPartitionObj(String db, String table, List<String> partitionVals, Table tableObj)
+            throws MetaException, NoSuchObjectException {
+      if (tableObj.isSetPartitionKeys() && !tableObj.getPartitionKeys().isEmpty()) {
+        return get_partition(db, table, partitionVals);
+      }
+      return null;
+    }
+
+    @Override
+    public WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)
+            throws MetaException, NoSuchObjectException {
+      Table tableObj = getTblObject(rqst.getDb(), rqst.getTable());
+      Partition ptnObj = getPartitionObj(rqst.getDb(), rqst.getTable(), rqst.getPartitionVals(), tableObj);
+      addTxnWriteNotificationLog(tableObj, ptnObj, rqst);
+      return new WriteNotificationLogResponse();
+    }
+
     @Override
     public LockResponse lock(LockRequest rqst) throws TException {
       return getTxnHandler().lock(rqst);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index da41e6e..bfd7141 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -2506,10 +2506,8 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
   }
 
   @Override
-  public void replCommitTxn(long srcTxnId, String replPolicy)
+  public void replCommitTxn(CommitTxnRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, TException {
-    CommitTxnRequest rqst = new CommitTxnRequest(srcTxnId);
-    rqst.setReplPolicy(replPolicy);
     client.commit_txn(rqst);
   }
 
@@ -2756,6 +2754,12 @@ public class HiveMetaStoreClient implements IMetaStoreClient, AutoCloseable {
     return client.fire_listener_event(rqst);
   }
 
+  @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
+  @Override
+  public void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException {
+    client.add_write_notification_log(rqst);
+  }
+
   /**
    * Creates a synchronized wrapper for any {@link IMetaStoreClient}.
    * This may be used by multi-threaded applications until we have

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index bc09076..b5d147b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
 import org.apache.hadoop.hive.metastore.api.CmRecycleResponse;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionResponse;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
@@ -125,6 +126,7 @@ import org.apache.hadoop.hive.metastore.api.WMPool;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.ObjectPair;
 import org.apache.thrift.TException;
@@ -2871,8 +2873,8 @@ public interface IMetaStoreClient {
   /**
    * Commit a transaction.  This will also unlock any locks associated with
    * this transaction.
-   * @param srcTxnid id of transaction at source which is committed and to be replicated.
-   * @param replPolicy the replication policy to identify the source cluster
+   * @param rqst Information containing the txn info and write event information
+   * of transaction at source which is committed and to be replicated
    * @throws NoSuchTxnException if the requested transaction does not exist.
    * This can result fro the transaction having timed out and been deleted by
    * the compactor.
@@ -2880,7 +2882,7 @@ public interface IMetaStoreClient {
    * aborted.  This can result from the transaction timing out.
    * @throws TException
    */
-  void replCommitTxn(long srcTxnid, String replPolicy)
+  void replCommitTxn(CommitTxnRequest rqst)
           throws NoSuchTxnException, TxnAbortedException, TException;
 
   /**
@@ -3193,6 +3195,14 @@ public interface IMetaStoreClient {
   @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
   FireEventResponse fireListenerEvent(FireEventRequest request) throws TException;
 
+  /**
+   * Add a event related to write operations in an ACID table.
+   * @param rqst message containing information for acid write operation.
+   * @throws TException
+   */
+  @InterfaceAudience.LimitedPrivate({"Apache Hive, HCatalog"})
+  void addWriteNotificationLog(WriteNotificationLogRequest rqst) throws TException;
+
   class IncompatibleMetastoreException extends MetaException {
     IncompatibleMetastoreException(String message) {
       super(message);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
index e0e65cf..de226bf 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreEventListener.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import java.sql.Connection;
 
@@ -282,6 +283,17 @@ public abstract class MetaStoreEventListener implements Configurable {
           throws MetaException {
   }
 
+  /**
+   * This will be called to perform acid write operation.
+   * @param acidWriteEvent event to be processed
+   * @param dbConn jdbc connection to remote meta store db.
+   * @param sqlGenerator helper class to generate db specific sql string.
+   * @throws MetaException
+   */
+  public void onAcidWrite(AcidWriteEvent acidWriteEvent, Connection dbConn, SQLGenerator sqlGenerator)
+          throws MetaException {
+  }
+
   @Override
   public Configuration getConf() {
     return this.conf;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
index 3cf8314..c296f57 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/MetaStoreListenerNotifier.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.tools.SQLGenerator;
 import java.sql.Connection;
 import java.util.List;
@@ -221,6 +222,8 @@ public class MetaStoreListenerNotifier {
               (listener, event) -> listener.onAbortTxn((AbortTxnEvent) event, null, null))
           .put(EventType.ALLOC_WRITE_ID,
               (listener, event) -> listener.onAllocWriteId((AllocWriteIdEvent) event, null, null))
+          .put(EventType.ACID_WRITE,
+                  (listener, event) -> listener.onAcidWrite((AcidWriteEvent) event, null, null))
           .build()
   );
 
@@ -241,6 +244,9 @@ public class MetaStoreListenerNotifier {
       .put(EventType.ALLOC_WRITE_ID,
         (listener, event, dbConn, sqlGenerator) ->
                 listener.onAllocWriteId((AllocWriteIdEvent) event, dbConn, sqlGenerator))
+      .put(EventType.ACID_WRITE,
+        (listener, event, dbConn, sqlGenerator) ->
+                listener.onAcidWrite((AcidWriteEvent) event, dbConn, sqlGenerator))
       .build()
   );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 7490243..8721022 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -159,6 +159,7 @@ import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMResourcePlanStatus;
 import org.apache.hadoop.hive.metastore.api.WMTrigger;
 import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.datasource.DataSourceProvider;
@@ -206,6 +207,7 @@ import org.apache.hadoop.hive.metastore.model.MWMPool;
 import org.apache.hadoop.hive.metastore.model.MWMResourcePlan;
 import org.apache.hadoop.hive.metastore.model.MWMResourcePlan.Status;
 import org.apache.hadoop.hive.metastore.model.MWMTrigger;
+import org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree;
 import org.apache.hadoop.hive.metastore.parser.ExpressionTree.FilterBuilder;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
@@ -9611,6 +9613,64 @@ public class ObjectStore implements RawStore, Configurable {
     }
   }
 
+  @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      long tmp = System.currentTimeMillis() / 1000 - olderThan;
+      int tooOld = (tmp > Integer.MAX_VALUE) ? 0 : (int) tmp;
+      query = pm.newQuery(MTxnWriteNotificationLog.class, "eventTime < tooOld");
+      query.declareParameters("java.lang.Integer tooOld");
+      Collection<MTxnWriteNotificationLog> toBeRemoved = (Collection) query.execute(tooOld);
+      if (CollectionUtils.isNotEmpty(toBeRemoved)) {
+        pm.deletePersistentAll(toBeRemoved);
+      }
+      commited = commitTransaction();
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    List<WriteEventInfo> writeEventInfoList = null;
+    boolean commited = false;
+    Query query = null;
+    try {
+      openTransaction();
+      List<String> parameterVals = new ArrayList<>();
+      StringBuilder filterBuilder = new StringBuilder(" txnId == " + Long.toString(txnId));
+      if (dbName != null && !"*".equals(dbName)) { // * means get all database, so no need to add filter
+        appendSimpleCondition(filterBuilder, "database", new String[]{dbName}, parameterVals);
+      }
+      if (tableName != null && !"*".equals(tableName)) {
+        appendSimpleCondition(filterBuilder, "table", new String[]{tableName}, parameterVals);
+      }
+      query = pm.newQuery(MTxnWriteNotificationLog.class, filterBuilder.toString());
+      query.setOrdering("database,table ascending");
+      List<MTxnWriteNotificationLog> mplans = (List<MTxnWriteNotificationLog>)query.executeWithArray(
+              parameterVals.toArray(new String[parameterVals.size()]));
+      pm.retrieveAll(mplans);
+      commited = commitTransaction();
+      if (mplans != null && mplans.size() > 0) {
+        writeEventInfoList = Lists.newArrayList();
+        for (MTxnWriteNotificationLog mplan : mplans) {
+          WriteEventInfo writeEventInfo = new WriteEventInfo(mplan.getWriteId(), mplan.getDatabase(),
+                  mplan.getTable(), mplan.getFiles());
+          writeEventInfo.setPartition(mplan.getPartition());
+          writeEventInfo.setPartitionObj(mplan.getPartObject());
+          writeEventInfo.setTableObj(mplan.getTableObject());
+          writeEventInfoList.add(writeEventInfo);
+        }
+      }
+    } finally {
+      rollbackAndCleanup(commited, query);
+    }
+    return writeEventInfoList;
+  }
+
   private void prepareQuotes() throws SQLException {
     if (dbType == DatabaseProduct.MYSQL) {
       assert pm.currentTransaction().isActive();

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
index d019941..73a518d 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/RawStore.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.api.CreationMetadata;
 import org.apache.hadoop.hive.metastore.api.ISchemaName;
 import org.apache.hadoop.hive.metastore.api.SchemaVersionDescriptor;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 
 import java.lang.annotation.ElementType;
 import java.lang.annotation.Retention;
@@ -1665,4 +1666,17 @@ public interface RawStore extends Configurable {
   Map<String, List<String>> getPartitionColsWithStats(String catName, String dbName,
       String tableName) throws MetaException, NoSuchObjectException;
 
+  /**
+   * Remove older notification events.
+   * @param olderThan Remove any events older than a given number of seconds
+   */
+  void cleanWriteNotificationEvents(int olderThan);
+
+  /**
+   * Get all write events for a specific transaction .
+   * @param txnId get all the events done by this transaction
+   * @param dbName the name of db for which dump is being taken
+   * @param tableName the name of the table for which the dump is being taken
+   */
+  List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException;
 }


[19/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
index f7018c2..ac1d3c8 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/ReplChangeManager.java
@@ -59,6 +59,7 @@ public class ReplChangeManager {
   static final String REMAIN_IN_TRASH_TAG = "user.remain-in-trash";
   private static final String URI_FRAGMENT_SEPARATOR = "#";
   public static final String SOURCE_OF_REPLICATION = "repl.source.for";
+  private static final String TXN_WRITE_EVENT_FILE_SEPARATOR = "]";
 
   public enum RecycleType {
     MOVE,
@@ -472,7 +473,6 @@ public class ReplChangeManager {
   }
 
   public static boolean isSourceOfReplication(Database db) {
-    // Can not judge, so assuming replication is not enabled.
     assert (db != null);
     String replPolicyIds = getReplPolicyIdString(db);
     return  !StringUtils.isEmpty(replPolicyIds);
@@ -490,4 +490,12 @@ public class ReplChangeManager {
     }
     return null;
   }
+
+  public static String joinWithSeparator(Iterable<?> strings) {
+    return org.apache.hadoop.util.StringUtils.join(TXN_WRITE_EVENT_FILE_SEPARATOR, strings);
+  }
+
+  public static String[] getListFromSeparatedString(String commaSeparatedString) {
+    return commaSeparatedString.split("\\s*" + TXN_WRITE_EVENT_FILE_SEPARATOR + "\\s*");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
index a526019..8ff056f 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/cache/CachedStore.java
@@ -111,6 +111,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
@@ -2414,6 +2415,17 @@ public class CachedStore implements RawStore, Configurable {
     return sharedCache.getUpdateCount();
   }
 
+  @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    rawStore.cleanWriteNotificationEvents(olderThan);
+  }
+
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return rawStore.getAllWriteEventInfo(txnId, dbName, tableName);
+  }
+
   static boolean isNotInBlackList(String catName, String dbName, String tblName) {
     String str = TableName.getQualified(catName, dbName, tblName);
     for (Pattern pattern : blacklistPatterns) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java
new file mode 100644
index 0000000..001179a
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/events/AcidWriteEvent.java
@@ -0,0 +1,91 @@
+/*
+ * 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.events;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.WriteNotificationLogRequest;
+import org.apache.hadoop.hive.metastore.utils.StringUtils;
+
+import java.util.List;
+
+/**
+ * AcidWriteEvent
+ * Event generated for acid write operations
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class AcidWriteEvent extends ListenerEvent {
+  private final WriteNotificationLogRequest writeNotificationLogRequest;
+  private final String partition;
+  private final Table tableObj;
+  private final Partition partitionObj;
+
+  public AcidWriteEvent(String partition, Table tableObj, Partition partitionObj,
+                        WriteNotificationLogRequest writeNotificationLogRequest) {
+    super(true, null);
+    this.writeNotificationLogRequest = writeNotificationLogRequest;
+    this.partition = partition;
+    this.tableObj = tableObj;
+    this.partitionObj = partitionObj;
+  }
+
+  public Long getTxnId() {
+    return writeNotificationLogRequest.getTxnId();
+  }
+
+  public List<String> getFiles() {
+    return writeNotificationLogRequest.getFileInfo().getFilesAdded();
+  }
+
+  public List<String> getChecksums() {
+    return writeNotificationLogRequest.getFileInfo().getFilesAddedChecksum();
+  }
+
+  public String getDatabase() {
+    return StringUtils.normalizeIdentifier(writeNotificationLogRequest.getDb());
+  }
+
+  public String getTable() {
+    return StringUtils.normalizeIdentifier(writeNotificationLogRequest.getTable());
+  }
+
+  public String getPartition() {
+    return partition; //Don't normalize partition value, as its case sensitive.
+  }
+
+  public Long getWriteId() {
+    return writeNotificationLogRequest.getWriteId();
+  }
+
+  public Table getTableObj() {
+    return tableObj;
+  }
+
+  public Partition getPartitionObj() {
+    return partitionObj;
+  }
+
+  public List<String> getSubDirs() {
+    return writeNotificationLogRequest.getFileInfo().getSubDirectoryList();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java
new file mode 100644
index 0000000..e2c9ccf
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/AcidWriteMessage.java
@@ -0,0 +1,50 @@
+/* * 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.messaging;
+
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import java.util.List;
+
+/**
+ * HCat message sent when an ACID write is done.
+ */
+public abstract class AcidWriteMessage extends EventMessage {
+
+  protected AcidWriteMessage() {
+    super(EventType.ACID_WRITE);
+  }
+
+  public abstract Long getTxnId();
+
+  public abstract String getTable();
+
+  public abstract Long getWriteId();
+
+  public abstract String getPartition();
+
+  public abstract List<String> getFiles();
+
+  public abstract Table getTableObj() throws Exception;
+
+  public abstract Partition getPartitionObj() throws Exception;
+
+  public abstract String getTableObjStr();
+
+  public abstract String getPartitionObjStr();
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
index 49004f2..9733039 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/CommitTxnMessage.java
@@ -17,6 +17,12 @@
 
 package org.apache.hadoop.hive.metastore.messaging;
 
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
+
+import java.util.List;
+
 /**
  * HCat message sent when an commit transaction is done.
  */
@@ -33,4 +39,21 @@ public abstract class CommitTxnMessage extends EventMessage {
    */
   public abstract Long getTxnId();
 
+  public abstract List<Long> getWriteIds();
+
+  public abstract List<String> getDatabases();
+
+  public abstract List<String> getTables();
+
+  public abstract List<String> getPartitions();
+
+  public abstract Table getTableObj(int idx) throws Exception;
+
+  public abstract Partition getPartitionObj(int idx) throws Exception;
+
+  public abstract String getFiles(int idx);
+
+  public abstract List<String> getFilesList();
+
+  public abstract void addWriteEventInfo(List<WriteEventInfo> writeEventInfoList);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
index 969dd7b..f24b419 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/EventMessage.java
@@ -60,7 +60,8 @@ public abstract class EventMessage {
     COMMIT_TXN(MessageFactory.COMMIT_TXN_EVENT),
     ABORT_TXN(MessageFactory.ABORT_TXN_EVENT),
     ALLOC_WRITE_ID(MessageFactory.ALLOC_WRITE_ID_EVENT),
-    ALTER_CATALOG(MessageFactory.ALTER_CATALOG_EVENT);
+    ALTER_CATALOG(MessageFactory.ALTER_CATALOG_EVENT),
+    ACID_WRITE(MessageFactory.ACID_WRITE_EVENT);
 
     private String typeString;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
index ca33579..b701d84 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageDeserializer.java
@@ -70,6 +70,10 @@ public abstract class MessageDeserializer {
       return getCommitTxnMessage(messageBody);
     case ABORT_TXN:
       return getAbortTxnMessage(messageBody);
+    case ALLOC_WRITE_ID:
+      return getAllocWriteIdMessage(messageBody);
+    case ACID_WRITE:
+      return getAcidWriteMessage(messageBody);
     default:
       throw new IllegalArgumentException("Unsupported event-type: " + eventTypeString);
     }
@@ -186,6 +190,11 @@ public abstract class MessageDeserializer {
    */
   public abstract AllocWriteIdMessage getAllocWriteIdMessage(String messageBody);
 
+  /*
+   * Method to de-serialize AcidWriteMessage instance.
+   */
+  public abstract AcidWriteMessage getAcidWriteMessage(String messageBody);
+
   // Protection against construction.
   protected MessageDeserializer() {}
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
index e0629ea..d529147 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/MessageFactory.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.utils.JavaUtils;
 
 import java.util.Iterator;
@@ -74,6 +75,7 @@ public abstract class MessageFactory {
   public static final String ABORT_TXN_EVENT = "ABORT_TXN";
   public static final String ALLOC_WRITE_ID_EVENT = "ALLOC_WRITE_ID_EVENT";
   public static final String ALTER_CATALOG_EVENT = "ALTER_CATALOG";
+  public static final String ACID_WRITE_EVENT = "ACID_WRITE_EVENT";
 
   private static MessageFactory instance = null;
 
@@ -326,4 +328,14 @@ public abstract class MessageFactory {
   public abstract DropCatalogMessage buildDropCatalogMessage(Catalog catalog);
 
   public abstract AlterCatalogMessage buildAlterCatalogMessage(Catalog oldCat, Catalog newCat);
+
+  /**
+   * Factory method for building acid write message
+   *
+   *
+   * @param acidWriteEvent information related to the acid write operation
+   * @param files files added by this write operation
+   * @return instance of AcidWriteMessage
+   */
+  public abstract AcidWriteMessage buildAcidWriteMessage(AcidWriteEvent acidWriteEvent, Iterator<String> files);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java
new file mode 100644
index 0000000..515a2cb
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONAcidWriteMessage.java
@@ -0,0 +1,150 @@
+/*
+ * 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.messaging.json;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
+import org.apache.thrift.TException;
+import org.codehaus.jackson.annotate.JsonProperty;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * JSON implementation of AcidWriteMessage
+ */
+public class JSONAcidWriteMessage extends AcidWriteMessage {
+
+  @JsonProperty
+  private Long txnid, writeId, timestamp;
+
+  @JsonProperty
+  private String server, servicePrincipal, database, table, partition, tableObjJson, partitionObjJson;
+
+  @JsonProperty
+  private List<String> files;
+
+  /**
+   * Default constructor, needed for Jackson.
+   */
+  public JSONAcidWriteMessage() {
+  }
+
+  public JSONAcidWriteMessage(String server, String servicePrincipal, Long timestamp, AcidWriteEvent acidWriteEvent,
+                              Iterator<String> files) {
+    this.timestamp = timestamp;
+    this.txnid = acidWriteEvent.getTxnId();
+    this.server = server;
+    this.servicePrincipal = servicePrincipal;
+    this.database = acidWriteEvent.getDatabase();
+    this.table = acidWriteEvent.getTable();
+    this.writeId = acidWriteEvent.getWriteId();
+    this.partition = acidWriteEvent.getPartition();
+    try {
+      this.tableObjJson = JSONMessageFactory.createTableObjJson(acidWriteEvent.getTableObj());
+      if (acidWriteEvent.getPartitionObj() != null) {
+        this.partitionObjJson = JSONMessageFactory.createPartitionObjJson(acidWriteEvent.getPartitionObj());
+      } else {
+        this.partitionObjJson = null;
+      }
+    } catch (TException e) {
+      throw new IllegalArgumentException("Could not serialize JSONAcidWriteMessage : ", e);
+    }
+    this.files = Lists.newArrayList(files);
+  }
+
+  @Override
+  public Long getTxnId() {
+    return txnid;
+  }
+
+  @Override
+  public Long getTimestamp() {
+    return timestamp;
+  }
+
+  @Override
+  public String getDB() {
+    return database;
+  }
+
+  @Override
+  public String getServicePrincipal() {
+    return servicePrincipal;
+  }
+
+  @Override
+  public String getServer() {
+    return server;
+  }
+
+  @Override
+  public String getTable() {
+    return table;
+  }
+
+  @Override
+  public Long getWriteId() {
+    return writeId;
+  }
+
+  @Override
+  public String getPartition() {
+    return partition;
+  }
+
+  @Override
+  public List<String> getFiles() {
+    return files;
+  }
+
+  @Override
+  public Table getTableObj() throws Exception {
+    return (tableObjJson == null) ? null : (Table) JSONMessageFactory.getTObj(tableObjJson, Table.class);
+  }
+
+  @Override
+  public Partition getPartitionObj() throws Exception {
+    return ((partitionObjJson == null) ? null :
+            (Partition) JSONMessageFactory.getTObj(partitionObjJson, Partition.class));
+  }
+
+  @Override
+  public String getTableObjStr() {
+    return tableObjJson;
+  }
+
+  @Override
+  public String getPartitionObjStr() {
+    return partitionObjJson;
+  }
+
+  @Override
+  public String toString() {
+    try {
+      return JSONMessageDeserializer.mapper.writeValueAsString(this);
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Could not serialize: ", exception);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
index 595a3d1..6082b8e 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONCommitTxnMessage.java
@@ -18,9 +18,15 @@
  */
 
 package org.apache.hadoop.hive.metastore.messaging.json;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.codehaus.jackson.annotate.JsonProperty;
 
+import java.util.List;
+
 /**
  * JSON implementation of CommitTxnMessage
  */
@@ -38,6 +44,12 @@ public class JSONCommitTxnMessage extends CommitTxnMessage {
   @JsonProperty
   private String servicePrincipal;
 
+  @JsonProperty
+  private List<Long> writeIds;
+
+  @JsonProperty
+  private List<String> databases, tables, partitions, tableObjs, partitionObjs, files;
+
   /**
    * Default constructor, needed for Jackson.
    */
@@ -49,6 +61,13 @@ public class JSONCommitTxnMessage extends CommitTxnMessage {
     this.txnid = txnid;
     this.server = server;
     this.servicePrincipal = servicePrincipal;
+    this.databases = null;
+    this.tables = null;
+    this.writeIds = null;
+    this.partitions = null;
+    this.tableObjs = null;
+    this.partitionObjs = null;
+    this.files = null;
   }
 
   @Override
@@ -77,6 +96,82 @@ public class JSONCommitTxnMessage extends CommitTxnMessage {
   }
 
   @Override
+  public List<Long> getWriteIds() {
+    return writeIds;
+  }
+
+  @Override
+  public List<String> getDatabases() {
+    return databases;
+  }
+
+  @Override
+  public List<String> getTables() {
+    return tables;
+  }
+
+  @Override
+  public List<String> getPartitions() {
+    return partitions;
+  }
+
+  @Override
+  public Table getTableObj(int idx) throws Exception {
+    return tableObjs == null ? null :  (Table) JSONMessageFactory.getTObj(tableObjs.get(idx), Table.class);
+  }
+
+  @Override
+  public Partition getPartitionObj(int idx) throws Exception {
+    return (partitionObjs == null ? null : (partitionObjs.get(idx) == null ? null :
+            (Partition)JSONMessageFactory.getTObj(partitionObjs.get(idx), Partition.class)));
+  }
+
+  @Override
+  public String getFiles(int idx) {
+    return files == null ? null : files.get(idx);
+  }
+
+  @Override
+  public List<String> getFilesList() {
+    return files;
+  }
+
+  @Override
+  public void addWriteEventInfo(List<WriteEventInfo> writeEventInfoList) {
+    if (this.databases == null) {
+      this.databases = Lists.newArrayList();
+    }
+    if (this.tables == null) {
+      this.tables = Lists.newArrayList();
+    }
+    if (this.writeIds == null) {
+      this.writeIds = Lists.newArrayList();
+    }
+    if (this.tableObjs == null) {
+      this.tableObjs = Lists.newArrayList();
+    }
+    if (this.partitions == null) {
+      this.partitions = Lists.newArrayList();
+    }
+    if (this.partitionObjs == null) {
+      this.partitionObjs = Lists.newArrayList();
+    }
+    if (this.files == null) {
+      this.files = Lists.newArrayList();
+    }
+
+    for (WriteEventInfo writeEventInfo : writeEventInfoList) {
+      this.databases.add(writeEventInfo.getDatabase());
+      this.tables.add(writeEventInfo.getTable());
+      this.writeIds.add(writeEventInfo.getWriteId());
+      this.partitions.add(writeEventInfo.getPartition());
+      this.tableObjs.add(writeEventInfo.getTableObj());
+      this.partitionObjs.add(writeEventInfo.getPartitionObj());
+      this.files.add(writeEventInfo.getFiles());
+    }
+  }
+
+  @Override
   public String toString() {
     try {
       return JSONMessageDeserializer.mapper.writeValueAsString(this);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
index f54e24d..be6b751 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageDeserializer.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
 import org.codehaus.jackson.map.DeserializationConfig;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.codehaus.jackson.map.SerializationConfig;
@@ -259,4 +260,12 @@ public class JSONMessageDeserializer extends MessageDeserializer {
       throw new IllegalArgumentException("Could not construct AllocWriteIdMessage", e);
     }
   }
+
+  public AcidWriteMessage getAcidWriteMessage(String messageBody) {
+    try {
+      return mapper.readValue(messageBody, JSONAcidWriteMessage.class);
+    } catch (Exception e) {
+      throw new IllegalArgumentException("Could not construct AcidWriteMessage", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
index d64c3ff..07f51f0 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/messaging/json/JSONMessageFactory.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.messaging.AddForeignKeyMessage;
 import org.apache.hadoop.hive.metastore.messaging.AddNotNullConstraintMessage;
 import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
@@ -66,6 +67,7 @@ import org.apache.hadoop.hive.metastore.messaging.OpenTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AbortTxnMessage;
 import org.apache.hadoop.hive.metastore.messaging.AllocWriteIdMessage;
+import org.apache.hadoop.hive.metastore.messaging.AcidWriteMessage;
 import org.apache.thrift.TBase;
 import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
@@ -230,11 +232,17 @@ public class JSONMessageFactory extends MessageFactory {
     return new JSONAbortTxnMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnId, now());
   }
 
+  @Override
   public AllocWriteIdMessage buildAllocWriteIdMessage(List<TxnToWriteId> txnToWriteIdList,
                                                       String dbName, String tableName) {
     return new JSONAllocWriteIdMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, txnToWriteIdList, dbName, tableName, now());
   }
 
+  @Override
+  public AcidWriteMessage buildAcidWriteMessage(AcidWriteEvent acidWriteEvent, Iterator<String> files) {
+    return new JSONAcidWriteMessage(MS_SERVER_URL, MS_SERVICE_PRINCIPAL, now(), acidWriteEvent, files);
+  }
+
   private long now() {
     return System.currentTimeMillis() / 1000;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java
new file mode 100644
index 0000000..f5ca386
--- /dev/null
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/model/MTxnWriteNotificationLog.java
@@ -0,0 +1,123 @@
+/*
+ * 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.model;
+
+/**
+ * MTxnWriteNotificationLog
+ * DN table for ACID write events.
+ */
+public class MTxnWriteNotificationLog {
+  private long txnId;
+  private long writeId;
+  private int eventTime;
+  private String database;
+  private String table;
+  private String partition;
+  private String tableObject;
+  private String partObject;
+  private String files;
+
+  public MTxnWriteNotificationLog() {
+  }
+
+  public MTxnWriteNotificationLog(long txnId, long writeId, int eventTime, String database, String table,
+                               String partition, String tableObject, String partObject, String files) {
+    this.txnId = txnId;
+    this.writeId = writeId;
+    this.eventTime = eventTime;
+    this.database = database;
+    this.table = table;
+    this.partition = partition;
+    this.tableObject = tableObject;
+    this.partObject = partObject;
+    this.files = files;
+  }
+
+  public long getTxnId() {
+    return txnId;
+  }
+
+  public void setTxnId(long txnId) {
+    this.txnId = txnId;
+  }
+
+  public long getWriteId() {
+    return writeId;
+  }
+
+  public void setWriteId(long writeId) {
+    this.writeId = writeId;
+  }
+
+  public int getEventTime() {
+    return eventTime;
+  }
+
+  public void setEventTime(int eventTime) {
+    this.eventTime = eventTime;
+  }
+
+  public String getDatabase() {
+    return database;
+  }
+
+  public void setDatabase(String database) {
+    this.database = database;
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public String getPartition() {
+    return partition;
+  }
+
+  public void setPartition(String partition) {
+    this.partition = partition;
+  }
+
+  public String getTableObject() {
+    return tableObject;
+  }
+
+  public void setTableObject(String tableObject) {
+    this.tableObject = tableObject;
+  }
+
+  public String getPartObject() {
+    return partObject;
+  }
+
+  public void setPartObject(String partObject) {
+    this.partObject = partObject;
+  }
+
+  public String getFiles() {
+    return files;
+  }
+
+  public void setFiles(String files) {
+    this.files = files;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
index b23a6d7..d0ac7db 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/tools/SQLGenerator.java
@@ -175,4 +175,13 @@ public final class SQLGenerator {
     return dbProduct;
   }
 
+  // This is required for SQL executed directly. If the SQL has double quotes then some dbs tend to
+  // remove the escape characters and store the variable without double quote.
+  public String addEscapeCharacters(String s) {
+    if (dbProduct ==  DatabaseProduct.MYSQL) {
+      return s.replaceAll("\\\\", "\\\\\\\\");
+    }
+    return s;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
index 50bfca3..f8c2ca2 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnDbUtil.java
@@ -245,6 +245,34 @@ public final class TxnDbUtil {
       stmt.execute("INSERT INTO \"APP\".\"NOTIFICATION_SEQUENCE\" (\"NNI_ID\", \"NEXT_EVENT_ID\")" +
               " SELECT * FROM (VALUES (1,1)) tmp_table WHERE NOT EXISTS ( SELECT " +
               "\"NEXT_EVENT_ID\" FROM \"APP\".\"NOTIFICATION_SEQUENCE\")");
+
+      try {
+        stmt.execute("CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (" +
+                "WNL_ID bigint NOT NULL," +
+                "WNL_TXNID bigint NOT NULL," +
+                "WNL_WRITEID bigint NOT NULL," +
+                "WNL_DATABASE varchar(128) NOT NULL," +
+                "WNL_TABLE varchar(128) NOT NULL," +
+                "WNL_PARTITION varchar(1024) NOT NULL," +
+                "WNL_TABLE_OBJ clob NOT NULL," +
+                "WNL_PARTITION_OBJ clob," +
+                "WNL_FILES clob," +
+                "WNL_EVENT_TIME integer NOT NULL," +
+                "PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION))"
+        );
+      } catch (SQLException e) {
+        if (e.getMessage() != null && e.getMessage().contains("already exists")) {
+          LOG.info("TXN_WRITE_NOTIFICATION_LOG table already exist, ignoring");
+        } else {
+          throw e;
+        }
+      }
+
+      stmt.execute("INSERT INTO \"APP\".\"SEQUENCE_TABLE\" (\"SEQUENCE_NAME\", \"NEXT_VAL\") " +
+              "SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', " +
+              "1)) tmp_table WHERE NOT EXISTS ( SELECT \"NEXT_VAL\" FROM \"APP\"" +
+              ".\"SEQUENCE_TABLE\" WHERE \"SEQUENCE_NAME\" = 'org.apache.hadoop.hive.metastore" +
+              ".model.MTxnWriteNotificationLog')");
     } catch (SQLException e) {
       try {
         conn.rollback();

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 361ede5..3785f89 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hive.metastore.events.AbortTxnEvent;
 import org.apache.hadoop.hive.metastore.events.AllocWriteIdEvent;
 import org.apache.hadoop.hive.metastore.events.CommitTxnEvent;
 import org.apache.hadoop.hive.metastore.events.OpenTxnEvent;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 import org.apache.hadoop.hive.metastore.messaging.EventMessage;
 import org.apache.hadoop.hive.metastore.metrics.Metrics;
 import org.apache.hadoop.hive.metastore.metrics.MetricsConstants;
@@ -698,6 +699,38 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
   @Override
   @RetrySemantics.Idempotent
+  public long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException {
+    try {
+      Connection dbConn = null;
+      Statement stmt = null;
+      try {
+        lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        stmt = dbConn.createStatement();
+        List<Long> targetTxnIds = getTargetTxnIdList(replPolicy, Collections.singletonList(sourceTxnId), stmt);
+        if (targetTxnIds.isEmpty()) {
+          LOG.info("Txn {} not present for repl policy {}", sourceTxnId, replPolicy);
+          return -1;
+        }
+        assert (targetTxnIds.size() == 1);
+        return targetTxnIds.get(0);
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        checkRetryable(dbConn, e, "getTargetTxnId(" + replPolicy + sourceTxnId + ")");
+        throw new MetaException("Unable to get target transaction id "
+                + StringUtils.stringifyException(e));
+      } finally {
+        close(null, stmt, dbConn);
+        unlockInternal();
+      }
+    } catch (RetryException e) {
+      return getTargetTxnId(replPolicy, sourceTxnId);
+    }
+  }
+
+  @Override
+  @RetrySemantics.Idempotent
   public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaException, TxnAbortedException {
     long txnid = rqst.getTxnid();
     long sourceTxnId = -1;
@@ -892,10 +925,17 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           shouldNeverHappen(txnid);
           //dbConn is rolled back in finally{}
         }
-        String conflictSQLSuffix = "from TXN_COMPONENTS where tc_txnid=" + txnid + " and tc_operation_type IN(" +
-          quoteChar(OpertaionType.UPDATE.sqlConst) + "," + quoteChar(OpertaionType.DELETE.sqlConst) + ")";
-        rs = stmt.executeQuery(sqlGenerator.addLimitClause(1, "tc_operation_type " + conflictSQLSuffix));
-        if (rs.next()) {
+
+        String conflictSQLSuffix = null;
+        if (rqst.isSetReplPolicy()) {
+          rs = null;
+        } else {
+          conflictSQLSuffix = "from TXN_COMPONENTS where tc_txnid=" + txnid + " and tc_operation_type IN(" +
+                  quoteChar(OpertaionType.UPDATE.sqlConst) + "," + quoteChar(OpertaionType.DELETE.sqlConst) + ")";
+          rs = stmt.executeQuery(sqlGenerator.addLimitClause(1,
+                  "tc_operation_type " + conflictSQLSuffix));
+        }
+        if (rs != null && rs.next()) {
           isUpdateDelete = true;
           close(rs);
           //if here it means currently committing txn performed update/delete and we should check WW conflict
@@ -984,23 +1024,52 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
            * Consider: if RO txn is after a W txn, then RO's openTxns() will be mutexed with W's
            * commitTxn() because both do S4U on NEXT_TXN_ID and thus RO will see result of W txn.
            * If RO < W, then there is no reads-from relationship.
+           * In replication flow we don't expect any write write conflict as it should have been handled at source.
            */
         }
-        // Move the record from txn_components into completed_txn_components so that the compactor
-        // knows where to look to compact.
-        String s = "insert into COMPLETED_TXN_COMPONENTS (ctc_txnid, ctc_database, " +
-            "ctc_table, ctc_partition, ctc_writeid) select tc_txnid, tc_database, tc_table, " +
-            "tc_partition, tc_writeid from TXN_COMPONENTS where tc_txnid = " + txnid;
-        LOG.debug("Going to execute insert <" + s + ">");
-        int modCount = 0;
-        if ((modCount = stmt.executeUpdate(s)) < 1) {
-          //this can be reasonable for an empty txn START/COMMIT or read-only txn
-          //also an IUD with DP that didn't match any rows.
-          LOG.info("Expected to move at least one record from txn_components to " +
-            "completed_txn_components when committing txn! " + JavaUtils.txnIdToString(txnid));
+
+        String s;
+        if (!rqst.isSetReplPolicy()) {
+          // Move the record from txn_components into completed_txn_components so that the compactor
+          // knows where to look to compact.
+          s = "insert into COMPLETED_TXN_COMPONENTS (ctc_txnid, ctc_database, " +
+                  "ctc_table, ctc_partition, ctc_writeid) select tc_txnid, tc_database, tc_table, " +
+                  "tc_partition, tc_writeid from TXN_COMPONENTS where tc_txnid = " + txnid;
+          LOG.debug("Going to execute insert <" + s + ">");
+
+          if ((stmt.executeUpdate(s)) < 1) {
+            //this can be reasonable for an empty txn START/COMMIT or read-only txn
+            //also an IUD with DP that didn't match any rows.
+            LOG.info("Expected to move at least one record from txn_components to " +
+                    "completed_txn_components when committing txn! " + JavaUtils.txnIdToString(txnid));
+          }
+        } else {
+          if (rqst.isSetWriteEventInfos()) {
+            List<String> rows = new ArrayList<>();
+            for (WriteEventInfo writeEventInfo : rqst.getWriteEventInfos()) {
+              rows.add(txnid + "," + quoteString(writeEventInfo.getDatabase()) + "," +
+                      quoteString(writeEventInfo.getTable()) + "," +
+                      quoteString(writeEventInfo.getPartition()) + "," +
+                      writeEventInfo.getWriteId());
+            }
+            List<String> queries = sqlGenerator.createInsertValuesStmt("COMPLETED_TXN_COMPONENTS " +
+                    "(ctc_txnid," + " ctc_database, ctc_table, ctc_partition, ctc_writeid)", rows);
+            for (String q : queries) {
+              LOG.debug("Going to execute insert  <" + q + "> ");
+              stmt.execute(q);
+            }
+          }
+
+          s = "delete from REPL_TXN_MAP where RTM_SRC_TXN_ID = " + sourceTxnId +
+                  " and RTM_REPL_POLICY = " + quoteString(rqst.getReplPolicy());
+          LOG.info("Repl going to execute  <" + s + ">");
+          stmt.executeUpdate(s);
         }
+
         // Obtain information that we need to update registry
-        s = "select ctc_database, ctc_table, ctc_writeid, ctc_timestamp from COMPLETED_TXN_COMPONENTS where ctc_txnid = " + txnid;
+        s = "select ctc_database, ctc_table, ctc_writeid, ctc_timestamp from COMPLETED_TXN_COMPONENTS" +
+                " where ctc_txnid = " + txnid;
+
         LOG.debug("Going to extract table modification information for invalidation cache <" + s + ">");
         rs = stmt.executeQuery(s);
         while (rs.next()) {
@@ -1008,27 +1077,21 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           txnComponents.add(new TransactionRegistryInfo(rs.getString(1), rs.getString(2),
               rs.getLong(3), rs.getTimestamp(4, Calendar.getInstance(TimeZone.getTimeZone("UTC"))).getTime()));
         }
+
+        // cleanup all txn related metadata
         s = "delete from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         s = "delete from HIVE_LOCKS where hl_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         s = "delete from TXNS where txn_id = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         s = "delete from MIN_HISTORY_LEVEL where mhl_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        modCount = stmt.executeUpdate(s);
+        stmt.executeUpdate(s);
         LOG.info("Removed committed transaction: (" + txnid + ") from MIN_HISTORY_LEVEL");
-
-        if (rqst.isSetReplPolicy()) {
-          s = "delete from REPL_TXN_MAP where RTM_SRC_TXN_ID = " + sourceTxnId +
-                  " and RTM_REPL_POLICY = " + quoteString(rqst.getReplPolicy());
-          LOG.info("Repl going to execute  <" + s + ">");
-          stmt.executeUpdate(s);
-        }
-
         if (transactionalListeners != null) {
           MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
                   EventMessage.EventType.COMMIT_TXN, new CommitTxnEvent(txnid, null), dbConn, sqlGenerator);
@@ -1548,6 +1611,43 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
 
   }
   @Override
+  @RetrySemantics.Idempotent
+  public void addWriteNotificationLog(AcidWriteEvent acidWriteEvent)
+          throws MetaException {
+    Connection dbConn = null;
+    try {
+      try {
+        //Idempotent case is handled by notify Event
+        lockInternal();
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+        MetaStoreListenerNotifier.notifyEventWithDirectSql(transactionalListeners,
+                EventMessage.EventType.ACID_WRITE, acidWriteEvent, dbConn, sqlGenerator);
+        LOG.debug("Going to commit");
+        dbConn.commit();
+        return;
+      } catch (SQLException e) {
+        LOG.debug("Going to rollback");
+        rollbackDBConn(dbConn);
+        if (isDuplicateKeyError(e)) {
+          // in case of key duplicate error, retry as it might be because of race condition
+          if (waitForRetry("addWriteNotificationLog(" + acidWriteEvent + ")", e.getMessage())) {
+            throw new RetryException();
+          }
+          retryNum = 0;
+          throw new MetaException(e.getMessage());
+        }
+        checkRetryable(dbConn, e, "addWriteNotificationLog(" + acidWriteEvent + ")");
+        throw new MetaException("Unable to add write notification event " + StringUtils.stringifyException(e));
+      } finally{
+        closeDbConn(dbConn);
+        unlockInternal();
+      }
+    } catch (RetryException e) {
+      addWriteNotificationLog(acidWriteEvent);
+    }
+  }
+
+  @Override
   @RetrySemantics.SafeToRetry
   public void performWriteSetGC() {
     Connection dbConn = null;
@@ -3046,6 +3146,22 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     closeStmt(stmt);
     closeDbConn(dbConn);
   }
+
+  private boolean waitForRetry(String caller, String errMsg) {
+    if (retryNum++ < retryLimit) {
+      LOG.warn("Retryable error detected in " + caller + ".  Will wait " + retryInterval +
+              "ms and retry up to " + (retryLimit - retryNum + 1) + " times.  Error: " + errMsg);
+      try {
+        Thread.sleep(retryInterval);
+      } catch (InterruptedException ex) {
+        //
+      }
+      return true;
+    } else {
+      LOG.error("Fatal error in " + caller + ". Retry limit (" + retryLimit + ") reached. Last error: " + errMsg);
+    }
+    return false;
+  }
   /**
    * Determine if an exception was such that it makes sense to retry.  Unfortunately there is no standard way to do
    * this, so we have to inspect the error messages and catch the telltale signs for each
@@ -3089,18 +3205,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         }
       } else if (isRetryable(conf, e)) {
         //in MSSQL this means Communication Link Failure
-        if (retryNum++ < retryLimit) {
-          LOG.warn("Retryable error detected in " + caller + ".  Will wait " + retryInterval +
-            "ms and retry up to " + (retryLimit - retryNum + 1) + " times.  Error: " + getMessage(e));
-          try {
-            Thread.sleep(retryInterval);
-          } catch (InterruptedException ex) {
-            //
-          }
-          sendRetrySignal = true;
-        } else {
-          LOG.error("Fatal error in " + caller + ". Retry limit (" + retryLimit + ") reached. Last error: " + getMessage(e));
-        }
+        sendRetrySignal = waitForRetry(caller, e.getMessage());
       }
       else {
         //make sure we know we saw an error that we don't recognize

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
index ef447e1..d972d10 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.common.classification.RetrySemantics;
 import org.apache.hadoop.hive.metastore.api.*;
+import org.apache.hadoop.hive.metastore.events.AcidWriteEvent;
 
 import java.sql.SQLException;
 import java.util.Iterator;
@@ -86,6 +87,9 @@ public interface TxnStore extends Configurable {
   @RetrySemantics.Idempotent
   OpenTxnsResponse openTxns(OpenTxnRequest rqst) throws MetaException;
 
+  @RetrySemantics.Idempotent
+  long getTargetTxnId(String replPolicy, long sourceTxnId) throws MetaException;
+
   /**
    * Abort (rollback) a transaction.
    * @param rqst info on transaction to abort
@@ -476,4 +480,11 @@ public interface TxnStore extends Configurable {
    */
   @RetrySemantics.Idempotent
   void setHadoopJobId(String hadoopJobId, long id);
+
+  /**
+   * Add the ACID write event information to writeNotificationLog table.
+   * @param acidWriteEvent
+   */
+  @RetrySemantics.Idempotent
+  void addWriteNotificationLog(AcidWriteEvent acidWriteEvent) throws MetaException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
index 963e12f..154db4b 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/utils/FileUtils.java
@@ -511,7 +511,6 @@ public class FileUtils {
     return new Path(scheme, authority, pathUri.getPath());
   }
 
-
   /**
    * Returns a BEST GUESS as to whether or not other is a subdirectory of parent. It does not
    * take into account any intricacies of the underlying file system, which is assumed to be
@@ -524,4 +523,15 @@ public class FileUtils {
   public static boolean isSubdirectory(String parent, String other) {
     return other.startsWith(parent.endsWith(Path.SEPARATOR) ? parent : parent + Path.SEPARATOR);
   }
+
+  public static Path getTransformedPath(String name, String subDir, String root) {
+    if (root != null) {
+      Path newPath = new Path(root);
+      if (subDir != null) {
+        newPath = new Path(newPath, subDir);
+      }
+      return new Path(newPath, name);
+    }
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/resources/package.jdo
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/resources/package.jdo b/standalone-metastore/src/main/resources/package.jdo
index 1be3e98..5fb548c 100644
--- a/standalone-metastore/src/main/resources/package.jdo
+++ b/standalone-metastore/src/main/resources/package.jdo
@@ -1182,6 +1182,41 @@
       </field>
     </class>
 
+    <class name="MTxnWriteNotificationLog" table="TXN_WRITE_NOTIFICATION_LOG" identity-type="datastore" detachable="true">
+      <datastore-identity strategy="increment"/>
+      <datastore-identity key-cache-size="1"/>
+      <datastore-identity>
+        <column name="WNL_ID"/>
+      </datastore-identity>
+      <field name="txnId">
+        <column name="WNL_TXNID" jdbc-type="BIGINT" allows-null="false"/>
+      </field>
+      <field name="writeId">
+        <column name="WNL_WRITEID" jdbc-type="BIGINT" allows-null="false"/>
+      </field>
+      <field name="database">
+        <column name="WNL_DATABASE" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="table">
+        <column name="WNL_TABLE" length="128" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="partition">
+        <column name="WNL_PARTITION" length="1024" jdbc-type="VARCHAR" allows-null="false"/>
+      </field>
+      <field name="tableObject">
+        <column name="WNL_TABLE_OBJ" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="partObject">
+        <column name="WNL_PARTITION_OBJ" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="files">
+        <column name="WNL_FILES" jdbc-type="LONGVARCHAR"/>
+      </field>
+      <field name="eventTime">
+        <column name="WNL_EVENT_TIME" jdbc-type="INTEGER" allows-null="false"/>
+      </field>
+    </class>
+
     <class name="MWMResourcePlan" identity-type="datastore" table="WM_RESOURCEPLAN" detachable="true">
       <datastore-identity>
         <column name="RP_ID"/>

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
index 352b43e..a696d06 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-3.1.0.derby.sql
@@ -689,6 +689,21 @@ CREATE TABLE "APP"."RUNTIME_STATS" (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME integer NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
index bb69105..7cab4fb 100644
--- a/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/hive-schema-4.0.0.derby.sql
@@ -689,6 +689,21 @@ CREATE TABLE "APP"."RUNTIME_STATS" (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME integer NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
index 7b7a8a2..10f1373 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-2.3.0-to-3.0.0.derby.sql
@@ -244,7 +244,6 @@ CREATE TABLE MIN_HISTORY_LEVEL (
 
 CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID);
 
-
 CREATE TABLE "APP"."RUNTIME_STATS" (
   "RS_ID" bigint primary key,
   "CREATE_TIME" integer not null,

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql b/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
index 6621ef7..7058ab0 100644
--- a/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
+++ b/standalone-metastore/src/main/sql/derby/upgrade-3.0.0-to-3.1.0.derby.sql
@@ -29,5 +29,21 @@ ALTER TABLE TXNS ADD COLUMN TXN_TYPE integer;
 
 CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_NAME");
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME integer NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- This needs to be the last thing done.  Insert any changes above this line.
 UPDATE "APP".VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
index bc11b40..d7722dc 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-3.1.0.mssql.sql
@@ -1248,6 +1248,23 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE nvarchar(128) NOT NULL,
+  WNL_TABLE nvarchar(128) NOT NULL,
+  WNL_PARTITION nvarchar(1024) NOT NULL,
+  WNL_TABLE_OBJ text NOT NULL,
+  WNL_PARTITION_OBJ text,
+  WNL_FILES text,
+  WNL_EVENT_TIME int NOT NULL
+);
+
+ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
index 922e8fe..a81fc40 100644
--- a/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql
@@ -1249,6 +1249,23 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE nvarchar(128) NOT NULL,
+  WNL_TABLE nvarchar(128) NOT NULL,
+  WNL_PARTITION nvarchar(1024) NOT NULL,
+  WNL_TABLE_OBJ text NOT NULL,
+  WNL_PARTITION_OBJ text,
+  WNL_FILES text,
+  WNL_EVENT_TIME int NOT NULL
+);
+
+ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql b/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
index abb80d6..41f23f7 100644
--- a/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
+++ b/standalone-metastore/src/main/sql/mssql/upgrade-3.0.0-to-3.1.0.mssql.sql
@@ -30,6 +30,22 @@ ALTER TABLE TXNS ADD TXN_TYPE int NULL;
 
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME);
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE nvarchar(128) NOT NULL,
+  WNL_TABLE nvarchar(128) NOT NULL,
+  WNL_PARTITION nvarchar(1024) NOT NULL,
+  WNL_TABLE_OBJ text NOT NULL,
+  WNL_PARTITION_OBJ text,
+  WNL_FILES text,
+  WNL_EVENT_TIME int NOT NULL
+);
+ALTER TABLE TXN_WRITE_NOTIFICATION_LOG ADD CONSTRAINT TXN_WRITE_NOTIFICATION_LOG_PK PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
index c54df55..c65af1e 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.0.0.mysql.sql
@@ -1155,7 +1155,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 ) ENGINE=InnoDB DEFAULT CHARSET=latin1;
 
-
 CREATE TABLE RUNTIME_STATS (
   RS_ID bigint primary key,
   CREATE_TIME bigint NOT NULL,

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
index af955dc..29d4a43 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-3.1.0.mysql.sql
@@ -1173,6 +1173,22 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ longtext NOT NULL,
+  WNL_PARTITION_OBJ longtext,
+  WNL_FILES longtext,
+  WNL_EVENT_TIME INT(11) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
index 6c40e6e..968f4a4 100644
--- a/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql
@@ -1173,6 +1173,22 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ longtext NOT NULL,
+  WNL_PARTITION_OBJ longtext,
+  WNL_FILES longtext,
+  WNL_EVENT_TIME INT(11) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+
+INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
index 9b87563..786e38a 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-2.3.0-to-3.0.0.mysql.sql
@@ -319,8 +319,8 @@ UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
 
 ALTER TABLE TXN_COMPONENTS MODIFY COLUMN TC_TABLE varchar(128) NULL;
 
+ALTER TABLE `TBLS` ADD COLUMN `OWNER_TYPE` VARCHAR(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS ' ';
-
-ALTER TABLE `TBLS` ADD COLUMN `OWNER_TYPE` VARCHAR(10) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql b/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
index 305fa1d..e103bef 100644
--- a/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
+++ b/standalone-metastore/src/main/sql/mysql/upgrade-3.0.0-to-3.1.0.mysql.sql
@@ -30,6 +30,22 @@ ALTER TABLE TXNS ADD COLUMN TXN_TYPE int DEFAULT NULL;
 
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME) USING BTREE;
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID bigint NOT NULL,
+  WNL_TXNID bigint NOT NULL,
+  WNL_WRITEID bigint NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ longtext NOT NULL,
+  WNL_PARTITION_OBJ longtext,
+  WNL_FILES longtext,
+  WNL_EVENT_TIME INT(11) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+) ENGINE=InnoDB DEFAULT CHARSET=latin1;
+INSERT INTO `SEQUENCE_TABLE` (`SEQUENCE_NAME`, `NEXT_VAL`) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
index 63cc1f7..3e2e282 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.0.0.oracle.sql
@@ -1134,7 +1134,6 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
index bc13703..9adea31 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-3.1.0.oracle.sql
@@ -1143,6 +1143,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID number(19) NOT NULL,
+  WNL_TXNID number(19) NOT NULL,
+  WNL_WRITEID number(19) NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME number(10) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
index e12150a..faca669 100644
--- a/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql
@@ -1143,6 +1143,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID number(19) NOT NULL,
+  WNL_TXNID number(19) NOT NULL,
+  WNL_WRITEID number(19) NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME number(10) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
index ce3437f..71f5034 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-2.3.0-to-3.0.0.oracle.sql
@@ -335,8 +335,8 @@ INSERT INTO TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID, T2W_WRITEID)
 UPDATE TXN_COMPONENTS SET TC_WRITEID = TC_TXNID;
 UPDATE COMPLETED_TXN_COMPONENTS SET CTC_WRITEID = CTC_TXNID;
 
+ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR2(10) NULL;
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.0.0', VERSION_COMMENT='Hive release version 3.0.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.3.0 to 3.0.0' AS Status from dual;
-
-ALTER TABLE TBLS ADD OWNER_TYPE VARCHAR2(10) NULL;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql b/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
index ccdea54..cf8699b 100644
--- a/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
+++ b/standalone-metastore/src/main/sql/oracle/upgrade-3.0.0-to-3.1.0.oracle.sql
@@ -30,6 +30,22 @@ ALTER TABLE TXNS ADD TXN_TYPE number(10) NULL;
 
 CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (CAT_NAME, DB_NAME, TABLE_NAME, COLUMN_NAME);
 
+-- HIVE-19267
+CREATE TABLE TXN_WRITE_NOTIFICATION_LOG (
+  WNL_ID number(19) NOT NULL,
+  WNL_TXNID number(19) NOT NULL,
+  WNL_WRITEID number(19) NOT NULL,
+  WNL_DATABASE varchar(128) NOT NULL,
+  WNL_TABLE varchar(128) NOT NULL,
+  WNL_PARTITION varchar(1024) NOT NULL,
+  WNL_TABLE_OBJ clob NOT NULL,
+  WNL_PARTITION_OBJ clob,
+  WNL_FILES clob,
+  WNL_EVENT_TIME number(10) NOT NULL,
+  PRIMARY KEY (WNL_TXNID, WNL_DATABASE, WNL_TABLE, WNL_PARTITION)
+);
+INSERT INTO SEQUENCE_TABLE (SEQUENCE_NAME, NEXT_VAL) VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE VERSION SET SCHEMA_VERSION='3.1.0', VERSION_COMMENT='Hive release version 3.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
index 97697f8..b89c87f 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.0.0.postgres.sql
@@ -1812,7 +1812,6 @@ CREATE TABLE REPL_TXN_MAP (
   PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID)
 );
 
-
 CREATE TABLE RUNTIME_STATS (
  RS_ID bigint primary key,
  CREATE_TIME bigint NOT NULL,
@@ -1822,7 +1821,6 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
-
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script
 -- -----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
index 36bab70..7a8a419 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-3.1.0.postgres.sql
@@ -1834,6 +1834,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
+  "WNL_ID" bigint NOT NULL,
+  "WNL_TXNID" bigint NOT NULL,
+  "WNL_WRITEID" bigint NOT NULL,
+  "WNL_DATABASE" varchar(128) NOT NULL,
+  "WNL_TABLE" varchar(128) NOT NULL,
+  "WNL_PARTITION" varchar(1024) NOT NULL,
+  "WNL_TABLE_OBJ" text NOT NULL,
+  "WNL_PARTITION_OBJ" text,
+  "WNL_FILES" text,
+  "WNL_EVENT_TIME" integer NOT NULL,
+  PRIMARY KEY ("WNL_TXNID", "WNL_DATABASE", "WNL_TABLE", "WNL_PARTITION")
+);
+
+INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
index b73e1d1..2e7ac5a 100644
--- a/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql
@@ -1836,6 +1836,21 @@ CREATE TABLE RUNTIME_STATS (
 
 CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME);
 
+CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
+  "WNL_ID" bigint NOT NULL,
+  "WNL_TXNID" bigint NOT NULL,
+  "WNL_WRITEID" bigint NOT NULL,
+  "WNL_DATABASE" varchar(128) NOT NULL,
+  "WNL_TABLE" varchar(128) NOT NULL,
+  "WNL_PARTITION" varchar(1024) NOT NULL,
+  "WNL_TABLE_OBJ" text NOT NULL,
+  "WNL_PARTITION_OBJ" text,
+  "WNL_FILES" text,
+  "WNL_EVENT_TIME" integer NOT NULL,
+  PRIMARY KEY ("WNL_TXNID", "WNL_DATABASE", "WNL_TABLE", "WNL_PARTITION")
+);
+
+INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
 
 -- -----------------------------------------------------------------
 -- Record schema version. Should be the last step in the init script

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql b/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
index 2c0eb31..445c3a2 100644
--- a/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
+++ b/standalone-metastore/src/main/sql/postgres/upgrade-3.0.0-to-3.1.0.postgres.sql
@@ -32,6 +32,22 @@ ALTER TABLE TXNS ADD COLUMN TXN_TYPE integer DEFAULT NULL;
 
 CREATE INDEX "TAB_COL_STATS_IDX" ON "TAB_COL_STATS" USING btree ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME");
 
+-- HIVE-19267
+CREATE TABLE "TXN_WRITE_NOTIFICATION_LOG" (
+  "WNL_ID" bigint NOT NULL,
+  "WNL_TXNID" bigint NOT NULL,
+  "WNL_WRITEID" bigint NOT NULL,
+  "WNL_DATABASE" varchar(128) NOT NULL,
+  "WNL_TABLE" varchar(128) NOT NULL,
+  "WNL_PARTITION" varchar(1024) NOT NULL,
+  "WNL_TABLE_OBJ" text NOT NULL,
+  "WNL_PARTITION_OBJ" text,
+  "WNL_FILES" text,
+  "WNL_EVENT_TIME" integer NOT NULL,
+  PRIMARY KEY ("WNL_TXNID", "WNL_DATABASE", "WNL_TABLE", "WNL_PARTITION")
+);
+INSERT INTO "SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") VALUES ('org.apache.hadoop.hive.metastore.model.MTxnWriteNotificationLog', 1);
+
 -- These lines need to be last.  Insert any changes above.
 UPDATE "VERSION" SET "SCHEMA_VERSION"='3.1.0', "VERSION_COMMENT"='Hive release version 3.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 3.0.0 to 3.1.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index 6e503eb..1ca6454 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -867,6 +867,18 @@ struct AbortTxnsRequest {
 struct CommitTxnRequest {
     1: required i64 txnid,
     2: optional string replPolicy,
+    // Information related to write operations done in this transaction.
+    3: optional list<WriteEventInfo> writeEventInfos,
+}
+
+struct WriteEventInfo {
+    1: required i64    writeId,
+    2: required string database,
+    3: required string table,
+    4: required string files,
+    5: optional string partition,
+    6: optional string tableObj, // repl txn task does not need table object for commit
+    7: optional string partitionObj,
 }
 
 struct ReplTblWriteIdStateRequest {
@@ -1102,6 +1114,8 @@ struct InsertEventRequestData {
     2: required list<string> filesAdded,
     // Checksum of files (hex string of checksum byte payload)
     3: optional list<string> filesAddedChecksum,
+    // Used by acid operation to create the sub directory
+    4: optional list<string> subDirectoryList,
 }
 
 union FireEventRequestData {
@@ -1122,7 +1136,20 @@ struct FireEventRequest {
 struct FireEventResponse {
     // NOP for now, this is just a place holder for future responses
 }
-    
+
+struct WriteNotificationLogRequest {
+    1: required i64 txnId,
+    2: required i64 writeId,
+    3: required string db,
+    4: required string table,
+    5: required InsertEventRequestData fileInfo,
+    6: optional list<string> partitionVals,
+}
+
+struct WriteNotificationLogResponse {
+    // NOP for now, this is just a place holder for future responses
+}
+
 struct MetadataPpdResult {
   1: optional binary metadata,
   2: optional binary includeBitset
@@ -2104,6 +2131,7 @@ service ThriftHiveMetastore extends fb303.FacebookService
   NotificationEventsCountResponse get_notification_events_count(1:NotificationEventsCountRequest rqst)
   FireEventResponse fire_listener_event(1:FireEventRequest rqst)
   void flushCache()
+  WriteNotificationLogResponse add_write_notification_log(WriteNotificationLogRequest rqst)
 
   // Repl Change Management api
   CmRecycleResponse cm_recycle(1:CmRecycleRequest request) throws(1:MetaException o1)

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
index c482d28..2454479 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreControlledCommit.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.ColStatsObjWithSourceInfo;
 import org.apache.thrift.TException;
@@ -1195,6 +1196,16 @@ public class DummyRawStoreControlledCommit implements RawStore, Configurable {
   }
 
   @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+    objectStore.cleanWriteNotificationEvents(olderThan);
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return objectStore.getAllWriteEventInfo(txnId, dbName, tableName);
+  }
+
+  @Override
   public List<TableName> getTableNamesWithStats() throws MetaException,
       NoSuchObjectException {
     return null;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
index d253005..9b79446 100644
--- a/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
+++ b/standalone-metastore/src/test/java/org/apache/hadoop/hive/metastore/DummyRawStoreForJdoConnection.java
@@ -83,6 +83,7 @@ import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.WMMapping;
 import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WriteEventInfo;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
@@ -1199,4 +1200,13 @@ public class DummyRawStoreForJdoConnection implements RawStore {
       NoSuchObjectException {
     return null;
   }
+
+  @Override
+  public void cleanWriteNotificationEvents(int olderThan) {
+  }
+
+  @Override
+  public List<WriteEventInfo> getAllWriteEventInfo(long txnId, String dbName, String tableName) throws MetaException {
+    return null;
+  }
 }


[09/46] hive git commit: HIVE-19176: Add HoS support to progress bar on Beeline client (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)

Posted by se...@apache.org.
HIVE-19176: Add HoS support to progress bar on Beeline client (Bharathkrishna Guruvayoor Murali, reviewed by Sahil Takiar)


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

Branch: refs/heads/master-txnstats
Commit: e7d1781ec4662e088dcd6ffbe3f866738792ad9b
Parents: e19b861
Author: Bharathkrishna Guruvayoor Murali <bh...@cloudera.com>
Authored: Mon Jul 2 11:42:59 2018 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Mon Jul 2 11:42:59 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   2 +-
 .../org/apache/hive/jdbc/HiveStatement.java     |   4 +-
 .../exec/spark/status/LocalSparkJobMonitor.java |   4 +-
 .../spark/status/RemoteSparkJobMonitor.java     |   4 +-
 .../ql/exec/spark/status/RenderStrategy.java    | 246 +++++++++++++++++++
 .../ql/exec/spark/status/SparkJobMonitor.java   | 157 +-----------
 .../hive/ql/exec/spark/TestSparkTask.java       |   1 +
 .../exec/spark/status/TestSparkJobMonitor.java  |  29 ++-
 .../org/apache/hive/service/ServiceUtils.java   |   5 +-
 .../cli/SparkProgressMonitorStatusMapper.java   |  52 ++++
 .../service/cli/thrift/ThriftCLIService.java    |   5 +-
 11 files changed, 349 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/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 a3dd53e..7ef22d6 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3780,7 +3780,7 @@ public class HiveConf extends Configuration {
         "hive.server2.in.place.progress",
         true,
         "Allows hive server 2 to send progress bar update information. This is currently available"
-            + " only if the execution engine is tez."),
+            + " only if the execution engine is tez or Spark."),
     TEZ_DAG_STATUS_CHECK_INTERVAL("hive.tez.dag.status.check.interval", "500ms",
       new TimeValidator(TimeUnit.MILLISECONDS), "Interval between subsequent DAG status invocation."),
     SPARK_EXEC_INPLACE_PROGRESS("hive.spark.exec.inplace.progress", true,

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
----------------------------------------------------------------------
diff --git a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
index ad8d1a7..0b38f9c 100644
--- a/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
+++ b/jdbc/src/java/org/apache/hive/jdbc/HiveStatement.java
@@ -376,7 +376,9 @@ public class HiveStatement implements java.sql.Statement {
          * essentially return after the HIVE_SERVER2_LONG_POLLING_TIMEOUT (a server config) expires
          */
         statusResp = client.GetOperationStatus(statusReq);
-        inPlaceUpdateStream.update(statusResp.getProgressUpdateResponse());
+        if(!isOperationComplete) {
+          inPlaceUpdateStream.update(statusResp.getProgressUpdateResponse());
+        }
         Utils.verifySuccessWithInfo(statusResp.getStatus());
         if (statusResp.isSetOperationState()) {
           switch (statusResp.getOperationState()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
index 2a6c33b..aeef3c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java
@@ -89,11 +89,11 @@ public class LocalSparkJobMonitor extends SparkJobMonitor {
                 + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]");
             }
 
-            printStatus(progressMap, lastProgressMap);
+            updateFunction.printStatus(progressMap, lastProgressMap);
             lastProgressMap = progressMap;
             break;
           case SUCCEEDED:
-            printStatus(progressMap, lastProgressMap);
+            updateFunction.printStatus(progressMap, lastProgressMap);
             lastProgressMap = progressMap;
             double duration = (System.currentTimeMillis() - startTime) / 1000.0;
             console.printInfo("Status: Finished successfully in "

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
index 560fb58..87b69cb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
@@ -131,13 +131,13 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor {
               }
             }
 
-            printStatus(progressMap, lastProgressMap);
+            updateFunction.printStatus(progressMap, lastProgressMap);
             lastProgressMap = progressMap;
           }
           break;
         case SUCCEEDED:
           Map<SparkStage, SparkStageProgress> progressMap = sparkJobStatus.getSparkStageProgress();
-          printStatus(progressMap, lastProgressMap);
+          updateFunction.printStatus(progressMap, lastProgressMap);
           lastProgressMap = progressMap;
           double duration = (System.currentTimeMillis() - startTime) / 1000.0;
           console.printInfo("Spark job[" + sparkJobStatus.getJobId() + "] finished successfully in "

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RenderStrategy.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RenderStrategy.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RenderStrategy.java
new file mode 100644
index 0000000..67a3a9c
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RenderStrategy.java
@@ -0,0 +1,246 @@
+/*
+ * 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.spark.status;
+
+import org.apache.hadoop.hive.common.log.InPlaceUpdate;
+import org.apache.hadoop.hive.common.log.ProgressMonitor;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.log.PerfLogger;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to render progress bar for Hive on Spark job status.
+ * Based on the configuration, appropriate render strategy is selected
+ * to show the progress bar on beeline or Hive CLI, as well as for logging
+ * the report String.
+ */
+class RenderStrategy {
+
+  interface UpdateFunction {
+    void printStatus(Map<SparkStage, SparkStageProgress> progressMap,
+        Map<SparkStage, SparkStageProgress> lastProgressMap);
+  }
+
+  private abstract static class BaseUpdateFunction implements UpdateFunction {
+    protected final SparkJobMonitor monitor;
+    private final PerfLogger perfLogger;
+    private long lastPrintTime;
+    private static final int PRINT_INTERVAL = 3000;
+    private final Set<String> completed = new HashSet<String>();
+    private String lastReport = null;
+
+    BaseUpdateFunction(SparkJobMonitor monitor) {
+      this.monitor = monitor;
+      this.perfLogger = SessionState.getPerfLogger();
+    }
+
+    private String getReport(Map<SparkStage, SparkStageProgress> progressMap) {
+      StringBuilder reportBuffer = new StringBuilder();
+      SimpleDateFormat dt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+      String currentDate = dt.format(new Date());
+      reportBuffer.append(currentDate + "\t");
+
+      // Num of total and completed tasks
+      int sumTotal = 0;
+      int sumComplete = 0;
+
+      SortedSet<SparkStage> keys = new TreeSet<SparkStage>(progressMap.keySet());
+      for (SparkStage stage : keys) {
+        SparkStageProgress progress = progressMap.get(stage);
+        final int complete = progress.getSucceededTaskCount();
+        final int total = progress.getTotalTaskCount();
+        final int running = progress.getRunningTaskCount();
+        final int failed = progress.getFailedTaskCount();
+        sumTotal += total;
+        sumComplete += complete;
+        String s = stage.toString();
+        String stageName = "Stage-" + s;
+        if (total <= 0) {
+          reportBuffer.append(String.format("%s: -/-\t", stageName));
+        } else {
+          if (complete == total && !completed.contains(s)) {
+            completed.add(s);
+
+            if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) {
+              perfLogger.PerfLogBegin(SparkJobMonitor.CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s);
+            }
+            perfLogger.PerfLogEnd(SparkJobMonitor.CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s);
+          }
+          if (complete < total && (complete > 0 || running > 0 || failed > 0)) {
+            /* stage is started, but not complete */
+            if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) {
+              perfLogger.PerfLogBegin(SparkJobMonitor.CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s);
+            }
+            if (failed > 0) {
+              reportBuffer.append(
+                  String.format(
+                      "%s: %d(+%d,-%d)/%d\t", stageName, complete, running, failed, total));
+            } else {
+              reportBuffer.append(
+                  String.format("%s: %d(+%d)/%d\t", stageName, complete, running, total));
+            }
+          } else {
+            /* stage is waiting for input/slots or complete */
+            if (failed > 0) {
+              /* tasks finished but some failed */
+              reportBuffer.append(
+                  String.format(
+                      "%s: %d(-%d)/%d Finished with failed tasks\t",
+                      stageName, complete, failed, total));
+            } else {
+              if (complete == total) {
+                reportBuffer.append(
+                    String.format("%s: %d/%d Finished\t", stageName, complete, total));
+              } else {
+                reportBuffer.append(String.format("%s: %d/%d\t", stageName, complete, total));
+              }
+            }
+          }
+        }
+      }
+
+      if (SessionState.get() != null) {
+        final float progress = (sumTotal == 0) ? 1.0f : (float) sumComplete / (float) sumTotal;
+        SessionState.get().updateProgressedPercentage(progress);
+      }
+      return reportBuffer.toString();
+    }
+
+    private boolean isSameAsPreviousProgress(
+        Map<SparkStage, SparkStageProgress> progressMap,
+        Map<SparkStage, SparkStageProgress> lastProgressMap) {
+
+      if (lastProgressMap == null) {
+        return false;
+      }
+
+      if (progressMap.isEmpty()) {
+        return lastProgressMap.isEmpty();
+      } else {
+        if (lastProgressMap.isEmpty()) {
+          return false;
+        } else {
+          if (progressMap.size() != lastProgressMap.size()) {
+            return false;
+          }
+          for (Map.Entry<SparkStage, SparkStageProgress> entry : progressMap.entrySet()) {
+            if (!lastProgressMap.containsKey(entry.getKey())
+                || !progressMap.get(entry.getKey()).equals(lastProgressMap.get(entry.getKey()))) {
+              return false;
+            }
+          }
+        }
+      }
+      return true;
+    }
+
+
+    private boolean showReport(String report) {
+      return !report.equals(lastReport) || System.currentTimeMillis() >= lastPrintTime + PRINT_INTERVAL;
+    }
+
+    @Override
+    public void printStatus(Map<SparkStage, SparkStageProgress> progressMap,
+        Map<SparkStage, SparkStageProgress> lastProgressMap) {
+      // do not print duplicate status while still in middle of print interval.
+      boolean isDuplicateState = isSameAsPreviousProgress(progressMap, lastProgressMap);
+      boolean withinInterval = System.currentTimeMillis() <= lastPrintTime + PRINT_INTERVAL;
+      if (isDuplicateState && withinInterval) {
+        return;
+      }
+
+      String report = getReport(progressMap);
+      renderProgress(monitor.getProgressMonitor(progressMap));
+      if (showReport(report)) {
+        renderReport(report);
+        lastReport = report;
+        lastPrintTime = System.currentTimeMillis();
+      }
+    }
+
+    abstract void renderProgress(ProgressMonitor monitor);
+
+    abstract void renderReport(String report);
+  }
+
+  /**
+   * This is used to show progress bar on Beeline while using HiveServer2.
+   */
+  static class LogToFileFunction extends BaseUpdateFunction {
+    private static final Logger LOGGER = LoggerFactory.getLogger(LogToFileFunction.class);
+    private boolean hiveServer2InPlaceProgressEnabled =
+        SessionState.get().getConf().getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_INPLACE_PROGRESS);
+
+    LogToFileFunction(SparkJobMonitor monitor) {
+      super(monitor);
+    }
+
+    @Override
+    void renderProgress(ProgressMonitor monitor) {
+      SessionState.get().updateProgressMonitor(monitor);
+    }
+
+    @Override
+    void renderReport(String report) {
+      if (hiveServer2InPlaceProgressEnabled) {
+        LOGGER.info(report);
+      } else {
+        monitor.console.printInfo(report);
+      }
+    }
+  }
+
+  /**
+   * This is used to show progress bar on Hive CLI.
+   */
+  static class InPlaceUpdateFunction extends BaseUpdateFunction {
+    /**
+     * Have to use the same instance to render else the number lines printed earlier is lost and the
+     * screen will print the table again and again.
+     */
+    private final InPlaceUpdate inPlaceUpdate;
+
+    InPlaceUpdateFunction(SparkJobMonitor monitor) {
+      super(monitor);
+      inPlaceUpdate = new InPlaceUpdate(SessionState.LogHelper.getInfoStream());
+    }
+
+    @Override
+    void renderProgress(ProgressMonitor monitor) {
+      inPlaceUpdate.render(monitor);
+    }
+
+    @Override
+    void renderReport(String report) {
+      monitor.console.logInfo(report);
+    }
+  }
+}
+
+

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
index 3531ac2..5fd0c02 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.spark.status;
 
+import org.apache.hadoop.hive.common.log.ProgressMonitor;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.common.log.InPlaceUpdate;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
@@ -25,13 +26,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 
 abstract class SparkJobMonitor {
@@ -42,60 +37,27 @@ abstract class SparkJobMonitor {
   protected final PerfLogger perfLogger = SessionState.getPerfLogger();
   protected final int checkInterval = 1000;
   protected final long monitorTimeoutInterval;
-  private final InPlaceUpdate inPlaceUpdateFn;
-
-  private final Set<String> completed = new HashSet<String>();
-  private final int printInterval = 3000;
-  private long lastPrintTime;
-
+  final RenderStrategy.UpdateFunction updateFunction;
   protected long startTime;
 
   protected enum StageState {
-    PENDING,
-    RUNNING,
-    FINISHED
+    PENDING, RUNNING, FINISHED
   }
 
   protected final boolean inPlaceUpdate;
 
   protected SparkJobMonitor(HiveConf hiveConf) {
-    monitorTimeoutInterval = hiveConf.getTimeVar(
-        HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT, TimeUnit.SECONDS);
+    monitorTimeoutInterval = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT, TimeUnit.SECONDS);
     inPlaceUpdate = InPlaceUpdate.canRenderInPlace(hiveConf) && !SessionState.getConsole().getIsSilent();
     console = new SessionState.LogHelper(LOG);
-    inPlaceUpdateFn = new InPlaceUpdate(SessionState.LogHelper.getInfoStream());
+    updateFunction = updateFunction();
   }
 
   public abstract int startMonitor();
 
-  private void printStatusInPlace(Map<SparkStage, SparkStageProgress> progressMap) {
-    inPlaceUpdateFn.render(getProgressMonitor(progressMap));
-  }
-
-  protected void printStatus(Map<SparkStage, SparkStageProgress> progressMap,
-      Map<SparkStage, SparkStageProgress> lastProgressMap) {
-
-    // do not print duplicate status while still in middle of print interval.
-    boolean isDuplicateState = isSameAsPreviousProgress(progressMap, lastProgressMap);
-    boolean withinInterval = System.currentTimeMillis() <= lastPrintTime + printInterval;
-    if (isDuplicateState && withinInterval) {
-      return;
-    }
-
-    String report = getReport(progressMap);
-    if (inPlaceUpdate) {
-      printStatusInPlace(progressMap);
-      console.logInfo(report);
-    } else {
-      console.printInfo(report);
-    }
-
-    lastPrintTime = System.currentTimeMillis();
-  }
-
   protected int getTotalTaskCount(Map<SparkStage, SparkStageProgress> progressMap) {
     int totalTasks = 0;
-    for (SparkStageProgress progress: progressMap.values() ) {
+    for (SparkStageProgress progress : progressMap.values()) {
       totalTasks += progress.getTotalTaskCount();
     }
 
@@ -104,7 +66,7 @@ abstract class SparkJobMonitor {
 
   protected int getStageMaxTaskCount(Map<SparkStage, SparkStageProgress> progressMap) {
     int stageMaxTasks = 0;
-    for (SparkStageProgress progress: progressMap.values() ) {
+    for (SparkStageProgress progress : progressMap.values()) {
       int tasks = progress.getTotalTaskCount();
       if (tasks > stageMaxTasks) {
         stageMaxTasks = tasks;
@@ -114,107 +76,12 @@ abstract class SparkJobMonitor {
     return stageMaxTasks;
   }
 
-  private String getReport(Map<SparkStage, SparkStageProgress> progressMap) {
-    StringBuilder reportBuffer = new StringBuilder();
-    SimpleDateFormat dt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
-    String currentDate = dt.format(new Date());
-    reportBuffer.append(currentDate + "\t");
-
-    // Num of total and completed tasks
-    int sumTotal = 0;
-    int sumComplete = 0;
-
-    SortedSet<SparkStage> keys = new TreeSet<SparkStage>(progressMap.keySet());
-    for (SparkStage stage : keys) {
-      SparkStageProgress progress = progressMap.get(stage);
-      final int complete = progress.getSucceededTaskCount();
-      final int total = progress.getTotalTaskCount();
-      final int running = progress.getRunningTaskCount();
-      final int failed = progress.getFailedTaskCount();
-      sumTotal += total;
-      sumComplete += complete;
-      String s = stage.toString();
-      String stageName = "Stage-" + s;
-      if (total <= 0) {
-        reportBuffer.append(String.format("%s: -/-\t", stageName));
-      } else {
-        if (complete == total && !completed.contains(s)) {
-          completed.add(s);
-
-          if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) {
-            perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s);
-          }
-          perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s);
-        }
-        if (complete < total && (complete > 0 || running > 0 || failed > 0)) {
-          /* stage is started, but not complete */
-          if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) {
-            perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s);
-          }
-          if (failed > 0) {
-            reportBuffer.append(
-                String.format(
-                    "%s: %d(+%d,-%d)/%d\t", stageName, complete, running, failed, total));
-          } else {
-            reportBuffer.append(
-                String.format("%s: %d(+%d)/%d\t", stageName, complete, running, total));
-          }
-        } else {
-          /* stage is waiting for input/slots or complete */
-          if (failed > 0) {
-            /* tasks finished but some failed */
-            reportBuffer.append(
-                String.format(
-                    "%s: %d(-%d)/%d Finished with failed tasks\t",
-                    stageName, complete, failed, total));
-          } else {
-            if (complete == total) {
-              reportBuffer.append(
-                  String.format("%s: %d/%d Finished\t", stageName, complete, total));
-            } else {
-              reportBuffer.append(String.format("%s: %d/%d\t", stageName, complete, total));
-            }
-          }
-        }
-      }
-    }
-
-    if (SessionState.get() != null) {
-      final float progress = (sumTotal == 0) ? 1.0f : (float) sumComplete / (float) sumTotal;
-      SessionState.get().updateProgressedPercentage(progress);
-    }
-    return reportBuffer.toString();
-  }
-
-  private boolean isSameAsPreviousProgress(
-      Map<SparkStage, SparkStageProgress> progressMap,
-      Map<SparkStage, SparkStageProgress> lastProgressMap) {
-
-    if (lastProgressMap == null) {
-      return false;
-    }
-
-    if (progressMap.isEmpty()) {
-      return lastProgressMap.isEmpty();
-    } else {
-      if (lastProgressMap.isEmpty()) {
-        return false;
-      } else {
-        if (progressMap.size() != lastProgressMap.size()) {
-          return false;
-        }
-        for (SparkStage key : progressMap.keySet()) {
-          if (!lastProgressMap.containsKey(key)
-              || !progressMap.get(key).equals(lastProgressMap.get(key))) {
-            return false;
-          }
-        }
-      }
-    }
-    return true;
+  ProgressMonitor getProgressMonitor(Map<SparkStage, SparkStageProgress> progressMap) {
+    return new SparkProgressMonitor(progressMap, startTime);
   }
 
-  private SparkProgressMonitor getProgressMonitor(Map<SparkStage, SparkStageProgress> progressMap) {
-    return new SparkProgressMonitor(progressMap, startTime);
+  private RenderStrategy.UpdateFunction updateFunction() {
+    return inPlaceUpdate && !SessionState.get().isHiveServerQuery() ? new RenderStrategy.InPlaceUpdateFunction(
+        this) : new RenderStrategy.LogToFileFunction(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
index 368fa9f..2017fc1 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java
@@ -105,6 +105,7 @@ public class TestSparkTask {
     when(jobSts.getRemoteJobState()).thenReturn(State.CANCELLED);
     when(jobSts.isRemoteActive()).thenReturn(true);
     HiveConf hiveConf = new HiveConf();
+    SessionState.start(hiveConf);
     RemoteSparkJobMonitor remoteSparkJobMonitor = new RemoteSparkJobMonitor(hiveConf, jobSts);
     Assert.assertEquals(remoteSparkJobMonitor.startMonitor(), 3);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/status/TestSparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/status/TestSparkJobMonitor.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/status/TestSparkJobMonitor.java
index e66354f..7257b32 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/status/TestSparkJobMonitor.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/status/TestSparkJobMonitor.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.exec.spark.status;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -40,22 +41,23 @@ public class TestSparkJobMonitor {
   private SparkJobMonitor monitor;
   private PrintStream curOut;
   private PrintStream curErr;
+  private RenderStrategy.InPlaceUpdateFunction updateFunction;
 
   @Before
   public void setUp() {
-    testConf = new HiveConf();
     curOut = System.out;
     curErr = System.err;
     System.setOut(new PrintStream(outContent));
     System.setErr(new PrintStream(errContent));
-
+    testConf = new HiveConf();
+    SessionState.start(testConf);
     monitor = new SparkJobMonitor(testConf) {
       @Override
       public int startMonitor() {
         return 0;
       }
     };
-
+    updateFunction = new RenderStrategy.InPlaceUpdateFunction(monitor);
   }
 
   private Map<SparkStage, SparkStageProgress> progressMap() {
@@ -72,12 +74,27 @@ public class TestSparkJobMonitor {
   }
 
   @Test
-  public void testGetReport() {
+  public void testProgress() {
     Map<SparkStage, SparkStageProgress> progressMap = progressMap();
-    monitor.printStatus(progressMap, null);
-    assertTrue(errContent.toString().contains(
+    updateFunction.printStatus(progressMap, null);
+    String testOutput = errContent.toString();
+    assertTrue(testOutput.contains(
         "Stage-1_0: 3(+1)/4\tStage-3_1: 4(+1,-1)/6\tStage-9_0: 5/5 Finished\tStage-10_2: 3(+2)/5\t"
             + "Stage-15_1: 3(+1)/4\tStage-15_2: 4/4 Finished\tStage-20_3: 1(+1,-1)/3\tStage-21_1: 2/2 Finished"));
+    String[] testStrings = new String[]{
+        "STAGES   ATTEMPT        STATUS  TOTAL  COMPLETED  RUNNING  PENDING  FAILED",
+        "Stage-1 ......           0       RUNNING      4          3        1        0       0",
+        "Stage-3 .....            1       RUNNING      6          4        1        1       1",
+        "Stage-9 ........         0      FINISHED      5          5        0        0       0",
+        "Stage-10 ....            2       RUNNING      5          3        2        0       0",
+        "Stage-15 .....           1       RUNNING      4          3        1        0       0",
+        "Stage-15 .......         2      FINISHED      4          4        0        0       0",
+        "Stage-20 ..              3       RUNNING      3          1        1        1       1",
+        "Stage-21 .......         1      FINISHED      2          2        0        0       0",
+        "STAGES: 03/08    [===================>>-------] 75%   ELAPSED TIME:"};
+    for(String testString : testStrings) {
+      assertTrue(testOutput.contains(testString));
+    }
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/service/src/java/org/apache/hive/service/ServiceUtils.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/ServiceUtils.java b/service/src/java/org/apache/hive/service/ServiceUtils.java
index 226e432..49fb5d5 100644
--- a/service/src/java/org/apache/hive/service/ServiceUtils.java
+++ b/service/src/java/org/apache/hive/service/ServiceUtils.java
@@ -69,8 +69,9 @@ public class ServiceUtils {
   }
 
   public static boolean canProvideProgressLog(HiveConf hiveConf) {
-    return "tez".equals(hiveConf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE))
-        && hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_INPLACE_PROGRESS);
+    return ("tez".equals(hiveConf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE)) || "spark"
+        .equals(hiveConf.getVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE))) && hiveConf
+        .getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_INPLACE_PROGRESS);
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/service/src/java/org/apache/hive/service/cli/SparkProgressMonitorStatusMapper.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/SparkProgressMonitorStatusMapper.java b/service/src/java/org/apache/hive/service/cli/SparkProgressMonitorStatusMapper.java
new file mode 100644
index 0000000..c2a222e
--- /dev/null
+++ b/service/src/java/org/apache/hive/service/cli/SparkProgressMonitorStatusMapper.java
@@ -0,0 +1,52 @@
+/*
+ * 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.service.cli;
+
+import org.apache.hive.service.rpc.thrift.TJobExecutionStatus;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Maps status of spark stages to job execution status.
+ */
+public class SparkProgressMonitorStatusMapper implements ProgressMonitorStatusMapper {
+  /**
+   * These states are taken form DAGStatus.State, could not use that here directly as it was
+   * optional dependency and did not want to include it just for the enum.
+   */
+  enum SparkStatus {
+    PENDING, RUNNING, FINISHED
+
+  }
+
+  @Override
+  public TJobExecutionStatus forStatus(String status) {
+    if (StringUtils.isEmpty(status)) {
+      return TJobExecutionStatus.NOT_AVAILABLE;
+    }
+    SparkProgressMonitorStatusMapper.SparkStatus sparkStatus =
+        SparkProgressMonitorStatusMapper.SparkStatus.valueOf(status);
+    switch (sparkStatus) {
+    case PENDING:
+    case RUNNING:
+      return TJobExecutionStatus.IN_PROGRESS;
+    default:
+      return TJobExecutionStatus.COMPLETE;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e7d1781e/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
index 68fe8d8..259ca63 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
@@ -21,6 +21,7 @@ package org.apache.hive.service.cli.thrift;
 import static com.google.common.base.Preconditions.checkArgument;
 
 import org.apache.hive.service.cli.OperationState;
+import org.apache.hive.service.cli.SparkProgressMonitorStatusMapper;
 import org.apache.hive.service.rpc.thrift.TSetClientInfoReq;
 import org.apache.hive.service.rpc.thrift.TSetClientInfoResp;
 
@@ -707,7 +708,9 @@ public abstract class ThriftCLIService extends AbstractService implements TCLISe
       if ("tez".equals(hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) {
         mapper = new TezProgressMonitorStatusMapper();
       }
-
+      if ("spark".equals(hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE))) {
+        mapper = new SparkProgressMonitorStatusMapper();
+      }
       TJobExecutionStatus executionStatus =
           mapper.forStatus(progressUpdate.status);
       resp.setProgressUpdateResponse(new TProgressUpdateResp(


[25/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
index eda462e..4467479 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMFullResourcePlan.java
@@ -755,14 +755,14 @@ import org.slf4j.LoggerFactory;
           case 2: // POOLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
-                struct.pools = new ArrayList<WMPool>(_list848.size);
-                WMPool _elem849;
-                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
+                org.apache.thrift.protocol.TList _list872 = iprot.readListBegin();
+                struct.pools = new ArrayList<WMPool>(_list872.size);
+                WMPool _elem873;
+                for (int _i874 = 0; _i874 < _list872.size; ++_i874)
                 {
-                  _elem849 = new WMPool();
-                  _elem849.read(iprot);
-                  struct.pools.add(_elem849);
+                  _elem873 = new WMPool();
+                  _elem873.read(iprot);
+                  struct.pools.add(_elem873);
                 }
                 iprot.readListEnd();
               }
@@ -774,14 +774,14 @@ import org.slf4j.LoggerFactory;
           case 3: // MAPPINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list851 = iprot.readListBegin();
-                struct.mappings = new ArrayList<WMMapping>(_list851.size);
-                WMMapping _elem852;
-                for (int _i853 = 0; _i853 < _list851.size; ++_i853)
+                org.apache.thrift.protocol.TList _list875 = iprot.readListBegin();
+                struct.mappings = new ArrayList<WMMapping>(_list875.size);
+                WMMapping _elem876;
+                for (int _i877 = 0; _i877 < _list875.size; ++_i877)
                 {
-                  _elem852 = new WMMapping();
-                  _elem852.read(iprot);
-                  struct.mappings.add(_elem852);
+                  _elem876 = new WMMapping();
+                  _elem876.read(iprot);
+                  struct.mappings.add(_elem876);
                 }
                 iprot.readListEnd();
               }
@@ -793,14 +793,14 @@ import org.slf4j.LoggerFactory;
           case 4: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list854 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list854.size);
-                WMTrigger _elem855;
-                for (int _i856 = 0; _i856 < _list854.size; ++_i856)
+                org.apache.thrift.protocol.TList _list878 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list878.size);
+                WMTrigger _elem879;
+                for (int _i880 = 0; _i880 < _list878.size; ++_i880)
                 {
-                  _elem855 = new WMTrigger();
-                  _elem855.read(iprot);
-                  struct.triggers.add(_elem855);
+                  _elem879 = new WMTrigger();
+                  _elem879.read(iprot);
+                  struct.triggers.add(_elem879);
                 }
                 iprot.readListEnd();
               }
@@ -812,14 +812,14 @@ import org.slf4j.LoggerFactory;
           case 5: // POOL_TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list857 = iprot.readListBegin();
-                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list857.size);
-                WMPoolTrigger _elem858;
-                for (int _i859 = 0; _i859 < _list857.size; ++_i859)
+                org.apache.thrift.protocol.TList _list881 = iprot.readListBegin();
+                struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list881.size);
+                WMPoolTrigger _elem882;
+                for (int _i883 = 0; _i883 < _list881.size; ++_i883)
                 {
-                  _elem858 = new WMPoolTrigger();
-                  _elem858.read(iprot);
-                  struct.poolTriggers.add(_elem858);
+                  _elem882 = new WMPoolTrigger();
+                  _elem882.read(iprot);
+                  struct.poolTriggers.add(_elem882);
                 }
                 iprot.readListEnd();
               }
@@ -850,9 +850,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(POOLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.pools.size()));
-          for (WMPool _iter860 : struct.pools)
+          for (WMPool _iter884 : struct.pools)
           {
-            _iter860.write(oprot);
+            _iter884.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -863,9 +863,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(MAPPINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.mappings.size()));
-            for (WMMapping _iter861 : struct.mappings)
+            for (WMMapping _iter885 : struct.mappings)
             {
-              _iter861.write(oprot);
+              _iter885.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -877,9 +877,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter862 : struct.triggers)
+            for (WMTrigger _iter886 : struct.triggers)
             {
-              _iter862.write(oprot);
+              _iter886.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -891,9 +891,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(POOL_TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.poolTriggers.size()));
-            for (WMPoolTrigger _iter863 : struct.poolTriggers)
+            for (WMPoolTrigger _iter887 : struct.poolTriggers)
             {
-              _iter863.write(oprot);
+              _iter887.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -920,9 +920,9 @@ import org.slf4j.LoggerFactory;
       struct.plan.write(oprot);
       {
         oprot.writeI32(struct.pools.size());
-        for (WMPool _iter864 : struct.pools)
+        for (WMPool _iter888 : struct.pools)
         {
-          _iter864.write(oprot);
+          _iter888.write(oprot);
         }
       }
       BitSet optionals = new BitSet();
@@ -939,27 +939,27 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetMappings()) {
         {
           oprot.writeI32(struct.mappings.size());
-          for (WMMapping _iter865 : struct.mappings)
+          for (WMMapping _iter889 : struct.mappings)
           {
-            _iter865.write(oprot);
+            _iter889.write(oprot);
           }
         }
       }
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter866 : struct.triggers)
+          for (WMTrigger _iter890 : struct.triggers)
           {
-            _iter866.write(oprot);
+            _iter890.write(oprot);
           }
         }
       }
       if (struct.isSetPoolTriggers()) {
         {
           oprot.writeI32(struct.poolTriggers.size());
-          for (WMPoolTrigger _iter867 : struct.poolTriggers)
+          for (WMPoolTrigger _iter891 : struct.poolTriggers)
           {
-            _iter867.write(oprot);
+            _iter891.write(oprot);
           }
         }
       }
@@ -972,56 +972,56 @@ import org.slf4j.LoggerFactory;
       struct.plan.read(iprot);
       struct.setPlanIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list868 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.pools = new ArrayList<WMPool>(_list868.size);
-        WMPool _elem869;
-        for (int _i870 = 0; _i870 < _list868.size; ++_i870)
+        org.apache.thrift.protocol.TList _list892 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.pools = new ArrayList<WMPool>(_list892.size);
+        WMPool _elem893;
+        for (int _i894 = 0; _i894 < _list892.size; ++_i894)
         {
-          _elem869 = new WMPool();
-          _elem869.read(iprot);
-          struct.pools.add(_elem869);
+          _elem893 = new WMPool();
+          _elem893.read(iprot);
+          struct.pools.add(_elem893);
         }
       }
       struct.setPoolsIsSet(true);
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list871 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.mappings = new ArrayList<WMMapping>(_list871.size);
-          WMMapping _elem872;
-          for (int _i873 = 0; _i873 < _list871.size; ++_i873)
+          org.apache.thrift.protocol.TList _list895 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.mappings = new ArrayList<WMMapping>(_list895.size);
+          WMMapping _elem896;
+          for (int _i897 = 0; _i897 < _list895.size; ++_i897)
           {
-            _elem872 = new WMMapping();
-            _elem872.read(iprot);
-            struct.mappings.add(_elem872);
+            _elem896 = new WMMapping();
+            _elem896.read(iprot);
+            struct.mappings.add(_elem896);
           }
         }
         struct.setMappingsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list874 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list874.size);
-          WMTrigger _elem875;
-          for (int _i876 = 0; _i876 < _list874.size; ++_i876)
+          org.apache.thrift.protocol.TList _list898 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list898.size);
+          WMTrigger _elem899;
+          for (int _i900 = 0; _i900 < _list898.size; ++_i900)
           {
-            _elem875 = new WMTrigger();
-            _elem875.read(iprot);
-            struct.triggers.add(_elem875);
+            _elem899 = new WMTrigger();
+            _elem899.read(iprot);
+            struct.triggers.add(_elem899);
           }
         }
         struct.setTriggersIsSet(true);
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list877 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list877.size);
-          WMPoolTrigger _elem878;
-          for (int _i879 = 0; _i879 < _list877.size; ++_i879)
+          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.poolTriggers = new ArrayList<WMPoolTrigger>(_list901.size);
+          WMPoolTrigger _elem902;
+          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
           {
-            _elem878 = new WMPoolTrigger();
-            _elem878.read(iprot);
-            struct.poolTriggers.add(_elem878);
+            _elem902 = new WMPoolTrigger();
+            _elem902.read(iprot);
+            struct.poolTriggers.add(_elem902);
           }
         }
         struct.setPoolTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
index 9bbc97b..c6cb845 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetAllResourcePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // RESOURCE_PLANS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list880 = iprot.readListBegin();
-                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list880.size);
-                WMResourcePlan _elem881;
-                for (int _i882 = 0; _i882 < _list880.size; ++_i882)
+                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
+                struct.resourcePlans = new ArrayList<WMResourcePlan>(_list904.size);
+                WMResourcePlan _elem905;
+                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
                 {
-                  _elem881 = new WMResourcePlan();
-                  _elem881.read(iprot);
-                  struct.resourcePlans.add(_elem881);
+                  _elem905 = new WMResourcePlan();
+                  _elem905.read(iprot);
+                  struct.resourcePlans.add(_elem905);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(RESOURCE_PLANS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.resourcePlans.size()));
-            for (WMResourcePlan _iter883 : struct.resourcePlans)
+            for (WMResourcePlan _iter907 : struct.resourcePlans)
             {
-              _iter883.write(oprot);
+              _iter907.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetResourcePlans()) {
         {
           oprot.writeI32(struct.resourcePlans.size());
-          for (WMResourcePlan _iter884 : struct.resourcePlans)
+          for (WMResourcePlan _iter908 : struct.resourcePlans)
           {
-            _iter884.write(oprot);
+            _iter908.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list885 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list885.size);
-          WMResourcePlan _elem886;
-          for (int _i887 = 0; _i887 < _list885.size; ++_i887)
+          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.resourcePlans = new ArrayList<WMResourcePlan>(_list909.size);
+          WMResourcePlan _elem910;
+          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
           {
-            _elem886 = new WMResourcePlan();
-            _elem886.read(iprot);
-            struct.resourcePlans.add(_elem886);
+            _elem910 = new WMResourcePlan();
+            _elem910.read(iprot);
+            struct.resourcePlans.add(_elem910);
           }
         }
         struct.setResourcePlansIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
index 6918953..9eed335 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMGetTriggersForResourePlanResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TRIGGERS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list904 = iprot.readListBegin();
-                struct.triggers = new ArrayList<WMTrigger>(_list904.size);
-                WMTrigger _elem905;
-                for (int _i906 = 0; _i906 < _list904.size; ++_i906)
+                org.apache.thrift.protocol.TList _list928 = iprot.readListBegin();
+                struct.triggers = new ArrayList<WMTrigger>(_list928.size);
+                WMTrigger _elem929;
+                for (int _i930 = 0; _i930 < _list928.size; ++_i930)
                 {
-                  _elem905 = new WMTrigger();
-                  _elem905.read(iprot);
-                  struct.triggers.add(_elem905);
+                  _elem929 = new WMTrigger();
+                  _elem929.read(iprot);
+                  struct.triggers.add(_elem929);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TRIGGERS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.triggers.size()));
-            for (WMTrigger _iter907 : struct.triggers)
+            for (WMTrigger _iter931 : struct.triggers)
             {
-              _iter907.write(oprot);
+              _iter931.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTriggers()) {
         {
           oprot.writeI32(struct.triggers.size());
-          for (WMTrigger _iter908 : struct.triggers)
+          for (WMTrigger _iter932 : struct.triggers)
           {
-            _iter908.write(oprot);
+            _iter932.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list909 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.triggers = new ArrayList<WMTrigger>(_list909.size);
-          WMTrigger _elem910;
-          for (int _i911 = 0; _i911 < _list909.size; ++_i911)
+          org.apache.thrift.protocol.TList _list933 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.triggers = new ArrayList<WMTrigger>(_list933.size);
+          WMTrigger _elem934;
+          for (int _i935 = 0; _i935 < _list933.size; ++_i935)
           {
-            _elem910 = new WMTrigger();
-            _elem910.read(iprot);
-            struct.triggers.add(_elem910);
+            _elem934 = new WMTrigger();
+            _elem934.read(iprot);
+            struct.triggers.add(_elem934);
           }
         }
         struct.setTriggersIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
index 66a478d..ee9251c 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WMValidateResourcePlanResponse.java
@@ -441,13 +441,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ERRORS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list888 = iprot.readListBegin();
-                struct.errors = new ArrayList<String>(_list888.size);
-                String _elem889;
-                for (int _i890 = 0; _i890 < _list888.size; ++_i890)
+                org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
+                struct.errors = new ArrayList<String>(_list912.size);
+                String _elem913;
+                for (int _i914 = 0; _i914 < _list912.size; ++_i914)
                 {
-                  _elem889 = iprot.readString();
-                  struct.errors.add(_elem889);
+                  _elem913 = iprot.readString();
+                  struct.errors.add(_elem913);
                 }
                 iprot.readListEnd();
               }
@@ -459,13 +459,13 @@ import org.slf4j.LoggerFactory;
           case 2: // WARNINGS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list891 = iprot.readListBegin();
-                struct.warnings = new ArrayList<String>(_list891.size);
-                String _elem892;
-                for (int _i893 = 0; _i893 < _list891.size; ++_i893)
+                org.apache.thrift.protocol.TList _list915 = iprot.readListBegin();
+                struct.warnings = new ArrayList<String>(_list915.size);
+                String _elem916;
+                for (int _i917 = 0; _i917 < _list915.size; ++_i917)
                 {
-                  _elem892 = iprot.readString();
-                  struct.warnings.add(_elem892);
+                  _elem916 = iprot.readString();
+                  struct.warnings.add(_elem916);
                 }
                 iprot.readListEnd();
               }
@@ -492,9 +492,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(ERRORS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.errors.size()));
-            for (String _iter894 : struct.errors)
+            for (String _iter918 : struct.errors)
             {
-              oprot.writeString(_iter894);
+              oprot.writeString(_iter918);
             }
             oprot.writeListEnd();
           }
@@ -506,9 +506,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(WARNINGS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.warnings.size()));
-            for (String _iter895 : struct.warnings)
+            for (String _iter919 : struct.warnings)
             {
-              oprot.writeString(_iter895);
+              oprot.writeString(_iter919);
             }
             oprot.writeListEnd();
           }
@@ -543,18 +543,18 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetErrors()) {
         {
           oprot.writeI32(struct.errors.size());
-          for (String _iter896 : struct.errors)
+          for (String _iter920 : struct.errors)
           {
-            oprot.writeString(_iter896);
+            oprot.writeString(_iter920);
           }
         }
       }
       if (struct.isSetWarnings()) {
         {
           oprot.writeI32(struct.warnings.size());
-          for (String _iter897 : struct.warnings)
+          for (String _iter921 : struct.warnings)
           {
-            oprot.writeString(_iter897);
+            oprot.writeString(_iter921);
           }
         }
       }
@@ -566,26 +566,26 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list898 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.errors = new ArrayList<String>(_list898.size);
-          String _elem899;
-          for (int _i900 = 0; _i900 < _list898.size; ++_i900)
+          org.apache.thrift.protocol.TList _list922 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.errors = new ArrayList<String>(_list922.size);
+          String _elem923;
+          for (int _i924 = 0; _i924 < _list922.size; ++_i924)
           {
-            _elem899 = iprot.readString();
-            struct.errors.add(_elem899);
+            _elem923 = iprot.readString();
+            struct.errors.add(_elem923);
           }
         }
         struct.setErrorsIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list901 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.warnings = new ArrayList<String>(_list901.size);
-          String _elem902;
-          for (int _i903 = 0; _i903 < _list901.size; ++_i903)
+          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.warnings = new ArrayList<String>(_list925.size);
+          String _elem926;
+          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
           {
-            _elem902 = iprot.readString();
-            struct.warnings.add(_elem902);
+            _elem926 = iprot.readString();
+            struct.warnings.add(_elem926);
           }
         }
         struct.setWarningsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java
new file mode 100644
index 0000000..22f2609
--- /dev/null
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/WriteEventInfo.java
@@ -0,0 +1,1012 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.3)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.hadoop.hive.metastore.api;
+
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import org.apache.thrift.async.AsyncMethodCallback;
+import org.apache.thrift.server.AbstractNonblockingServer.*;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import javax.annotation.Generated;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"})
+@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)")
+@org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public class WriteEventInfo implements org.apache.thrift.TBase<WriteEventInfo, WriteEventInfo._Fields>, java.io.Serializable, Cloneable, Comparable<WriteEventInfo> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("WriteEventInfo");
+
+  private static final org.apache.thrift.protocol.TField WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("writeId", org.apache.thrift.protocol.TType.I64, (short)1);
+  private static final org.apache.thrift.protocol.TField DATABASE_FIELD_DESC = new org.apache.thrift.protocol.TField("database", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField FILES_FIELD_DESC = new org.apache.thrift.protocol.TField("files", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField PARTITION_FIELD_DESC = new org.apache.thrift.protocol.TField("partition", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField TABLE_OBJ_FIELD_DESC = new org.apache.thrift.protocol.TField("tableObj", org.apache.thrift.protocol.TType.STRING, (short)6);
+  private static final org.apache.thrift.protocol.TField PARTITION_OBJ_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionObj", org.apache.thrift.protocol.TType.STRING, (short)7);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new WriteEventInfoStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new WriteEventInfoTupleSchemeFactory());
+  }
+
+  private long writeId; // required
+  private String database; // required
+  private String table; // required
+  private String files; // required
+  private String partition; // optional
+  private String tableObj; // optional
+  private String partitionObj; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    WRITE_ID((short)1, "writeId"),
+    DATABASE((short)2, "database"),
+    TABLE((short)3, "table"),
+    FILES((short)4, "files"),
+    PARTITION((short)5, "partition"),
+    TABLE_OBJ((short)6, "tableObj"),
+    PARTITION_OBJ((short)7, "partitionObj");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // WRITE_ID
+          return WRITE_ID;
+        case 2: // DATABASE
+          return DATABASE;
+        case 3: // TABLE
+          return TABLE;
+        case 4: // FILES
+          return FILES;
+        case 5: // PARTITION
+          return PARTITION;
+        case 6: // TABLE_OBJ
+          return TABLE_OBJ;
+        case 7: // PARTITION_OBJ
+          return PARTITION_OBJ;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private static final int __WRITEID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.PARTITION,_Fields.TABLE_OBJ,_Fields.PARTITION_OBJ};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("writeId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    tmpMap.put(_Fields.DATABASE, new org.apache.thrift.meta_data.FieldMetaData("database", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.FILES, new org.apache.thrift.meta_data.FieldMetaData("files", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PARTITION, new org.apache.thrift.meta_data.FieldMetaData("partition", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE_OBJ, new org.apache.thrift.meta_data.FieldMetaData("tableObj", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.PARTITION_OBJ, new org.apache.thrift.meta_data.FieldMetaData("partitionObj", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(WriteEventInfo.class, metaDataMap);
+  }
+
+  public WriteEventInfo() {
+  }
+
+  public WriteEventInfo(
+    long writeId,
+    String database,
+    String table,
+    String files)
+  {
+    this();
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+    this.database = database;
+    this.table = table;
+    this.files = files;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public WriteEventInfo(WriteEventInfo other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.writeId = other.writeId;
+    if (other.isSetDatabase()) {
+      this.database = other.database;
+    }
+    if (other.isSetTable()) {
+      this.table = other.table;
+    }
+    if (other.isSetFiles()) {
+      this.files = other.files;
+    }
+    if (other.isSetPartition()) {
+      this.partition = other.partition;
+    }
+    if (other.isSetTableObj()) {
+      this.tableObj = other.tableObj;
+    }
+    if (other.isSetPartitionObj()) {
+      this.partitionObj = other.partitionObj;
+    }
+  }
+
+  public WriteEventInfo deepCopy() {
+    return new WriteEventInfo(this);
+  }
+
+  @Override
+  public void clear() {
+    setWriteIdIsSet(false);
+    this.writeId = 0;
+    this.database = null;
+    this.table = null;
+    this.files = null;
+    this.partition = null;
+    this.tableObj = null;
+    this.partitionObj = null;
+  }
+
+  public long getWriteId() {
+    return this.writeId;
+  }
+
+  public void setWriteId(long writeId) {
+    this.writeId = writeId;
+    setWriteIdIsSet(true);
+  }
+
+  public void unsetWriteId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  /** Returns true if field writeId is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteId() {
+    return EncodingUtils.testBit(__isset_bitfield, __WRITEID_ISSET_ID);
+  }
+
+  public void setWriteIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __WRITEID_ISSET_ID, value);
+  }
+
+  public String getDatabase() {
+    return this.database;
+  }
+
+  public void setDatabase(String database) {
+    this.database = database;
+  }
+
+  public void unsetDatabase() {
+    this.database = null;
+  }
+
+  /** Returns true if field database is set (has been assigned a value) and false otherwise */
+  public boolean isSetDatabase() {
+    return this.database != null;
+  }
+
+  public void setDatabaseIsSet(boolean value) {
+    if (!value) {
+      this.database = null;
+    }
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public void unsetTable() {
+    this.table = null;
+  }
+
+  /** Returns true if field table is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable() {
+    return this.table != null;
+  }
+
+  public void setTableIsSet(boolean value) {
+    if (!value) {
+      this.table = null;
+    }
+  }
+
+  public String getFiles() {
+    return this.files;
+  }
+
+  public void setFiles(String files) {
+    this.files = files;
+  }
+
+  public void unsetFiles() {
+    this.files = null;
+  }
+
+  /** Returns true if field files is set (has been assigned a value) and false otherwise */
+  public boolean isSetFiles() {
+    return this.files != null;
+  }
+
+  public void setFilesIsSet(boolean value) {
+    if (!value) {
+      this.files = null;
+    }
+  }
+
+  public String getPartition() {
+    return this.partition;
+  }
+
+  public void setPartition(String partition) {
+    this.partition = partition;
+  }
+
+  public void unsetPartition() {
+    this.partition = null;
+  }
+
+  /** Returns true if field partition is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartition() {
+    return this.partition != null;
+  }
+
+  public void setPartitionIsSet(boolean value) {
+    if (!value) {
+      this.partition = null;
+    }
+  }
+
+  public String getTableObj() {
+    return this.tableObj;
+  }
+
+  public void setTableObj(String tableObj) {
+    this.tableObj = tableObj;
+  }
+
+  public void unsetTableObj() {
+    this.tableObj = null;
+  }
+
+  /** Returns true if field tableObj is set (has been assigned a value) and false otherwise */
+  public boolean isSetTableObj() {
+    return this.tableObj != null;
+  }
+
+  public void setTableObjIsSet(boolean value) {
+    if (!value) {
+      this.tableObj = null;
+    }
+  }
+
+  public String getPartitionObj() {
+    return this.partitionObj;
+  }
+
+  public void setPartitionObj(String partitionObj) {
+    this.partitionObj = partitionObj;
+  }
+
+  public void unsetPartitionObj() {
+    this.partitionObj = null;
+  }
+
+  /** Returns true if field partitionObj is set (has been assigned a value) and false otherwise */
+  public boolean isSetPartitionObj() {
+    return this.partitionObj != null;
+  }
+
+  public void setPartitionObjIsSet(boolean value) {
+    if (!value) {
+      this.partitionObj = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case WRITE_ID:
+      if (value == null) {
+        unsetWriteId();
+      } else {
+        setWriteId((Long)value);
+      }
+      break;
+
+    case DATABASE:
+      if (value == null) {
+        unsetDatabase();
+      } else {
+        setDatabase((String)value);
+      }
+      break;
+
+    case TABLE:
+      if (value == null) {
+        unsetTable();
+      } else {
+        setTable((String)value);
+      }
+      break;
+
+    case FILES:
+      if (value == null) {
+        unsetFiles();
+      } else {
+        setFiles((String)value);
+      }
+      break;
+
+    case PARTITION:
+      if (value == null) {
+        unsetPartition();
+      } else {
+        setPartition((String)value);
+      }
+      break;
+
+    case TABLE_OBJ:
+      if (value == null) {
+        unsetTableObj();
+      } else {
+        setTableObj((String)value);
+      }
+      break;
+
+    case PARTITION_OBJ:
+      if (value == null) {
+        unsetPartitionObj();
+      } else {
+        setPartitionObj((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case WRITE_ID:
+      return getWriteId();
+
+    case DATABASE:
+      return getDatabase();
+
+    case TABLE:
+      return getTable();
+
+    case FILES:
+      return getFiles();
+
+    case PARTITION:
+      return getPartition();
+
+    case TABLE_OBJ:
+      return getTableObj();
+
+    case PARTITION_OBJ:
+      return getPartitionObj();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case WRITE_ID:
+      return isSetWriteId();
+    case DATABASE:
+      return isSetDatabase();
+    case TABLE:
+      return isSetTable();
+    case FILES:
+      return isSetFiles();
+    case PARTITION:
+      return isSetPartition();
+    case TABLE_OBJ:
+      return isSetTableObj();
+    case PARTITION_OBJ:
+      return isSetPartitionObj();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof WriteEventInfo)
+      return this.equals((WriteEventInfo)that);
+    return false;
+  }
+
+  public boolean equals(WriteEventInfo that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_writeId = true;
+    boolean that_present_writeId = true;
+    if (this_present_writeId || that_present_writeId) {
+      if (!(this_present_writeId && that_present_writeId))
+        return false;
+      if (this.writeId != that.writeId)
+        return false;
+    }
+
+    boolean this_present_database = true && this.isSetDatabase();
+    boolean that_present_database = true && that.isSetDatabase();
+    if (this_present_database || that_present_database) {
+      if (!(this_present_database && that_present_database))
+        return false;
+      if (!this.database.equals(that.database))
+        return false;
+    }
+
+    boolean this_present_table = true && this.isSetTable();
+    boolean that_present_table = true && that.isSetTable();
+    if (this_present_table || that_present_table) {
+      if (!(this_present_table && that_present_table))
+        return false;
+      if (!this.table.equals(that.table))
+        return false;
+    }
+
+    boolean this_present_files = true && this.isSetFiles();
+    boolean that_present_files = true && that.isSetFiles();
+    if (this_present_files || that_present_files) {
+      if (!(this_present_files && that_present_files))
+        return false;
+      if (!this.files.equals(that.files))
+        return false;
+    }
+
+    boolean this_present_partition = true && this.isSetPartition();
+    boolean that_present_partition = true && that.isSetPartition();
+    if (this_present_partition || that_present_partition) {
+      if (!(this_present_partition && that_present_partition))
+        return false;
+      if (!this.partition.equals(that.partition))
+        return false;
+    }
+
+    boolean this_present_tableObj = true && this.isSetTableObj();
+    boolean that_present_tableObj = true && that.isSetTableObj();
+    if (this_present_tableObj || that_present_tableObj) {
+      if (!(this_present_tableObj && that_present_tableObj))
+        return false;
+      if (!this.tableObj.equals(that.tableObj))
+        return false;
+    }
+
+    boolean this_present_partitionObj = true && this.isSetPartitionObj();
+    boolean that_present_partitionObj = true && that.isSetPartitionObj();
+    if (this_present_partitionObj || that_present_partitionObj) {
+      if (!(this_present_partitionObj && that_present_partitionObj))
+        return false;
+      if (!this.partitionObj.equals(that.partitionObj))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_writeId = true;
+    list.add(present_writeId);
+    if (present_writeId)
+      list.add(writeId);
+
+    boolean present_database = true && (isSetDatabase());
+    list.add(present_database);
+    if (present_database)
+      list.add(database);
+
+    boolean present_table = true && (isSetTable());
+    list.add(present_table);
+    if (present_table)
+      list.add(table);
+
+    boolean present_files = true && (isSetFiles());
+    list.add(present_files);
+    if (present_files)
+      list.add(files);
+
+    boolean present_partition = true && (isSetPartition());
+    list.add(present_partition);
+    if (present_partition)
+      list.add(partition);
+
+    boolean present_tableObj = true && (isSetTableObj());
+    list.add(present_tableObj);
+    if (present_tableObj)
+      list.add(tableObj);
+
+    boolean present_partitionObj = true && (isSetPartitionObj());
+    list.add(present_partitionObj);
+    if (present_partitionObj)
+      list.add(partitionObj);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(WriteEventInfo other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetWriteId()).compareTo(other.isSetWriteId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeId, other.writeId);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDatabase()).compareTo(other.isSetDatabase());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDatabase()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.database, other.database);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable()).compareTo(other.isSetTable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, other.table);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetFiles()).compareTo(other.isSetFiles());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetFiles()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.files, other.files);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPartition()).compareTo(other.isSetPartition());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartition()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partition, other.partition);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTableObj()).compareTo(other.isSetTableObj());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTableObj()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableObj, other.tableObj);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetPartitionObj()).compareTo(other.isSetPartitionObj());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetPartitionObj()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.partitionObj, other.partitionObj);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("WriteEventInfo(");
+    boolean first = true;
+
+    sb.append("writeId:");
+    sb.append(this.writeId);
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("database:");
+    if (this.database == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.database);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("table:");
+    if (this.table == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.table);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("files:");
+    if (this.files == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.files);
+    }
+    first = false;
+    if (isSetPartition()) {
+      if (!first) sb.append(", ");
+      sb.append("partition:");
+      if (this.partition == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partition);
+      }
+      first = false;
+    }
+    if (isSetTableObj()) {
+      if (!first) sb.append(", ");
+      sb.append("tableObj:");
+      if (this.tableObj == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableObj);
+      }
+      first = false;
+    }
+    if (isSetPartitionObj()) {
+      if (!first) sb.append(", ");
+      sb.append("partitionObj:");
+      if (this.partitionObj == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.partitionObj);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetWriteId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'writeId' is unset! Struct:" + toString());
+    }
+
+    if (!isSetDatabase()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'database' is unset! Struct:" + toString());
+    }
+
+    if (!isSetTable()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'table' is unset! Struct:" + toString());
+    }
+
+    if (!isSetFiles()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'files' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
+      __isset_bitfield = 0;
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class WriteEventInfoStandardSchemeFactory implements SchemeFactory {
+    public WriteEventInfoStandardScheme getScheme() {
+      return new WriteEventInfoStandardScheme();
+    }
+  }
+
+  private static class WriteEventInfoStandardScheme extends StandardScheme<WriteEventInfo> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, WriteEventInfo struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // WRITE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.writeId = iprot.readI64();
+              struct.setWriteIdIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // DATABASE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.database = iprot.readString();
+              struct.setDatabaseIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // TABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table = iprot.readString();
+              struct.setTableIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // FILES
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.files = iprot.readString();
+              struct.setFilesIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 5: // PARTITION
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.partition = iprot.readString();
+              struct.setPartitionIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 6: // TABLE_OBJ
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.tableObj = iprot.readString();
+              struct.setTableObjIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // PARTITION_OBJ
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.partitionObj = iprot.readString();
+              struct.setPartitionObjIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, WriteEventInfo struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(WRITE_ID_FIELD_DESC);
+      oprot.writeI64(struct.writeId);
+      oprot.writeFieldEnd();
+      if (struct.database != null) {
+        oprot.writeFieldBegin(DATABASE_FIELD_DESC);
+        oprot.writeString(struct.database);
+        oprot.writeFieldEnd();
+      }
+      if (struct.table != null) {
+        oprot.writeFieldBegin(TABLE_FIELD_DESC);
+        oprot.writeString(struct.table);
+        oprot.writeFieldEnd();
+      }
+      if (struct.files != null) {
+        oprot.writeFieldBegin(FILES_FIELD_DESC);
+        oprot.writeString(struct.files);
+        oprot.writeFieldEnd();
+      }
+      if (struct.partition != null) {
+        if (struct.isSetPartition()) {
+          oprot.writeFieldBegin(PARTITION_FIELD_DESC);
+          oprot.writeString(struct.partition);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.tableObj != null) {
+        if (struct.isSetTableObj()) {
+          oprot.writeFieldBegin(TABLE_OBJ_FIELD_DESC);
+          oprot.writeString(struct.tableObj);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.partitionObj != null) {
+        if (struct.isSetPartitionObj()) {
+          oprot.writeFieldBegin(PARTITION_OBJ_FIELD_DESC);
+          oprot.writeString(struct.partitionObj);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class WriteEventInfoTupleSchemeFactory implements SchemeFactory {
+    public WriteEventInfoTupleScheme getScheme() {
+      return new WriteEventInfoTupleScheme();
+    }
+  }
+
+  private static class WriteEventInfoTupleScheme extends TupleScheme<WriteEventInfo> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, WriteEventInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.writeId);
+      oprot.writeString(struct.database);
+      oprot.writeString(struct.table);
+      oprot.writeString(struct.files);
+      BitSet optionals = new BitSet();
+      if (struct.isSetPartition()) {
+        optionals.set(0);
+      }
+      if (struct.isSetTableObj()) {
+        optionals.set(1);
+      }
+      if (struct.isSetPartitionObj()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetPartition()) {
+        oprot.writeString(struct.partition);
+      }
+      if (struct.isSetTableObj()) {
+        oprot.writeString(struct.tableObj);
+      }
+      if (struct.isSetPartitionObj()) {
+        oprot.writeString(struct.partitionObj);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, WriteEventInfo struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.writeId = iprot.readI64();
+      struct.setWriteIdIsSet(true);
+      struct.database = iprot.readString();
+      struct.setDatabaseIsSet(true);
+      struct.table = iprot.readString();
+      struct.setTableIsSet(true);
+      struct.files = iprot.readString();
+      struct.setFilesIsSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.partition = iprot.readString();
+        struct.setPartitionIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.tableObj = iprot.readString();
+        struct.setTableObjIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.partitionObj = iprot.readString();
+        struct.setPartitionObjIsSet(true);
+      }
+    }
+  }
+
+}
+


[02/46] hive git commit: HIVE-20038: Update queries on non-bucketed + partitioned tables throws NPE (Prasanth Jayachandran reviewed by Gopal V)

Posted by se...@apache.org.
HIVE-20038: Update queries on non-bucketed + partitioned tables throws NPE (Prasanth Jayachandran 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/221dbe08
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/221dbe08
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/221dbe08

Branch: refs/heads/master-txnstats
Commit: 221dbe085950e198d0766d60d6b00a70b30e5935
Parents: b122aea
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Sun Jul 1 23:42:48 2018 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Sun Jul 1 23:42:48 2018 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  4 +-
 .../apache/hadoop/hive/ql/TestTxnNoBuckets.java | 41 ++++++++++++++++++++
 2 files changed, 44 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/221dbe08/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 21f8268..949a9e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -224,7 +224,9 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     private void commit(FileSystem fs, List<Path> commitPaths) throws HiveException {
       for (int idx = 0; idx < outPaths.length; ++idx) {
         try {
-          commitOneOutPath(idx, fs, commitPaths);
+          if (outPaths[idx] != null) {
+            commitOneOutPath(idx, fs, commitPaths);
+          }
         } catch (IOException e) {
           throw new HiveException("Unable to commit output from: " +
               outPaths[idx] + " to: " + finalPaths[idx], e);

http://git-wip-us.apache.org/repos/asf/hive/blob/221dbe08/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
index cf68d32..bbe9d5a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnNoBuckets.java
@@ -184,6 +184,47 @@ public class TestTxnNoBuckets extends TxnCommandsBaseForTests {
     assertExpectedFileSet(expectedFiles, getWarehouseDir() + "/nobuckets");
   }
 
+  @Test
+  public void testNoBucketsDP() throws Exception {
+    hiveConf.setVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE, "nonstrict");
+    int[][] sourceVals1 = {{0,0,0},{3,3,3}};
+    int[][] sourceVals2 = {{1,1,1},{2,2,2}};
+    int[][] sourceVals3 = {{3,3,3},{4,4,4}};
+    int[][] sourceVals4 = {{5,5,5},{6,6,6}};
+    runStatementOnDriver("drop table if exists tmp");
+    runStatementOnDriver("create table tmp (c1 integer, c2 integer) partitioned by (c3 integer) stored as orc " +
+      "tblproperties('transactional'='false')");
+    runStatementOnDriver("insert into tmp " + makeValuesClause(sourceVals1));
+    runStatementOnDriver("insert into tmp " + makeValuesClause(sourceVals2));
+    runStatementOnDriver("insert into tmp " + makeValuesClause(sourceVals3));
+    runStatementOnDriver("insert into tmp " + makeValuesClause(sourceVals4));
+    runStatementOnDriver("drop table if exists nobuckets");
+    runStatementOnDriver("create table nobuckets (c1 integer, c2 integer) partitioned by (c3 integer) stored " +
+      "as orc tblproperties('transactional'='true', 'transactional_properties'='default')");
+    String stmt = "insert into nobuckets partition(c3) select * from tmp";
+    runStatementOnDriver(stmt);
+    List<String> rs = runStatementOnDriver(
+      "select ROW__ID, c1, c2, c3, INPUT__FILE__NAME from nobuckets order by ROW__ID");
+    Assert.assertEquals("", 8, rs.size());
+    LOG.warn("after insert");
+    for(String s : rs) {
+      LOG.warn(s);
+    }
+
+    rs = runStatementOnDriver(
+      "select * from nobuckets where c2 in (0,3)");
+    Assert.assertEquals(3, rs.size());
+    runStatementOnDriver("update nobuckets set c2 = 17 where c2 in(0,3)");
+    rs = runStatementOnDriver("select ROW__ID, c1, c2, c3, INPUT__FILE__NAME from nobuckets order by INPUT__FILE__NAME, ROW__ID");
+    LOG.warn("after update");
+    for(String s : rs) {
+      LOG.warn(s);
+    }
+    rs = runStatementOnDriver(
+      "select * from nobuckets where c2=17");
+    Assert.assertEquals(3, rs.size());
+  }
+
   /**
    * See CTAS tests in TestAcidOnTez
    */


[44/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index da14ac1,7f06b3b..8980258
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@@ -17286,6 -16635,26 +17291,26 @@@ uint32_t CommitTxnRequest::read(::apach
            xfer += iprot->skip(ftype);
          }
          break;
+       case 3:
+         if (ftype == ::apache::thrift::protocol::T_LIST) {
+           {
+             this->writeEventInfos.clear();
 -            uint32_t _size673;
 -            ::apache::thrift::protocol::TType _etype676;
 -            xfer += iprot->readListBegin(_etype676, _size673);
 -            this->writeEventInfos.resize(_size673);
 -            uint32_t _i677;
 -            for (_i677 = 0; _i677 < _size673; ++_i677)
++            uint32_t _size681;
++            ::apache::thrift::protocol::TType _etype684;
++            xfer += iprot->readListBegin(_etype684, _size681);
++            this->writeEventInfos.resize(_size681);
++            uint32_t _i685;
++            for (_i685 = 0; _i685 < _size681; ++_i685)
+             {
 -              xfer += this->writeEventInfos[_i677].read(iprot);
++              xfer += this->writeEventInfos[_i685].read(iprot);
+             }
+             xfer += iprot->readListEnd();
+           }
+           this->__isset.writeEventInfos = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
        default:
          xfer += iprot->skip(ftype);
          break;
@@@ -17314,6 -16683,19 +17339,19 @@@ uint32_t CommitTxnRequest::write(::apac
      xfer += oprot->writeString(this->replPolicy);
      xfer += oprot->writeFieldEnd();
    }
+   if (this->__isset.writeEventInfos) {
+     xfer += oprot->writeFieldBegin("writeEventInfos", ::apache::thrift::protocol::T_LIST, 3);
+     {
+       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->writeEventInfos.size()));
 -      std::vector<WriteEventInfo> ::const_iterator _iter678;
 -      for (_iter678 = this->writeEventInfos.begin(); _iter678 != this->writeEventInfos.end(); ++_iter678)
++      std::vector<WriteEventInfo> ::const_iterator _iter686;
++      for (_iter686 = this->writeEventInfos.begin(); _iter686 != this->writeEventInfos.end(); ++_iter686)
+       {
 -        xfer += (*_iter678).write(oprot);
++        xfer += (*_iter686).write(oprot);
+       }
+       xfer += oprot->writeListEnd();
+     }
+     xfer += oprot->writeFieldEnd();
+   }
    xfer += oprot->writeFieldStop();
    xfer += oprot->writeStructEnd();
    return xfer;
@@@ -17326,15 -16709,17 +17365,17 @@@ void swap(CommitTxnRequest &a, CommitTx
    swap(a.__isset, b.__isset);
  }
  
- CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other681) {
-   txnid = other681.txnid;
-   replPolicy = other681.replPolicy;
-   __isset = other681.__isset;
 -CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other679) {
 -  txnid = other679.txnid;
 -  replPolicy = other679.replPolicy;
 -  writeEventInfos = other679.writeEventInfos;
 -  __isset = other679.__isset;
++CommitTxnRequest::CommitTxnRequest(const CommitTxnRequest& other687) {
++  txnid = other687.txnid;
++  replPolicy = other687.replPolicy;
++  writeEventInfos = other687.writeEventInfos;
++  __isset = other687.__isset;
  }
- CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other682) {
-   txnid = other682.txnid;
-   replPolicy = other682.replPolicy;
-   __isset = other682.__isset;
 -CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other680) {
 -  txnid = other680.txnid;
 -  replPolicy = other680.replPolicy;
 -  writeEventInfos = other680.writeEventInfos;
 -  __isset = other680.__isset;
++CommitTxnRequest& CommitTxnRequest::operator=(const CommitTxnRequest& other688) {
++  txnid = other688.txnid;
++  replPolicy = other688.replPolicy;
++  writeEventInfos = other688.writeEventInfos;
++  __isset = other688.__isset;
    return *this;
  }
  void CommitTxnRequest::printTo(std::ostream& out) const {
@@@ -17342,6 -16727,231 +17383,231 @@@
    out << "CommitTxnRequest(";
    out << "txnid=" << to_string(txnid);
    out << ", " << "replPolicy="; (__isset.replPolicy ? (out << to_string(replPolicy)) : (out << "<null>"));
+   out << ", " << "writeEventInfos="; (__isset.writeEventInfos ? (out << to_string(writeEventInfos)) : (out << "<null>"));
+   out << ")";
+ }
+ 
+ 
+ WriteEventInfo::~WriteEventInfo() throw() {
+ }
+ 
+ 
+ void WriteEventInfo::__set_writeId(const int64_t val) {
+   this->writeId = val;
+ }
+ 
+ void WriteEventInfo::__set_database(const std::string& val) {
+   this->database = val;
+ }
+ 
+ void WriteEventInfo::__set_table(const std::string& val) {
+   this->table = val;
+ }
+ 
+ void WriteEventInfo::__set_files(const std::string& val) {
+   this->files = val;
+ }
+ 
+ void WriteEventInfo::__set_partition(const std::string& val) {
+   this->partition = val;
+ __isset.partition = true;
+ }
+ 
+ void WriteEventInfo::__set_tableObj(const std::string& val) {
+   this->tableObj = val;
+ __isset.tableObj = true;
+ }
+ 
+ void WriteEventInfo::__set_partitionObj(const std::string& val) {
+   this->partitionObj = val;
+ __isset.partitionObj = true;
+ }
+ 
+ uint32_t WriteEventInfo::read(::apache::thrift::protocol::TProtocol* iprot) {
+ 
+   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
+   uint32_t xfer = 0;
+   std::string fname;
+   ::apache::thrift::protocol::TType ftype;
+   int16_t fid;
+ 
+   xfer += iprot->readStructBegin(fname);
+ 
+   using ::apache::thrift::protocol::TProtocolException;
+ 
+   bool isset_writeId = false;
+   bool isset_database = false;
+   bool isset_table = false;
+   bool isset_files = false;
+ 
+   while (true)
+   {
+     xfer += iprot->readFieldBegin(fname, ftype, fid);
+     if (ftype == ::apache::thrift::protocol::T_STOP) {
+       break;
+     }
+     switch (fid)
+     {
+       case 1:
+         if (ftype == ::apache::thrift::protocol::T_I64) {
+           xfer += iprot->readI64(this->writeId);
+           isset_writeId = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 2:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->database);
+           isset_database = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 3:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->table);
+           isset_table = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 4:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->files);
+           isset_files = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 5:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->partition);
+           this->__isset.partition = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 6:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->tableObj);
+           this->__isset.tableObj = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       case 7:
+         if (ftype == ::apache::thrift::protocol::T_STRING) {
+           xfer += iprot->readString(this->partitionObj);
+           this->__isset.partitionObj = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
+       default:
+         xfer += iprot->skip(ftype);
+         break;
+     }
+     xfer += iprot->readFieldEnd();
+   }
+ 
+   xfer += iprot->readStructEnd();
+ 
+   if (!isset_writeId)
+     throw TProtocolException(TProtocolException::INVALID_DATA);
+   if (!isset_database)
+     throw TProtocolException(TProtocolException::INVALID_DATA);
+   if (!isset_table)
+     throw TProtocolException(TProtocolException::INVALID_DATA);
+   if (!isset_files)
+     throw TProtocolException(TProtocolException::INVALID_DATA);
+   return xfer;
+ }
+ 
+ uint32_t WriteEventInfo::write(::apache::thrift::protocol::TProtocol* oprot) const {
+   uint32_t xfer = 0;
+   apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot);
+   xfer += oprot->writeStructBegin("WriteEventInfo");
+ 
+   xfer += oprot->writeFieldBegin("writeId", ::apache::thrift::protocol::T_I64, 1);
+   xfer += oprot->writeI64(this->writeId);
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("database", ::apache::thrift::protocol::T_STRING, 2);
+   xfer += oprot->writeString(this->database);
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("table", ::apache::thrift::protocol::T_STRING, 3);
+   xfer += oprot->writeString(this->table);
+   xfer += oprot->writeFieldEnd();
+ 
+   xfer += oprot->writeFieldBegin("files", ::apache::thrift::protocol::T_STRING, 4);
+   xfer += oprot->writeString(this->files);
+   xfer += oprot->writeFieldEnd();
+ 
+   if (this->__isset.partition) {
+     xfer += oprot->writeFieldBegin("partition", ::apache::thrift::protocol::T_STRING, 5);
+     xfer += oprot->writeString(this->partition);
+     xfer += oprot->writeFieldEnd();
+   }
+   if (this->__isset.tableObj) {
+     xfer += oprot->writeFieldBegin("tableObj", ::apache::thrift::protocol::T_STRING, 6);
+     xfer += oprot->writeString(this->tableObj);
+     xfer += oprot->writeFieldEnd();
+   }
+   if (this->__isset.partitionObj) {
+     xfer += oprot->writeFieldBegin("partitionObj", ::apache::thrift::protocol::T_STRING, 7);
+     xfer += oprot->writeString(this->partitionObj);
+     xfer += oprot->writeFieldEnd();
+   }
+   xfer += oprot->writeFieldStop();
+   xfer += oprot->writeStructEnd();
+   return xfer;
+ }
+ 
+ void swap(WriteEventInfo &a, WriteEventInfo &b) {
+   using ::std::swap;
+   swap(a.writeId, b.writeId);
+   swap(a.database, b.database);
+   swap(a.table, b.table);
+   swap(a.files, b.files);
+   swap(a.partition, b.partition);
+   swap(a.tableObj, b.tableObj);
+   swap(a.partitionObj, b.partitionObj);
+   swap(a.__isset, b.__isset);
+ }
+ 
 -WriteEventInfo::WriteEventInfo(const WriteEventInfo& other681) {
 -  writeId = other681.writeId;
 -  database = other681.database;
 -  table = other681.table;
 -  files = other681.files;
 -  partition = other681.partition;
 -  tableObj = other681.tableObj;
 -  partitionObj = other681.partitionObj;
 -  __isset = other681.__isset;
 -}
 -WriteEventInfo& WriteEventInfo::operator=(const WriteEventInfo& other682) {
 -  writeId = other682.writeId;
 -  database = other682.database;
 -  table = other682.table;
 -  files = other682.files;
 -  partition = other682.partition;
 -  tableObj = other682.tableObj;
 -  partitionObj = other682.partitionObj;
 -  __isset = other682.__isset;
++WriteEventInfo::WriteEventInfo(const WriteEventInfo& other689) {
++  writeId = other689.writeId;
++  database = other689.database;
++  table = other689.table;
++  files = other689.files;
++  partition = other689.partition;
++  tableObj = other689.tableObj;
++  partitionObj = other689.partitionObj;
++  __isset = other689.__isset;
++}
++WriteEventInfo& WriteEventInfo::operator=(const WriteEventInfo& other690) {
++  writeId = other690.writeId;
++  database = other690.database;
++  table = other690.table;
++  files = other690.files;
++  partition = other690.partition;
++  tableObj = other690.tableObj;
++  partitionObj = other690.partitionObj;
++  __isset = other690.__isset;
+   return *this;
+ }
+ void WriteEventInfo::printTo(std::ostream& out) const {
+   using ::apache::thrift::to_string;
+   out << "WriteEventInfo(";
+   out << "writeId=" << to_string(writeId);
+   out << ", " << "database=" << to_string(database);
+   out << ", " << "table=" << to_string(table);
+   out << ", " << "files=" << to_string(files);
+   out << ", " << "partition="; (__isset.partition ? (out << to_string(partition)) : (out << "<null>"));
+   out << ", " << "tableObj="; (__isset.tableObj ? (out << to_string(tableObj)) : (out << "<null>"));
+   out << ", " << "partitionObj="; (__isset.partitionObj ? (out << to_string(partitionObj)) : (out << "<null>"));
    out << ")";
  }
  
@@@ -17445,14 -17055,14 +17711,14 @@@ uint32_t ReplTblWriteIdStateRequest::re
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->partNames.clear();
--            uint32_t _size683;
--            ::apache::thrift::protocol::TType _etype686;
--            xfer += iprot->readListBegin(_etype686, _size683);
--            this->partNames.resize(_size683);
--            uint32_t _i687;
--            for (_i687 = 0; _i687 < _size683; ++_i687)
++            uint32_t _size691;
++            ::apache::thrift::protocol::TType _etype694;
++            xfer += iprot->readListBegin(_etype694, _size691);
++            this->partNames.resize(_size691);
++            uint32_t _i695;
++            for (_i695 = 0; _i695 < _size691; ++_i695)
              {
--              xfer += iprot->readString(this->partNames[_i687]);
++              xfer += iprot->readString(this->partNames[_i695]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -17512,10 -17122,10 +17778,10 @@@ uint32_t ReplTblWriteIdStateRequest::wr
      xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 6);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
--      std::vector<std::string> ::const_iterator _iter688;
--      for (_iter688 = this->partNames.begin(); _iter688 != this->partNames.end(); ++_iter688)
++      std::vector<std::string> ::const_iterator _iter696;
++      for (_iter696 = this->partNames.begin(); _iter696 != this->partNames.end(); ++_iter696)
        {
--        xfer += oprot->writeString((*_iter688));
++        xfer += oprot->writeString((*_iter696));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -17537,23 -17147,23 +17803,23 @@@ void swap(ReplTblWriteIdStateRequest &a
    swap(a.__isset, b.__isset);
  }
  
--ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other689) {
--  validWriteIdlist = other689.validWriteIdlist;
--  user = other689.user;
--  hostName = other689.hostName;
--  dbName = other689.dbName;
--  tableName = other689.tableName;
--  partNames = other689.partNames;
--  __isset = other689.__isset;
--}
--ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other690) {
--  validWriteIdlist = other690.validWriteIdlist;
--  user = other690.user;
--  hostName = other690.hostName;
--  dbName = other690.dbName;
--  tableName = other690.tableName;
--  partNames = other690.partNames;
--  __isset = other690.__isset;
++ReplTblWriteIdStateRequest::ReplTblWriteIdStateRequest(const ReplTblWriteIdStateRequest& other697) {
++  validWriteIdlist = other697.validWriteIdlist;
++  user = other697.user;
++  hostName = other697.hostName;
++  dbName = other697.dbName;
++  tableName = other697.tableName;
++  partNames = other697.partNames;
++  __isset = other697.__isset;
++}
++ReplTblWriteIdStateRequest& ReplTblWriteIdStateRequest::operator=(const ReplTblWriteIdStateRequest& other698) {
++  validWriteIdlist = other698.validWriteIdlist;
++  user = other698.user;
++  hostName = other698.hostName;
++  dbName = other698.dbName;
++  tableName = other698.tableName;
++  partNames = other698.partNames;
++  __isset = other698.__isset;
    return *this;
  }
  void ReplTblWriteIdStateRequest::printTo(std::ostream& out) const {
@@@ -17608,14 -17218,14 +17874,14 @@@ uint32_t GetValidWriteIdsRequest::read(
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->fullTableNames.clear();
--            uint32_t _size691;
--            ::apache::thrift::protocol::TType _etype694;
--            xfer += iprot->readListBegin(_etype694, _size691);
--            this->fullTableNames.resize(_size691);
--            uint32_t _i695;
--            for (_i695 = 0; _i695 < _size691; ++_i695)
++            uint32_t _size699;
++            ::apache::thrift::protocol::TType _etype702;
++            xfer += iprot->readListBegin(_etype702, _size699);
++            this->fullTableNames.resize(_size699);
++            uint32_t _i703;
++            for (_i703 = 0; _i703 < _size699; ++_i703)
              {
--              xfer += iprot->readString(this->fullTableNames[_i695]);
++              xfer += iprot->readString(this->fullTableNames[_i703]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -17656,10 -17266,10 +17922,10 @@@ uint32_t GetValidWriteIdsRequest::write
    xfer += oprot->writeFieldBegin("fullTableNames", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->fullTableNames.size()));
--    std::vector<std::string> ::const_iterator _iter696;
--    for (_iter696 = this->fullTableNames.begin(); _iter696 != this->fullTableNames.end(); ++_iter696)
++    std::vector<std::string> ::const_iterator _iter704;
++    for (_iter704 = this->fullTableNames.begin(); _iter704 != this->fullTableNames.end(); ++_iter704)
      {
--      xfer += oprot->writeString((*_iter696));
++      xfer += oprot->writeString((*_iter704));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -17680,13 -17290,13 +17946,13 @@@ void swap(GetValidWriteIdsRequest &a, G
    swap(a.validTxnList, b.validTxnList);
  }
  
--GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other697) {
--  fullTableNames = other697.fullTableNames;
--  validTxnList = other697.validTxnList;
++GetValidWriteIdsRequest::GetValidWriteIdsRequest(const GetValidWriteIdsRequest& other705) {
++  fullTableNames = other705.fullTableNames;
++  validTxnList = other705.validTxnList;
  }
--GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other698) {
--  fullTableNames = other698.fullTableNames;
--  validTxnList = other698.validTxnList;
++GetValidWriteIdsRequest& GetValidWriteIdsRequest::operator=(const GetValidWriteIdsRequest& other706) {
++  fullTableNames = other706.fullTableNames;
++  validTxnList = other706.validTxnList;
    return *this;
  }
  void GetValidWriteIdsRequest::printTo(std::ostream& out) const {
@@@ -17768,14 -17378,14 +18034,14 @@@ uint32_t TableValidWriteIds::read(::apa
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->invalidWriteIds.clear();
--            uint32_t _size699;
--            ::apache::thrift::protocol::TType _etype702;
--            xfer += iprot->readListBegin(_etype702, _size699);
--            this->invalidWriteIds.resize(_size699);
--            uint32_t _i703;
--            for (_i703 = 0; _i703 < _size699; ++_i703)
++            uint32_t _size707;
++            ::apache::thrift::protocol::TType _etype710;
++            xfer += iprot->readListBegin(_etype710, _size707);
++            this->invalidWriteIds.resize(_size707);
++            uint32_t _i711;
++            for (_i711 = 0; _i711 < _size707; ++_i711)
              {
--              xfer += iprot->readI64(this->invalidWriteIds[_i703]);
++              xfer += iprot->readI64(this->invalidWriteIds[_i711]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -17836,10 -17446,10 +18102,10 @@@ uint32_t TableValidWriteIds::write(::ap
    xfer += oprot->writeFieldBegin("invalidWriteIds", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->invalidWriteIds.size()));
--    std::vector<int64_t> ::const_iterator _iter704;
--    for (_iter704 = this->invalidWriteIds.begin(); _iter704 != this->invalidWriteIds.end(); ++_iter704)
++    std::vector<int64_t> ::const_iterator _iter712;
++    for (_iter712 = this->invalidWriteIds.begin(); _iter712 != this->invalidWriteIds.end(); ++_iter712)
      {
--      xfer += oprot->writeI64((*_iter704));
++      xfer += oprot->writeI64((*_iter712));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -17869,21 -17479,21 +18135,21 @@@ void swap(TableValidWriteIds &a, TableV
    swap(a.__isset, b.__isset);
  }
  
--TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other705) {
--  fullTableName = other705.fullTableName;
--  writeIdHighWaterMark = other705.writeIdHighWaterMark;
--  invalidWriteIds = other705.invalidWriteIds;
--  minOpenWriteId = other705.minOpenWriteId;
--  abortedBits = other705.abortedBits;
--  __isset = other705.__isset;
--}
--TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other706) {
--  fullTableName = other706.fullTableName;
--  writeIdHighWaterMark = other706.writeIdHighWaterMark;
--  invalidWriteIds = other706.invalidWriteIds;
--  minOpenWriteId = other706.minOpenWriteId;
--  abortedBits = other706.abortedBits;
--  __isset = other706.__isset;
++TableValidWriteIds::TableValidWriteIds(const TableValidWriteIds& other713) {
++  fullTableName = other713.fullTableName;
++  writeIdHighWaterMark = other713.writeIdHighWaterMark;
++  invalidWriteIds = other713.invalidWriteIds;
++  minOpenWriteId = other713.minOpenWriteId;
++  abortedBits = other713.abortedBits;
++  __isset = other713.__isset;
++}
++TableValidWriteIds& TableValidWriteIds::operator=(const TableValidWriteIds& other714) {
++  fullTableName = other714.fullTableName;
++  writeIdHighWaterMark = other714.writeIdHighWaterMark;
++  invalidWriteIds = other714.invalidWriteIds;
++  minOpenWriteId = other714.minOpenWriteId;
++  abortedBits = other714.abortedBits;
++  __isset = other714.__isset;
    return *this;
  }
  void TableValidWriteIds::printTo(std::ostream& out) const {
@@@ -17932,14 -17542,14 +18198,14 @@@ uint32_t GetValidWriteIdsResponse::read
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->tblValidWriteIds.clear();
--            uint32_t _size707;
--            ::apache::thrift::protocol::TType _etype710;
--            xfer += iprot->readListBegin(_etype710, _size707);
--            this->tblValidWriteIds.resize(_size707);
--            uint32_t _i711;
--            for (_i711 = 0; _i711 < _size707; ++_i711)
++            uint32_t _size715;
++            ::apache::thrift::protocol::TType _etype718;
++            xfer += iprot->readListBegin(_etype718, _size715);
++            this->tblValidWriteIds.resize(_size715);
++            uint32_t _i719;
++            for (_i719 = 0; _i719 < _size715; ++_i719)
              {
--              xfer += this->tblValidWriteIds[_i711].read(iprot);
++              xfer += this->tblValidWriteIds[_i719].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -17970,10 -17580,10 +18236,10 @@@ uint32_t GetValidWriteIdsResponse::writ
    xfer += oprot->writeFieldBegin("tblValidWriteIds", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->tblValidWriteIds.size()));
--    std::vector<TableValidWriteIds> ::const_iterator _iter712;
--    for (_iter712 = this->tblValidWriteIds.begin(); _iter712 != this->tblValidWriteIds.end(); ++_iter712)
++    std::vector<TableValidWriteIds> ::const_iterator _iter720;
++    for (_iter720 = this->tblValidWriteIds.begin(); _iter720 != this->tblValidWriteIds.end(); ++_iter720)
      {
--      xfer += (*_iter712).write(oprot);
++      xfer += (*_iter720).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -17989,11 -17599,11 +18255,11 @@@ void swap(GetValidWriteIdsResponse &a, 
    swap(a.tblValidWriteIds, b.tblValidWriteIds);
  }
  
--GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other713) {
--  tblValidWriteIds = other713.tblValidWriteIds;
++GetValidWriteIdsResponse::GetValidWriteIdsResponse(const GetValidWriteIdsResponse& other721) {
++  tblValidWriteIds = other721.tblValidWriteIds;
  }
--GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other714) {
--  tblValidWriteIds = other714.tblValidWriteIds;
++GetValidWriteIdsResponse& GetValidWriteIdsResponse::operator=(const GetValidWriteIdsResponse& other722) {
++  tblValidWriteIds = other722.tblValidWriteIds;
    return *this;
  }
  void GetValidWriteIdsResponse::printTo(std::ostream& out) const {
@@@ -18074,14 -17684,14 +18340,14 @@@ uint32_t AllocateTableWriteIdsRequest::
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->txnIds.clear();
--            uint32_t _size715;
--            ::apache::thrift::protocol::TType _etype718;
--            xfer += iprot->readListBegin(_etype718, _size715);
--            this->txnIds.resize(_size715);
--            uint32_t _i719;
--            for (_i719 = 0; _i719 < _size715; ++_i719)
++            uint32_t _size723;
++            ::apache::thrift::protocol::TType _etype726;
++            xfer += iprot->readListBegin(_etype726, _size723);
++            this->txnIds.resize(_size723);
++            uint32_t _i727;
++            for (_i727 = 0; _i727 < _size723; ++_i727)
              {
--              xfer += iprot->readI64(this->txnIds[_i719]);
++              xfer += iprot->readI64(this->txnIds[_i727]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -18102,14 -17712,14 +18368,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->srcTxnToWriteIdList.clear();
--            uint32_t _size720;
--            ::apache::thrift::protocol::TType _etype723;
--            xfer += iprot->readListBegin(_etype723, _size720);
--            this->srcTxnToWriteIdList.resize(_size720);
--            uint32_t _i724;
--            for (_i724 = 0; _i724 < _size720; ++_i724)
++            uint32_t _size728;
++            ::apache::thrift::protocol::TType _etype731;
++            xfer += iprot->readListBegin(_etype731, _size728);
++            this->srcTxnToWriteIdList.resize(_size728);
++            uint32_t _i732;
++            for (_i732 = 0; _i732 < _size728; ++_i732)
              {
--              xfer += this->srcTxnToWriteIdList[_i724].read(iprot);
++              xfer += this->srcTxnToWriteIdList[_i732].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -18151,10 -17761,10 +18417,10 @@@ uint32_t AllocateTableWriteIdsRequest::
      xfer += oprot->writeFieldBegin("txnIds", ::apache::thrift::protocol::T_LIST, 3);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->txnIds.size()));
--      std::vector<int64_t> ::const_iterator _iter725;
--      for (_iter725 = this->txnIds.begin(); _iter725 != this->txnIds.end(); ++_iter725)
++      std::vector<int64_t> ::const_iterator _iter733;
++      for (_iter733 = this->txnIds.begin(); _iter733 != this->txnIds.end(); ++_iter733)
        {
--        xfer += oprot->writeI64((*_iter725));
++        xfer += oprot->writeI64((*_iter733));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -18169,10 -17779,10 +18435,10 @@@
      xfer += oprot->writeFieldBegin("srcTxnToWriteIdList", ::apache::thrift::protocol::T_LIST, 5);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->srcTxnToWriteIdList.size()));
--      std::vector<TxnToWriteId> ::const_iterator _iter726;
--      for (_iter726 = this->srcTxnToWriteIdList.begin(); _iter726 != this->srcTxnToWriteIdList.end(); ++_iter726)
++      std::vector<TxnToWriteId> ::const_iterator _iter734;
++      for (_iter734 = this->srcTxnToWriteIdList.begin(); _iter734 != this->srcTxnToWriteIdList.end(); ++_iter734)
        {
--        xfer += (*_iter726).write(oprot);
++        xfer += (*_iter734).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -18193,21 -17803,21 +18459,21 @@@ void swap(AllocateTableWriteIdsRequest 
    swap(a.__isset, b.__isset);
  }
  
--AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other727) {
--  dbName = other727.dbName;
--  tableName = other727.tableName;
--  txnIds = other727.txnIds;
--  replPolicy = other727.replPolicy;
--  srcTxnToWriteIdList = other727.srcTxnToWriteIdList;
--  __isset = other727.__isset;
--}
--AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other728) {
--  dbName = other728.dbName;
--  tableName = other728.tableName;
--  txnIds = other728.txnIds;
--  replPolicy = other728.replPolicy;
--  srcTxnToWriteIdList = other728.srcTxnToWriteIdList;
--  __isset = other728.__isset;
++AllocateTableWriteIdsRequest::AllocateTableWriteIdsRequest(const AllocateTableWriteIdsRequest& other735) {
++  dbName = other735.dbName;
++  tableName = other735.tableName;
++  txnIds = other735.txnIds;
++  replPolicy = other735.replPolicy;
++  srcTxnToWriteIdList = other735.srcTxnToWriteIdList;
++  __isset = other735.__isset;
++}
++AllocateTableWriteIdsRequest& AllocateTableWriteIdsRequest::operator=(const AllocateTableWriteIdsRequest& other736) {
++  dbName = other736.dbName;
++  tableName = other736.tableName;
++  txnIds = other736.txnIds;
++  replPolicy = other736.replPolicy;
++  srcTxnToWriteIdList = other736.srcTxnToWriteIdList;
++  __isset = other736.__isset;
    return *this;
  }
  void AllocateTableWriteIdsRequest::printTo(std::ostream& out) const {
@@@ -18313,13 -17923,13 +18579,13 @@@ void swap(TxnToWriteId &a, TxnToWriteI
    swap(a.writeId, b.writeId);
  }
  
--TxnToWriteId::TxnToWriteId(const TxnToWriteId& other729) {
--  txnId = other729.txnId;
--  writeId = other729.writeId;
++TxnToWriteId::TxnToWriteId(const TxnToWriteId& other737) {
++  txnId = other737.txnId;
++  writeId = other737.writeId;
  }
--TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other730) {
--  txnId = other730.txnId;
--  writeId = other730.writeId;
++TxnToWriteId& TxnToWriteId::operator=(const TxnToWriteId& other738) {
++  txnId = other738.txnId;
++  writeId = other738.writeId;
    return *this;
  }
  void TxnToWriteId::printTo(std::ostream& out) const {
@@@ -18365,14 -17975,14 +18631,14 @@@ uint32_t AllocateTableWriteIdsResponse:
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->txnToWriteIds.clear();
--            uint32_t _size731;
--            ::apache::thrift::protocol::TType _etype734;
--            xfer += iprot->readListBegin(_etype734, _size731);
--            this->txnToWriteIds.resize(_size731);
--            uint32_t _i735;
--            for (_i735 = 0; _i735 < _size731; ++_i735)
++            uint32_t _size739;
++            ::apache::thrift::protocol::TType _etype742;
++            xfer += iprot->readListBegin(_etype742, _size739);
++            this->txnToWriteIds.resize(_size739);
++            uint32_t _i743;
++            for (_i743 = 0; _i743 < _size739; ++_i743)
              {
--              xfer += this->txnToWriteIds[_i735].read(iprot);
++              xfer += this->txnToWriteIds[_i743].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -18403,10 -18013,10 +18669,10 @@@ uint32_t AllocateTableWriteIdsResponse:
    xfer += oprot->writeFieldBegin("txnToWriteIds", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->txnToWriteIds.size()));
--    std::vector<TxnToWriteId> ::const_iterator _iter736;
--    for (_iter736 = this->txnToWriteIds.begin(); _iter736 != this->txnToWriteIds.end(); ++_iter736)
++    std::vector<TxnToWriteId> ::const_iterator _iter744;
++    for (_iter744 = this->txnToWriteIds.begin(); _iter744 != this->txnToWriteIds.end(); ++_iter744)
      {
--      xfer += (*_iter736).write(oprot);
++      xfer += (*_iter744).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -18422,11 -18032,11 +18688,11 @@@ void swap(AllocateTableWriteIdsRespons
    swap(a.txnToWriteIds, b.txnToWriteIds);
  }
  
--AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other737) {
--  txnToWriteIds = other737.txnToWriteIds;
++AllocateTableWriteIdsResponse::AllocateTableWriteIdsResponse(const AllocateTableWriteIdsResponse& other745) {
++  txnToWriteIds = other745.txnToWriteIds;
  }
--AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other738) {
--  txnToWriteIds = other738.txnToWriteIds;
++AllocateTableWriteIdsResponse& AllocateTableWriteIdsResponse::operator=(const AllocateTableWriteIdsResponse& other746) {
++  txnToWriteIds = other746.txnToWriteIds;
    return *this;
  }
  void AllocateTableWriteIdsResponse::printTo(std::ostream& out) const {
@@@ -18504,9 -18114,9 +18770,9 @@@ uint32_t LockComponent::read(::apache::
      {
        case 1:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast739;
--          xfer += iprot->readI32(ecast739);
--          this->type = (LockType::type)ecast739;
++          int32_t ecast747;
++          xfer += iprot->readI32(ecast747);
++          this->type = (LockType::type)ecast747;
            isset_type = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -18514,9 -18124,9 +18780,9 @@@
          break;
        case 2:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast740;
--          xfer += iprot->readI32(ecast740);
--          this->level = (LockLevel::type)ecast740;
++          int32_t ecast748;
++          xfer += iprot->readI32(ecast748);
++          this->level = (LockLevel::type)ecast748;
            isset_level = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -18548,9 -18158,9 +18814,9 @@@
          break;
        case 6:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast741;
--          xfer += iprot->readI32(ecast741);
--          this->operationType = (DataOperationType::type)ecast741;
++          int32_t ecast749;
++          xfer += iprot->readI32(ecast749);
++          this->operationType = (DataOperationType::type)ecast749;
            this->__isset.operationType = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -18650,27 -18260,27 +18916,27 @@@ void swap(LockComponent &a, LockCompone
    swap(a.__isset, b.__isset);
  }
  
--LockComponent::LockComponent(const LockComponent& other742) {
--  type = other742.type;
--  level = other742.level;
--  dbname = other742.dbname;
--  tablename = other742.tablename;
--  partitionname = other742.partitionname;
--  operationType = other742.operationType;
--  isTransactional = other742.isTransactional;
--  isDynamicPartitionWrite = other742.isDynamicPartitionWrite;
--  __isset = other742.__isset;
--}
--LockComponent& LockComponent::operator=(const LockComponent& other743) {
--  type = other743.type;
--  level = other743.level;
--  dbname = other743.dbname;
--  tablename = other743.tablename;
--  partitionname = other743.partitionname;
--  operationType = other743.operationType;
--  isTransactional = other743.isTransactional;
--  isDynamicPartitionWrite = other743.isDynamicPartitionWrite;
--  __isset = other743.__isset;
++LockComponent::LockComponent(const LockComponent& other750) {
++  type = other750.type;
++  level = other750.level;
++  dbname = other750.dbname;
++  tablename = other750.tablename;
++  partitionname = other750.partitionname;
++  operationType = other750.operationType;
++  isTransactional = other750.isTransactional;
++  isDynamicPartitionWrite = other750.isDynamicPartitionWrite;
++  __isset = other750.__isset;
++}
++LockComponent& LockComponent::operator=(const LockComponent& other751) {
++  type = other751.type;
++  level = other751.level;
++  dbname = other751.dbname;
++  tablename = other751.tablename;
++  partitionname = other751.partitionname;
++  operationType = other751.operationType;
++  isTransactional = other751.isTransactional;
++  isDynamicPartitionWrite = other751.isDynamicPartitionWrite;
++  __isset = other751.__isset;
    return *this;
  }
  void LockComponent::printTo(std::ostream& out) const {
@@@ -18742,14 -18352,14 +19008,14 @@@ uint32_t LockRequest::read(::apache::th
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->component.clear();
--            uint32_t _size744;
--            ::apache::thrift::protocol::TType _etype747;
--            xfer += iprot->readListBegin(_etype747, _size744);
--            this->component.resize(_size744);
--            uint32_t _i748;
--            for (_i748 = 0; _i748 < _size744; ++_i748)
++            uint32_t _size752;
++            ::apache::thrift::protocol::TType _etype755;
++            xfer += iprot->readListBegin(_etype755, _size752);
++            this->component.resize(_size752);
++            uint32_t _i756;
++            for (_i756 = 0; _i756 < _size752; ++_i756)
              {
--              xfer += this->component[_i748].read(iprot);
++              xfer += this->component[_i756].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -18816,10 -18426,10 +19082,10 @@@ uint32_t LockRequest::write(::apache::t
    xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->component.size()));
--    std::vector<LockComponent> ::const_iterator _iter749;
--    for (_iter749 = this->component.begin(); _iter749 != this->component.end(); ++_iter749)
++    std::vector<LockComponent> ::const_iterator _iter757;
++    for (_iter757 = this->component.begin(); _iter757 != this->component.end(); ++_iter757)
      {
--      xfer += (*_iter749).write(oprot);
++      xfer += (*_iter757).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -18858,21 -18468,21 +19124,21 @@@ void swap(LockRequest &a, LockRequest &
    swap(a.__isset, b.__isset);
  }
  
--LockRequest::LockRequest(const LockRequest& other750) {
--  component = other750.component;
--  txnid = other750.txnid;
--  user = other750.user;
--  hostname = other750.hostname;
--  agentInfo = other750.agentInfo;
--  __isset = other750.__isset;
--}
--LockRequest& LockRequest::operator=(const LockRequest& other751) {
--  component = other751.component;
--  txnid = other751.txnid;
--  user = other751.user;
--  hostname = other751.hostname;
--  agentInfo = other751.agentInfo;
--  __isset = other751.__isset;
++LockRequest::LockRequest(const LockRequest& other758) {
++  component = other758.component;
++  txnid = other758.txnid;
++  user = other758.user;
++  hostname = other758.hostname;
++  agentInfo = other758.agentInfo;
++  __isset = other758.__isset;
++}
++LockRequest& LockRequest::operator=(const LockRequest& other759) {
++  component = other759.component;
++  txnid = other759.txnid;
++  user = other759.user;
++  hostname = other759.hostname;
++  agentInfo = other759.agentInfo;
++  __isset = other759.__isset;
    return *this;
  }
  void LockRequest::printTo(std::ostream& out) const {
@@@ -18932,9 -18542,9 +19198,9 @@@ uint32_t LockResponse::read(::apache::t
          break;
        case 2:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast752;
--          xfer += iprot->readI32(ecast752);
--          this->state = (LockState::type)ecast752;
++          int32_t ecast760;
++          xfer += iprot->readI32(ecast760);
++          this->state = (LockState::type)ecast760;
            isset_state = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -18980,13 -18590,13 +19246,13 @@@ void swap(LockResponse &a, LockRespons
    swap(a.state, b.state);
  }
  
--LockResponse::LockResponse(const LockResponse& other753) {
--  lockid = other753.lockid;
--  state = other753.state;
++LockResponse::LockResponse(const LockResponse& other761) {
++  lockid = other761.lockid;
++  state = other761.state;
  }
--LockResponse& LockResponse::operator=(const LockResponse& other754) {
--  lockid = other754.lockid;
--  state = other754.state;
++LockResponse& LockResponse::operator=(const LockResponse& other762) {
++  lockid = other762.lockid;
++  state = other762.state;
    return *this;
  }
  void LockResponse::printTo(std::ostream& out) const {
@@@ -19108,17 -18718,17 +19374,17 @@@ void swap(CheckLockRequest &a, CheckLoc
    swap(a.__isset, b.__isset);
  }
  
--CheckLockRequest::CheckLockRequest(const CheckLockRequest& other755) {
--  lockid = other755.lockid;
--  txnid = other755.txnid;
--  elapsed_ms = other755.elapsed_ms;
--  __isset = other755.__isset;
++CheckLockRequest::CheckLockRequest(const CheckLockRequest& other763) {
++  lockid = other763.lockid;
++  txnid = other763.txnid;
++  elapsed_ms = other763.elapsed_ms;
++  __isset = other763.__isset;
  }
--CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other756) {
--  lockid = other756.lockid;
--  txnid = other756.txnid;
--  elapsed_ms = other756.elapsed_ms;
--  __isset = other756.__isset;
++CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other764) {
++  lockid = other764.lockid;
++  txnid = other764.txnid;
++  elapsed_ms = other764.elapsed_ms;
++  __isset = other764.__isset;
    return *this;
  }
  void CheckLockRequest::printTo(std::ostream& out) const {
@@@ -19202,11 -18812,11 +19468,11 @@@ void swap(UnlockRequest &a, UnlockReque
    swap(a.lockid, b.lockid);
  }
  
--UnlockRequest::UnlockRequest(const UnlockRequest& other757) {
--  lockid = other757.lockid;
++UnlockRequest::UnlockRequest(const UnlockRequest& other765) {
++  lockid = other765.lockid;
  }
--UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other758) {
--  lockid = other758.lockid;
++UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other766) {
++  lockid = other766.lockid;
    return *this;
  }
  void UnlockRequest::printTo(std::ostream& out) const {
@@@ -19345,19 -18955,19 +19611,19 @@@ void swap(ShowLocksRequest &a, ShowLock
    swap(a.__isset, b.__isset);
  }
  
--ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other759) {
--  dbname = other759.dbname;
--  tablename = other759.tablename;
--  partname = other759.partname;
--  isExtended = other759.isExtended;
--  __isset = other759.__isset;
++ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other767) {
++  dbname = other767.dbname;
++  tablename = other767.tablename;
++  partname = other767.partname;
++  isExtended = other767.isExtended;
++  __isset = other767.__isset;
  }
--ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other760) {
--  dbname = other760.dbname;
--  tablename = other760.tablename;
--  partname = other760.partname;
--  isExtended = other760.isExtended;
--  __isset = other760.__isset;
++ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other768) {
++  dbname = other768.dbname;
++  tablename = other768.tablename;
++  partname = other768.partname;
++  isExtended = other768.isExtended;
++  __isset = other768.__isset;
    return *this;
  }
  void ShowLocksRequest::printTo(std::ostream& out) const {
@@@ -19510,9 -19120,9 +19776,9 @@@ uint32_t ShowLocksResponseElement::read
          break;
        case 5:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast761;
--          xfer += iprot->readI32(ecast761);
--          this->state = (LockState::type)ecast761;
++          int32_t ecast769;
++          xfer += iprot->readI32(ecast769);
++          this->state = (LockState::type)ecast769;
            isset_state = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -19520,9 -19130,9 +19786,9 @@@
          break;
        case 6:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast762;
--          xfer += iprot->readI32(ecast762);
--          this->type = (LockType::type)ecast762;
++          int32_t ecast770;
++          xfer += iprot->readI32(ecast770);
++          this->type = (LockType::type)ecast770;
            isset_type = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -19738,43 -19348,43 +20004,43 @@@ void swap(ShowLocksResponseElement &a, 
    swap(a.__isset, b.__isset);
  }
  
--ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other763) {
--  lockid = other763.lockid;
--  dbname = other763.dbname;
--  tablename = other763.tablename;
--  partname = other763.partname;
--  state = other763.state;
--  type = other763.type;
--  txnid = other763.txnid;
--  lastheartbeat = other763.lastheartbeat;
--  acquiredat = other763.acquiredat;
--  user = other763.user;
--  hostname = other763.hostname;
--  heartbeatCount = other763.heartbeatCount;
--  agentInfo = other763.agentInfo;
--  blockedByExtId = other763.blockedByExtId;
--  blockedByIntId = other763.blockedByIntId;
--  lockIdInternal = other763.lockIdInternal;
--  __isset = other763.__isset;
++ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other771) {
++  lockid = other771.lockid;
++  dbname = other771.dbname;
++  tablename = other771.tablename;
++  partname = other771.partname;
++  state = other771.state;
++  type = other771.type;
++  txnid = other771.txnid;
++  lastheartbeat = other771.lastheartbeat;
++  acquiredat = other771.acquiredat;
++  user = other771.user;
++  hostname = other771.hostname;
++  heartbeatCount = other771.heartbeatCount;
++  agentInfo = other771.agentInfo;
++  blockedByExtId = other771.blockedByExtId;
++  blockedByIntId = other771.blockedByIntId;
++  lockIdInternal = other771.lockIdInternal;
++  __isset = other771.__isset;
  }
--ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other764) {
--  lockid = other764.lockid;
--  dbname = other764.dbname;
--  tablename = other764.tablename;
--  partname = other764.partname;
--  state = other764.state;
--  type = other764.type;
--  txnid = other764.txnid;
--  lastheartbeat = other764.lastheartbeat;
--  acquiredat = other764.acquiredat;
--  user = other764.user;
--  hostname = other764.hostname;
--  heartbeatCount = other764.heartbeatCount;
--  agentInfo = other764.agentInfo;
--  blockedByExtId = other764.blockedByExtId;
--  blockedByIntId = other764.blockedByIntId;
--  lockIdInternal = other764.lockIdInternal;
--  __isset = other764.__isset;
++ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other772) {
++  lockid = other772.lockid;
++  dbname = other772.dbname;
++  tablename = other772.tablename;
++  partname = other772.partname;
++  state = other772.state;
++  type = other772.type;
++  txnid = other772.txnid;
++  lastheartbeat = other772.lastheartbeat;
++  acquiredat = other772.acquiredat;
++  user = other772.user;
++  hostname = other772.hostname;
++  heartbeatCount = other772.heartbeatCount;
++  agentInfo = other772.agentInfo;
++  blockedByExtId = other772.blockedByExtId;
++  blockedByIntId = other772.blockedByIntId;
++  lockIdInternal = other772.lockIdInternal;
++  __isset = other772.__isset;
    return *this;
  }
  void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@@ -19833,14 -19443,14 +20099,14 @@@ uint32_t ShowLocksResponse::read(::apac
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->locks.clear();
--            uint32_t _size765;
--            ::apache::thrift::protocol::TType _etype768;
--            xfer += iprot->readListBegin(_etype768, _size765);
--            this->locks.resize(_size765);
--            uint32_t _i769;
--            for (_i769 = 0; _i769 < _size765; ++_i769)
++            uint32_t _size773;
++            ::apache::thrift::protocol::TType _etype776;
++            xfer += iprot->readListBegin(_etype776, _size773);
++            this->locks.resize(_size773);
++            uint32_t _i777;
++            for (_i777 = 0; _i777 < _size773; ++_i777)
              {
--              xfer += this->locks[_i769].read(iprot);
++              xfer += this->locks[_i777].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -19869,10 -19479,10 +20135,10 @@@ uint32_t ShowLocksResponse::write(::apa
    xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->locks.size()));
--    std::vector<ShowLocksResponseElement> ::const_iterator _iter770;
--    for (_iter770 = this->locks.begin(); _iter770 != this->locks.end(); ++_iter770)
++    std::vector<ShowLocksResponseElement> ::const_iterator _iter778;
++    for (_iter778 = this->locks.begin(); _iter778 != this->locks.end(); ++_iter778)
      {
--      xfer += (*_iter770).write(oprot);
++      xfer += (*_iter778).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -19889,13 -19499,13 +20155,13 @@@ void swap(ShowLocksResponse &a, ShowLoc
    swap(a.__isset, b.__isset);
  }
  
--ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other771) {
--  locks = other771.locks;
--  __isset = other771.__isset;
++ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other779) {
++  locks = other779.locks;
++  __isset = other779.__isset;
  }
--ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other772) {
--  locks = other772.locks;
--  __isset = other772.__isset;
++ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other780) {
++  locks = other780.locks;
++  __isset = other780.__isset;
    return *this;
  }
  void ShowLocksResponse::printTo(std::ostream& out) const {
@@@ -19996,15 -19606,15 +20262,15 @@@ void swap(HeartbeatRequest &a, Heartbea
    swap(a.__isset, b.__isset);
  }
  
--HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other773) {
--  lockid = other773.lockid;
--  txnid = other773.txnid;
--  __isset = other773.__isset;
++HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other781) {
++  lockid = other781.lockid;
++  txnid = other781.txnid;
++  __isset = other781.__isset;
  }
--HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other774) {
--  lockid = other774.lockid;
--  txnid = other774.txnid;
--  __isset = other774.__isset;
++HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other782) {
++  lockid = other782.lockid;
++  txnid = other782.txnid;
++  __isset = other782.__isset;
    return *this;
  }
  void HeartbeatRequest::printTo(std::ostream& out) const {
@@@ -20107,13 -19717,13 +20373,13 @@@ void swap(HeartbeatTxnRangeRequest &a, 
    swap(a.max, b.max);
  }
  
--HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other775) {
--  min = other775.min;
--  max = other775.max;
++HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other783) {
++  min = other783.min;
++  max = other783.max;
  }
--HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other776) {
--  min = other776.min;
--  max = other776.max;
++HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other784) {
++  min = other784.min;
++  max = other784.max;
    return *this;
  }
  void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@@ -20164,15 -19774,15 +20430,15 @@@ uint32_t HeartbeatTxnRangeResponse::rea
          if (ftype == ::apache::thrift::protocol::T_SET) {
            {
              this->aborted.clear();
--            uint32_t _size777;
--            ::apache::thrift::protocol::TType _etype780;
--            xfer += iprot->readSetBegin(_etype780, _size777);
--            uint32_t _i781;
--            for (_i781 = 0; _i781 < _size777; ++_i781)
++            uint32_t _size785;
++            ::apache::thrift::protocol::TType _etype788;
++            xfer += iprot->readSetBegin(_etype788, _size785);
++            uint32_t _i789;
++            for (_i789 = 0; _i789 < _size785; ++_i789)
              {
--              int64_t _elem782;
--              xfer += iprot->readI64(_elem782);
--              this->aborted.insert(_elem782);
++              int64_t _elem790;
++              xfer += iprot->readI64(_elem790);
++              this->aborted.insert(_elem790);
              }
              xfer += iprot->readSetEnd();
            }
@@@ -20185,15 -19795,15 +20451,15 @@@
          if (ftype == ::apache::thrift::protocol::T_SET) {
            {
              this->nosuch.clear();
--            uint32_t _size783;
--            ::apache::thrift::protocol::TType _etype786;
--            xfer += iprot->readSetBegin(_etype786, _size783);
--            uint32_t _i787;
--            for (_i787 = 0; _i787 < _size783; ++_i787)
++            uint32_t _size791;
++            ::apache::thrift::protocol::TType _etype794;
++            xfer += iprot->readSetBegin(_etype794, _size791);
++            uint32_t _i795;
++            for (_i795 = 0; _i795 < _size791; ++_i795)
              {
--              int64_t _elem788;
--              xfer += iprot->readI64(_elem788);
--              this->nosuch.insert(_elem788);
++              int64_t _elem796;
++              xfer += iprot->readI64(_elem796);
++              this->nosuch.insert(_elem796);
              }
              xfer += iprot->readSetEnd();
            }
@@@ -20226,10 -19836,10 +20492,10 @@@ uint32_t HeartbeatTxnRangeResponse::wri
    xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1);
    {
      xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->aborted.size()));
--    std::set<int64_t> ::const_iterator _iter789;
--    for (_iter789 = this->aborted.begin(); _iter789 != this->aborted.end(); ++_iter789)
++    std::set<int64_t> ::const_iterator _iter797;
++    for (_iter797 = this->aborted.begin(); _iter797 != this->aborted.end(); ++_iter797)
      {
--      xfer += oprot->writeI64((*_iter789));
++      xfer += oprot->writeI64((*_iter797));
      }
      xfer += oprot->writeSetEnd();
    }
@@@ -20238,10 -19848,10 +20504,10 @@@
    xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2);
    {
      xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->nosuch.size()));
--    std::set<int64_t> ::const_iterator _iter790;
--    for (_iter790 = this->nosuch.begin(); _iter790 != this->nosuch.end(); ++_iter790)
++    std::set<int64_t> ::const_iterator _iter798;
++    for (_iter798 = this->nosuch.begin(); _iter798 != this->nosuch.end(); ++_iter798)
      {
--      xfer += oprot->writeI64((*_iter790));
++      xfer += oprot->writeI64((*_iter798));
      }
      xfer += oprot->writeSetEnd();
    }
@@@ -20258,13 -19868,13 +20524,13 @@@ void swap(HeartbeatTxnRangeResponse &a
    swap(a.nosuch, b.nosuch);
  }
  
--HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other791) {
--  aborted = other791.aborted;
--  nosuch = other791.nosuch;
++HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other799) {
++  aborted = other799.aborted;
++  nosuch = other799.nosuch;
  }
--HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other792) {
--  aborted = other792.aborted;
--  nosuch = other792.nosuch;
++HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other800) {
++  aborted = other800.aborted;
++  nosuch = other800.nosuch;
    return *this;
  }
  void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@@ -20357,9 -19967,9 +20623,9 @@@ uint32_t CompactionRequest::read(::apac
          break;
        case 4:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast793;
--          xfer += iprot->readI32(ecast793);
--          this->type = (CompactionType::type)ecast793;
++          int32_t ecast801;
++          xfer += iprot->readI32(ecast801);
++          this->type = (CompactionType::type)ecast801;
            isset_type = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -20377,17 -19987,17 +20643,17 @@@
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              this->properties.clear();
--            uint32_t _size794;
--            ::apache::thrift::protocol::TType _ktype795;
--            ::apache::thrift::protocol::TType _vtype796;
--            xfer += iprot->readMapBegin(_ktype795, _vtype796, _size794);
--            uint32_t _i798;
--            for (_i798 = 0; _i798 < _size794; ++_i798)
++            uint32_t _size802;
++            ::apache::thrift::protocol::TType _ktype803;
++            ::apache::thrift::protocol::TType _vtype804;
++            xfer += iprot->readMapBegin(_ktype803, _vtype804, _size802);
++            uint32_t _i806;
++            for (_i806 = 0; _i806 < _size802; ++_i806)
              {
--              std::string _key799;
--              xfer += iprot->readString(_key799);
--              std::string& _val800 = this->properties[_key799];
--              xfer += iprot->readString(_val800);
++              std::string _key807;
++              xfer += iprot->readString(_key807);
++              std::string& _val808 = this->properties[_key807];
++              xfer += iprot->readString(_val808);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -20445,11 -20055,11 +20711,11 @@@ uint32_t CompactionRequest::write(::apa
      xfer += oprot->writeFieldBegin("properties", ::apache::thrift::protocol::T_MAP, 6);
      {
        xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->properties.size()));
--      std::map<std::string, std::string> ::const_iterator _iter801;
--      for (_iter801 = this->properties.begin(); _iter801 != this->properties.end(); ++_iter801)
++      std::map<std::string, std::string> ::const_iterator _iter809;
++      for (_iter809 = this->properties.begin(); _iter809 != this->properties.end(); ++_iter809)
        {
--        xfer += oprot->writeString(_iter801->first);
--        xfer += oprot->writeString(_iter801->second);
++        xfer += oprot->writeString(_iter809->first);
++        xfer += oprot->writeString(_iter809->second);
        }
        xfer += oprot->writeMapEnd();
      }
@@@ -20471,23 -20081,23 +20737,23 @@@ void swap(CompactionRequest &a, Compact
    swap(a.__isset, b.__isset);
  }
  
--CompactionRequest::CompactionRequest(const CompactionRequest& other802) {
--  dbname = other802.dbname;
--  tablename = other802.tablename;
--  partitionname = other802.partitionname;
--  type = other802.type;
--  runas = other802.runas;
--  properties = other802.properties;
--  __isset = other802.__isset;
--}
--CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other803) {
--  dbname = other803.dbname;
--  tablename = other803.tablename;
--  partitionname = other803.partitionname;
--  type = other803.type;
--  runas = other803.runas;
--  properties = other803.properties;
--  __isset = other803.__isset;
++CompactionRequest::CompactionRequest(const CompactionRequest& other810) {
++  dbname = other810.dbname;
++  tablename = other810.tablename;
++  partitionname = other810.partitionname;
++  type = other810.type;
++  runas = other810.runas;
++  properties = other810.properties;
++  __isset = other810.__isset;
++}
++CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other811) {
++  dbname = other811.dbname;
++  tablename = other811.tablename;
++  partitionname = other811.partitionname;
++  type = other811.type;
++  runas = other811.runas;
++  properties = other811.properties;
++  __isset = other811.__isset;
    return *this;
  }
  void CompactionRequest::printTo(std::ostream& out) const {
@@@ -20614,15 -20224,15 +20880,15 @@@ void swap(CompactionResponse &a, Compac
    swap(a.accepted, b.accepted);
  }
  
--CompactionResponse::CompactionResponse(const CompactionResponse& other804) {
--  id = other804.id;
--  state = other804.state;
--  accepted = other804.accepted;
++CompactionResponse::CompactionResponse(const CompactionResponse& other812) {
++  id = other812.id;
++  state = other812.state;
++  accepted = other812.accepted;
  }
--CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other805) {
--  id = other805.id;
--  state = other805.state;
--  accepted = other805.accepted;
++CompactionResponse& CompactionResponse::operator=(const CompactionResponse& other813) {
++  id = other813.id;
++  state = other813.state;
++  accepted = other813.accepted;
    return *this;
  }
  void CompactionResponse::printTo(std::ostream& out) const {
@@@ -20683,11 -20293,11 +20949,11 @@@ void swap(ShowCompactRequest &a, ShowCo
    (void) b;
  }
  
--ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other806) {
--  (void) other806;
++ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other814) {
++  (void) other814;
  }
--ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other807) {
--  (void) other807;
++ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other815) {
++  (void) other815;
    return *this;
  }
  void ShowCompactRequest::printTo(std::ostream& out) const {
@@@ -20813,9 -20423,9 +21079,9 @@@ uint32_t ShowCompactResponseElement::re
          break;
        case 4:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast808;
--          xfer += iprot->readI32(ecast808);
--          this->type = (CompactionType::type)ecast808;
++          int32_t ecast816;
++          xfer += iprot->readI32(ecast816);
++          this->type = (CompactionType::type)ecast816;
            isset_type = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -21002,37 -20612,37 +21268,37 @@@ void swap(ShowCompactResponseElement &a
    swap(a.__isset, b.__isset);
  }
  
--ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other809) {
--  dbname = other809.dbname;
--  tablename = other809.tablename;
--  partitionname = other809.partitionname;
--  type = other809.type;
--  state = other809.state;
--  workerid = other809.workerid;
--  start = other809.start;
--  runAs = other809.runAs;
--  hightestTxnId = other809.hightestTxnId;
--  metaInfo = other809.metaInfo;
--  endTime = other809.endTime;
--  hadoopJobId = other809.hadoopJobId;
--  id = other809.id;
--  __isset = other809.__isset;
--}
--ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other810) {
--  dbname = other810.dbname;
--  tablename = other810.tablename;
--  partitionname = other810.partitionname;
--  type = other810.type;
--  state = other810.state;
--  workerid = other810.workerid;
--  start = other810.start;
--  runAs = other810.runAs;
--  hightestTxnId = other810.hightestTxnId;
--  metaInfo = other810.metaInfo;
--  endTime = other810.endTime;
--  hadoopJobId = other810.hadoopJobId;
--  id = other810.id;
--  __isset = other810.__isset;
++ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other817) {
++  dbname = other817.dbname;
++  tablename = other817.tablename;
++  partitionname = other817.partitionname;
++  type = other817.type;
++  state = other817.state;
++  workerid = other817.workerid;
++  start = other817.start;
++  runAs = other817.runAs;
++  hightestTxnId = other817.hightestTxnId;
++  metaInfo = other817.metaInfo;
++  endTime = other817.endTime;
++  hadoopJobId = other817.hadoopJobId;
++  id = other817.id;
++  __isset = other817.__isset;
++}
++ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other818) {
++  dbname = other818.dbname;
++  tablename = other818.tablename;
++  partitionname = other818.partitionname;
++  type = other818.type;
++  state = other818.state;
++  workerid = other818.workerid;
++  start = other818.start;
++  runAs = other818.runAs;
++  hightestTxnId = other818.hightestTxnId;
++  metaInfo = other818.metaInfo;
++  endTime = other818.endTime;
++  hadoopJobId = other818.hadoopJobId;
++  id = other818.id;
++  __isset = other818.__isset;
    return *this;
  }
  void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@@ -21089,14 -20699,14 +21355,14 @@@ uint32_t ShowCompactResponse::read(::ap
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->compacts.clear();
--            uint32_t _size811;
--            ::apache::thrift::protocol::TType _etype814;
--            xfer += iprot->readListBegin(_etype814, _size811);
--            this->compacts.resize(_size811);
--            uint32_t _i815;
--            for (_i815 = 0; _i815 < _size811; ++_i815)
++            uint32_t _size819;
++            ::apache::thrift::protocol::TType _etype822;
++            xfer += iprot->readListBegin(_etype822, _size819);
++            this->compacts.resize(_size819);
++            uint32_t _i823;
++            for (_i823 = 0; _i823 < _size819; ++_i823)
              {
--              xfer += this->compacts[_i815].read(iprot);
++              xfer += this->compacts[_i823].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -21127,10 -20737,10 +21393,10 @@@ uint32_t ShowCompactResponse::write(::a
    xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
--    std::vector<ShowCompactResponseElement> ::const_iterator _iter816;
--    for (_iter816 = this->compacts.begin(); _iter816 != this->compacts.end(); ++_iter816)
++    std::vector<ShowCompactResponseElement> ::const_iterator _iter824;
++    for (_iter824 = this->compacts.begin(); _iter824 != this->compacts.end(); ++_iter824)
      {
--      xfer += (*_iter816).write(oprot);
++      xfer += (*_iter824).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -21146,11 -20756,11 +21412,11 @@@ void swap(ShowCompactResponse &a, ShowC
    swap(a.compacts, b.compacts);
  }
  
--ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other817) {
--  compacts = other817.compacts;
++ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other825) {
++  compacts = other825.compacts;
  }
--ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other818) {
--  compacts = other818.compacts;
++ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other826) {
++  compacts = other826.compacts;
    return *this;
  }
  void ShowCompactResponse::printTo(std::ostream& out) const {
@@@ -21252,14 -20862,14 +21518,14 @@@ uint32_t AddDynamicPartitions::read(::a
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->partitionnames.clear();
--            uint32_t _size819;
--            ::apache::thrift::protocol::TType _etype822;
--            xfer += iprot->readListBegin(_etype822, _size819);
--            this->partitionnames.resize(_size819);
--            uint32_t _i823;
--            for (_i823 = 0; _i823 < _size819; ++_i823)
++            uint32_t _size827;
++            ::apache::thrift::protocol::TType _etype830;
++            xfer += iprot->readListBegin(_etype830, _size827);
++            this->partitionnames.resize(_size827);
++            uint32_t _i831;
++            for (_i831 = 0; _i831 < _size827; ++_i831)
              {
--              xfer += iprot->readString(this->partitionnames[_i823]);
++              xfer += iprot->readString(this->partitionnames[_i831]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -21270,9 -20880,9 +21536,9 @@@
          break;
        case 6:
          if (ftype == ::apache::thrift::protocol::T_I32) {
--          int32_t ecast824;
--          xfer += iprot->readI32(ecast824);
--          this->operationType = (DataOperationType::type)ecast824;
++          int32_t ecast832;
++          xfer += iprot->readI32(ecast832);
++          this->operationType = (DataOperationType::type)ecast832;
            this->__isset.operationType = true;
          } else {
            xfer += iprot->skip(ftype);
@@@ -21324,10 -20934,10 +21590,10 @@@ uint32_t AddDynamicPartitions::write(::
    xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
--    std::vector<std::string> ::const_iterator _iter825;
--    for (_iter825 = this->partitionnames.begin(); _iter825 != this->partitionnames.end(); ++_iter825)
++    std::vector<std::string> ::const_iterator _iter833;
++    for (_iter833 = this->partitionnames.begin(); _iter833 != this->partitionnames.end(); ++_iter833)
      {
--      xfer += oprot->writeString((*_iter825));
++      xfer += oprot->writeString((*_iter833));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -21354,23 -20964,23 +21620,23 @@@ void swap(AddDynamicPartitions &a, AddD
    swap(a.__isset, b.__isset);
  }
  
--AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other826) {
--  txnid = other826.txnid;
--  writeid = other826.writeid;
--  dbname = other826.dbname;
--  tablename = other826.tablename;
--  partitionnames = other826.partitionnames;
--  operationType = other826.operationType;
--  __isset = other826.__isset;
--}
--AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other827) {
--  txnid = other827.txnid;
--  writeid = other827.writeid;
--  dbname = other827.dbname;
--  tablename = other827.tablename;
--  partitionnames = other827.partitionnames;
--  operationType = other827.operationType;
--  __isset = other827.__isset;
++AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other834) {
++  txnid = other834.txnid;
++  writeid = other834.writeid;
++  dbname = other834.dbname;
++  tablename = other834.tablename;
++  partitionnames = other834.partitionnames;
++  operationType = other834.operationType;
++  __isset = other834.__isset;
++}
++AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other835) {
++  txnid = other835.txnid;
++  writeid = other835.writeid;
++  dbname = other835.dbname;
++  tablename = other835.tablename;
++  partitionnames = other835.partitionnames;
++  operationType = other835.operationType;
++  __isset = other835.__isset;
    return *this;
  }
  void AddDynamicPartitions::printTo(std::ostream& out) const {
@@@ -21553,23 -21163,23 +21819,23 @@@ void swap(BasicTxnInfo &a, BasicTxnInf
    swap(a.__isset, b.__isset);
  }
  
--BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other828) {
--  isnull = other828.isnull;
--  time = other828.time;
--  txnid = other828.txnid;
--  dbname = other828.dbname;
--  tablename = other828.tablename;
--  partitionname = other828.partitionname;
--  __isset = other828.__isset;
--}
--BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other829) {
--  isnull = other829.isnull;
--  time = other829.time;
--  txnid = other829.txnid;
--  dbname = other829.dbname;
--  tablename = other829.tablename;
--  partitionname = other829.partitionname;
--  __isset = other829.__isset;
++BasicTxnInfo::BasicTxnInfo(const BasicTxnInfo& other836) {
++  isnull = other836.isnull;
++  time = other836.time;
++  txnid = other836.txnid;
++  dbname = other836.dbname;
++  tablename = other836.tablename;
++  partitionname = other836.partitionname;
++  __isset = other836.__isset;
++}
++BasicTxnInfo& BasicTxnInfo::operator=(const BasicTxnInfo& other837) {
++  isnull = other837.isnull;
++  time = other837.time;
++  txnid = other837.txnid;
++  dbname = other837.dbname;
++  tablename = other837.tablename;
++  partitionname = other837.partitionname;
++  __isset = other837.__isset;
    return *this;
  }
  void BasicTxnInfo::printTo(std::ostream& out) const {
@@@ -21663,15 -21273,15 +21929,15 @@@ uint32_t CreationMetadata::read(::apach
          if (ftype == ::apache::thrift::protocol::T_SET) {
            {
              this->tablesUsed.clear();
--            uint32_t _size830;
--            ::apache::thrift::protocol::TType _etype833;
--            xfer += iprot->readSetBegin(_etype833, _size830);
--            uint32_t _i834;
--            for (_i834 = 0; _i834 < _size830; ++_i834)
++            uint32_t _size838;
++            ::apache::thrift::protocol::TType _etype841;
++            xfer += iprot->readSetBegin(_etype841, _size838);
++            uint32_t _i842;
++            for (_i842 = 0; _i842 < _size838; ++_i842)
              {
--              std::string _elem835;
--              xfer += iprot->readString(_elem835);
--              this->tablesUsed.insert(_elem835);
++              std::string _elem843;
++              xfer += iprot->readString(_elem843);
++              this->tablesUsed.insert(_elem843);
              }
              xfer += iprot->readSetEnd();
            }
@@@ -21728,10 -21338,10 +21994,10 @@@ uint32_t CreationMetadata::write(::apac
    xfer += oprot->writeFieldBegin("tablesUsed", ::apache::thrift::protocol::T_SET, 4);
    {
      xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tablesUsed.size()));
--    std::set<std::string> ::const_iterator _iter836;
--    for (_iter836 = this->tablesUsed.begin(); _iter836 != this->tablesUsed.end(); ++_iter836)
++    std::set<std::string> ::const_iterator _iter844;
++    for (_iter844 = this->tablesUsed.begin(); _iter844 != this->tablesUsed.end(); ++_iter844)
      {
--      xfer += oprot->writeString((*_iter836));
++      xfer += oprot->writeString((*_iter844));
      }
      xfer += oprot->writeSetEnd();
    }
@@@ -21757,21 -21367,21 +22023,21 @@@ void swap(CreationMetadata &a, Creation
    swap(a.__isset, b.__isset);
  }
  
--CreationMetadata::CreationMetadata(const CreationMetadata& other837) {
--  catName = other837.catName;
--  dbName = other837.dbName;
--  tblName = other837.tblName;
--  tablesUsed = other837.tablesUsed;
--  validTxnList = other837.validTxnList;
--  __isset = other837.__isset;
--}
--CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other838) {
--  catName = other838.catName;
--  dbName = other838.dbName;
--  tblName = other838.tblName;
--  tablesUsed = other838.tablesUsed;
--  validTxnList = other838.validTxnList;
--  __isset = other838.__isset;
++CreationMetadata::CreationMetadata(const CreationMetadata& other845) {
++  catName = other845.catName;
++  dbName = other845.dbName;
++  tblName = other845.tblName;
++  tablesUsed = other845.tablesUsed;
++  validTxnList = other845.validTxnList;
++  __isset = other845.__isset;
++}
++CreationMetadata& CreationMetadata::operator=(const CreationMetadata& other846) {
++  catName = other846.catName;
++  dbName = other846.dbName;
++  tblName = other846.tblName;
++  tablesUsed = other846.tablesUsed;
++  validTxnList = other846.validTxnList;
++  __isset = other846.__isset;
    return *this;
  }
  void CreationMetadata::printTo(std::ostream& out) const {
@@@ -21877,15 -21487,15 +22143,15 @@@ void swap(NotificationEventRequest &a, 
    swap(a.__isset, b.__isset);
  }
  
--NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other839) {
--  lastEvent = other839.lastEvent;
--  maxEvents = other839.maxEvents;
--  __isset = other839.__isset;
++NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other847) {
++  lastEvent = other847.lastEvent;
++  maxEvents = other847.maxEvents;
++  __isset = other847.__isset;
  }
--NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other840) {
--  lastEvent = other840.lastEvent;
--  maxEvents = other840.maxEvents;
--  __isset = other840.__isset;
++NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other848) {
++  lastEvent = other848.lastEvent;
++  maxEvents = other848.maxEvents;
++  __isset = other848.__isset;
    return *this;
  }
  void NotificationEventRequest::printTo(std::ostream& out) const {
@@@ -22105,27 -21715,27 +22371,27 @@@ void swap(NotificationEvent &a, Notific
    swap(a.__isset, b.__isset);
  }
  
--NotificationEvent::NotificationEvent(const NotificationEvent& other841) {
--  eventId = other841.eventId;
--  eventTime = other841.eventTime;
--  eventType = other841.eventType;
--  dbName = other841.dbName;
--  tableName = other841.tableName;
--  message = other841.message;
--  messageFormat = other841.messageFormat;
--  catName = other841.catName;
--  __isset = other841.__isset;
--}
--NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other842) {
--  eventId = other842.eventId;
--  eventTime = other842.eventTime;
--  eventType = other842.eventType;
--  dbName = other842.dbName;
--  tableName = other842.tableName;
--  message = other842.message;
--  messageFormat = other842.messageFormat;
--  catName = other842.catName;
--  __isset = other842.__isset;
++NotificationEvent::NotificationEvent(const NotificationEvent& other849) {
++  eventId = other849.eventId;
++  eventTime = other849.eventTime;
++  eventType = other849.eventType;
++  dbName = other849.dbName;
++  tableName = other849.tableName;
++  message = other849.message;
++  messageFormat = other849.messageFormat;
++  catName = other849.catName;
++  __isset = other849.__isset;
++}
++NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other850) {
++  eventId = other850.eventId;
++  eventTime = other850.eventTime;
++  eventType = other850.eventType;
++  dbName = other850.dbName;
++  tableName = other850.tableName;
++  message = other850.message;
++  messageFormat = other850.messageFormat;
++  catName = other850.catName;
++  __isset = other850.__isset;
    return *this;
  }
  void NotificationEvent::printTo(std::ostream& out) const {
@@@ -22177,14 -21787,14 +22443,14 @@@ uint32_t NotificationEventResponse::rea
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->events.clear();
--            uint32_t _size843;
--            ::apache::thrift::protocol::TType _etype846;
--            xfer += iprot->readListBegin(_etype846, _size843);
--            this->events.resize(_size843);
--            uint32_t _i847;
--            for (_i847 = 0; _i847 < _size843; ++_i847)
++            uint32_t _size851;
++            ::apache::thrift::protocol::TType _etype854;
++            xfer += iprot->readListBegin(_etype854, _size851);
++            this->events.resize(_size851);
++            uint32_t _i855;
++            for (_i855 = 0; _i855 < _size851; ++_i855)
              {
--              xfer += this->events[_i847].read(iprot);
++              xfer += this->events[_i855].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -22215,10 -21825,10 +22481,10 @@@ uint32_t NotificationEventResponse::wri
    xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
--    std::vector<NotificationEvent> ::const_iterator _iter848;
--    for (_iter848 = this->events.begin(); _iter848 != this->events.end(); ++_iter848)
++    std::vector<NotificationEvent> ::const_iterator _iter856;
++    for (_iter856 = this->events.begin(); _iter856 != this->events.end(); ++_iter856)
      {
--      xfer += (*_iter848).write(oprot);
++      xfer += (*_iter856).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -22234,11 -21844,11 +22500,11 @@@ void swap(NotificationEventResponse &a
    swap(a.events, b.events);
  }
  
--NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other849) {
--  events = other849.events;
++NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other857) {
++  events = other857.events;
  }
--NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other850) {
--  events = other850.events;
++NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other858) {
++  events = other858.events;
    return *this;
  }
  void NotificationEventResponse::printTo(std::ostream& out) const {
@@@ -22320,11 -21930,11 +22586,11 @@@ void swap(CurrentNotificationEventId &a
    swap(a.eventId, b.eventId);
  }
  
--CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other851) {
--  eventId = other851.eventId;
++CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other859) {
++  eventId = other859.eventId;
  }
--CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other852) {
--  eventId = other852.eventId;
++CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other860) {
++  eventId = other860.eventId;
    return *this;
  }
  void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@@ -22446,17 -22056,17 +22712,17 @@@ void swap(NotificationEventsCountReques
    swap(a.__isset, b.__isset);
  }
  
--NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other853) {
--  fromEventId = other853.fromEventId;
--  dbName = other853.dbName;
--  catName = other853.catName;
--  __isset = other853.__isset;
++NotificationEventsCountRequest::NotificationEventsCountRequest(const NotificationEventsCountRequest& other861) {
++  fromEventId = other861.fromEventId;
++  dbName = other861.dbName;
++  catName = other861.catName;
++  __isset = other861.__isset;
  }
--NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other854) {
--  fromEventId = other854.fromEventId;
--  dbName = other854.dbName;
--  catName = other854.catName;
--  __isset = other854.__isset;
++NotificationEventsCountRequest& NotificationEventsCountRequest::operator=(const NotificationEventsCountRequest& other862) {
++  fromEventId = other862.fromEventId;
++  dbName = other862.dbName;
++  catName = other862.catName;
++  __isset = other862.__isset;
    return *this;
  }
  void NotificationEventsCountRequest::printTo(std::ostream& out) const {
@@@ -22540,11 -22150,11 +22806,11 @@@ void swap(NotificationEventsCountRespon
    swap(a.eventsCount, b.eventsCount);
  }
  
--NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other855) {
--  eventsCount = other855.eventsCount;
++NotificationEventsCountResponse::NotificationEventsCountResponse(const NotificationEventsCountResponse& other863) {
++  eventsCount = other863.eventsCount;
  }
--NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other856) {
--  eventsCount = other856.eventsCount;
++NotificationEventsCountResponse& NotificationEventsCountResponse::operator=(const NotificationEventsCountResponse& other864) {
++  eventsCount = other864.eventsCount;
    return *this;
  }
  void NotificationEventsCountResponse::printTo(std::ostream& out) const {
@@@ -22607,14 -22222,14 +22878,14 @@@ uint32_t InsertEventRequestData::read(:
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->filesAdded.clear();
--            uint32_t _size857;
--            ::apache::thrift::protocol::TType _etype860;
--            xfer += iprot->readListBegin(_etype860, _size857);
--            this->filesAdded.resize(_size857);
--            uint32_t _i861;
--            for (_i861 = 0; _i861 < _size857; ++_i861)
++            uint32_t _size865;
++            ::apache::thrift::protocol::TType _etype868;
++            xfer += iprot->readListBegin(_etype868, _size865);
++            this->filesAdded.resize(_size865);
++            uint32_t _i869;
++            for (_i869 = 0; _i869 < _size865; ++_i869)
              {
--              xfer += iprot->readString(this->filesAdded[_i861]);
++              xfer += iprot->readString(this->filesAdded[_i869]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -22627,14 -22242,14 +22898,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->filesAddedChecksum.clear();
--            uint32_t _size862;
--            ::apache::thrift::protocol::TType _etype865;
--            xfer += iprot->readListBegin(_etype865, _size862);
--            this->filesAddedChecksum.resize(_size862);
--            uint32_t _i866;
--            for (_i866 = 0; _i866 < _size862; ++_i866)
++            uint32_t _size870;
++            ::apache::thrift::protocol::TType _etype873;
++            xfer += iprot->readListBegin(_etype873, _size870);
++            this->filesAddedChecksum.resize(_size870);
++            uint32_t _i874;
++            for (_i874 = 0; _i874 < _size870; ++_i874)
              {
--              xfer += iprot->readString(this->filesAddedChecksum[_i866]);
++              xfer += iprot->readString(this->filesAddedChecksum[_i874]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -22643,6 -22258,26 +22914,26 @@@
            xfer += iprot->skip(ftype);
          }
          break;
+       case 4:
+         if (ftype == ::apache::thrift::protocol::T_LIST) {
+           {
+             this->subDirectoryList.clear();
 -            uint32_t _size867;
 -            ::apache::thrift::protocol::TType _etype870;
 -            xfer += iprot->readListBegin(_etype870, _size867);
 -            this->subDirectoryList.resize(_size867);
 -            uint32_t _i871;
 -            for (_i871 = 0; _i871 < _size867; ++_i871)
++            uint32_t _size875;
++            ::apache::thrift::protocol::TType _etype878;
++            xfer += iprot->readListBegin(_etype878, _size875);
++            this->subDirectoryList.resize(_size875);
++            uint32_t _i879;
++            for (_i879 = 0; _i879 < _size875; ++_i879)
+             {
 -              xfer += iprot->readString(this->subDirectoryList[_i871]);
++              xfer += iprot->readString(this->subDirectoryList[_i879]);
+             }
+             xfer += iprot->readListEnd();
+           }
+           this->__isset.subDirectoryList = true;
+         } else {
+           xfer += iprot->skip(ftype);
+         }
+         break;
        default:
          xfer += iprot->skip(ftype);
          break;
@@@ -22670,10 -22305,10 +22961,10 @@@ uint32_t InsertEventRequestData::write(
    xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-     std::vector<std::string> ::const_iterator _iter867;
-     for (_iter867 = this->filesAdded.begin(); _iter867 != this->filesAdded.end(); ++_iter867)
 -    std::vector<std::string> ::const_iterator _iter872;
 -    for (_iter872 = this->filesAdded.begin(); _iter872 != this->filesAdded.end(); ++_iter872)
++    std::vector<std::string> ::const_iterator _iter880;
++    for (_iter880 = this->filesAdded.begin(); _iter880 != this->filesAdded.end(); ++_iter880)
      {
-       xfer += oprot->writeString((*_iter867));
 -      xfer += oprot->writeString((*_iter872));
++      xfer += oprot->writeString((*_iter880));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -22683,10 -22318,23 +22974,23 @@@
      xfer += oprot->writeFieldBegin("filesAddedChecksum", ::apache::thrift::protocol::T_LIST, 3);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAddedChecksum.size()));
-       std::vector<std::string> ::const_iterator _iter868;
-       for (_iter868 = this->filesAddedChecksum.begin(); _iter868 != this->filesAddedChecksum.end(); ++_iter868)
 -      std::vector<std::string> ::const_iterator _iter873;
 -      for (_iter873 = this->filesAddedChecksum.begin(); _iter873 != this->filesAddedChecksum.end(); ++_iter873)
++      std::vector<std::string> ::const_iterator _iter881;
++      for (_iter881 = this->filesAddedChecksum.begin(); _iter881 != this->filesAddedChecksum.end(); ++_iter881)
+       {
 -        xfer += oprot->writeString((*_iter873));
++        xfer += oprot->writeString((*_iter881));
+       }
+       xfer += oprot->writeListEnd();
+     }
+     xfer += oprot->writeFieldEnd();
+   }
+   if (this->__isset.subDirectoryList) {
+     xfer += oprot->writeFieldBegin("subDirectoryList", ::apache::thrift::protocol::T_LIST, 4);
+     {
+       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->subDirectoryList.size()));
 -      std:

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------


[45/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 7a81dfb,a816ae7..ab2c837
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@@ -2334,14 -2334,14 +2334,14 @@@ uint32_t ThriftHiveMetastore_get_databa
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1221;
-             ::apache::thrift::protocol::TType _etype1224;
-             xfer += iprot->readListBegin(_etype1224, _size1221);
-             this->success.resize(_size1221);
-             uint32_t _i1225;
-             for (_i1225 = 0; _i1225 < _size1221; ++_i1225)
 -            uint32_t _size1226;
 -            ::apache::thrift::protocol::TType _etype1229;
 -            xfer += iprot->readListBegin(_etype1229, _size1226);
 -            this->success.resize(_size1226);
 -            uint32_t _i1230;
 -            for (_i1230 = 0; _i1230 < _size1226; ++_i1230)
++            uint32_t _size1245;
++            ::apache::thrift::protocol::TType _etype1248;
++            xfer += iprot->readListBegin(_etype1248, _size1245);
++            this->success.resize(_size1245);
++            uint32_t _i1249;
++            for (_i1249 = 0; _i1249 < _size1245; ++_i1249)
              {
-               xfer += iprot->readString(this->success[_i1225]);
 -              xfer += iprot->readString(this->success[_i1230]);
++              xfer += iprot->readString(this->success[_i1249]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -2380,10 -2380,10 +2380,10 @@@ uint32_t ThriftHiveMetastore_get_databa
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-       std::vector<std::string> ::const_iterator _iter1226;
-       for (_iter1226 = this->success.begin(); _iter1226 != this->success.end(); ++_iter1226)
 -      std::vector<std::string> ::const_iterator _iter1231;
 -      for (_iter1231 = this->success.begin(); _iter1231 != this->success.end(); ++_iter1231)
++      std::vector<std::string> ::const_iterator _iter1250;
++      for (_iter1250 = this->success.begin(); _iter1250 != this->success.end(); ++_iter1250)
        {
-         xfer += oprot->writeString((*_iter1226));
 -        xfer += oprot->writeString((*_iter1231));
++        xfer += oprot->writeString((*_iter1250));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -2428,14 -2428,14 +2428,14 @@@ uint32_t ThriftHiveMetastore_get_databa
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1227;
-             ::apache::thrift::protocol::TType _etype1230;
-             xfer += iprot->readListBegin(_etype1230, _size1227);
-             (*(this->success)).resize(_size1227);
-             uint32_t _i1231;
-             for (_i1231 = 0; _i1231 < _size1227; ++_i1231)
 -            uint32_t _size1232;
 -            ::apache::thrift::protocol::TType _etype1235;
 -            xfer += iprot->readListBegin(_etype1235, _size1232);
 -            (*(this->success)).resize(_size1232);
 -            uint32_t _i1236;
 -            for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
++            uint32_t _size1251;
++            ::apache::thrift::protocol::TType _etype1254;
++            xfer += iprot->readListBegin(_etype1254, _size1251);
++            (*(this->success)).resize(_size1251);
++            uint32_t _i1255;
++            for (_i1255 = 0; _i1255 < _size1251; ++_i1255)
              {
-               xfer += iprot->readString((*(this->success))[_i1231]);
 -              xfer += iprot->readString((*(this->success))[_i1236]);
++              xfer += iprot->readString((*(this->success))[_i1255]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -2552,14 -2552,14 +2552,14 @@@ uint32_t ThriftHiveMetastore_get_all_da
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1232;
-             ::apache::thrift::protocol::TType _etype1235;
-             xfer += iprot->readListBegin(_etype1235, _size1232);
-             this->success.resize(_size1232);
-             uint32_t _i1236;
-             for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
 -            uint32_t _size1237;
 -            ::apache::thrift::protocol::TType _etype1240;
 -            xfer += iprot->readListBegin(_etype1240, _size1237);
 -            this->success.resize(_size1237);
 -            uint32_t _i1241;
 -            for (_i1241 = 0; _i1241 < _size1237; ++_i1241)
++            uint32_t _size1256;
++            ::apache::thrift::protocol::TType _etype1259;
++            xfer += iprot->readListBegin(_etype1259, _size1256);
++            this->success.resize(_size1256);
++            uint32_t _i1260;
++            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
              {
-               xfer += iprot->readString(this->success[_i1236]);
 -              xfer += iprot->readString(this->success[_i1241]);
++              xfer += iprot->readString(this->success[_i1260]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -2598,10 -2598,10 +2598,10 @@@ uint32_t ThriftHiveMetastore_get_all_da
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-       std::vector<std::string> ::const_iterator _iter1237;
-       for (_iter1237 = this->success.begin(); _iter1237 != this->success.end(); ++_iter1237)
 -      std::vector<std::string> ::const_iterator _iter1242;
 -      for (_iter1242 = this->success.begin(); _iter1242 != this->success.end(); ++_iter1242)
++      std::vector<std::string> ::const_iterator _iter1261;
++      for (_iter1261 = this->success.begin(); _iter1261 != this->success.end(); ++_iter1261)
        {
-         xfer += oprot->writeString((*_iter1237));
 -        xfer += oprot->writeString((*_iter1242));
++        xfer += oprot->writeString((*_iter1261));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -2646,14 -2646,14 +2646,14 @@@ uint32_t ThriftHiveMetastore_get_all_da
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1238;
-             ::apache::thrift::protocol::TType _etype1241;
-             xfer += iprot->readListBegin(_etype1241, _size1238);
-             (*(this->success)).resize(_size1238);
-             uint32_t _i1242;
-             for (_i1242 = 0; _i1242 < _size1238; ++_i1242)
 -            uint32_t _size1243;
 -            ::apache::thrift::protocol::TType _etype1246;
 -            xfer += iprot->readListBegin(_etype1246, _size1243);
 -            (*(this->success)).resize(_size1243);
 -            uint32_t _i1247;
 -            for (_i1247 = 0; _i1247 < _size1243; ++_i1247)
++            uint32_t _size1262;
++            ::apache::thrift::protocol::TType _etype1265;
++            xfer += iprot->readListBegin(_etype1265, _size1262);
++            (*(this->success)).resize(_size1262);
++            uint32_t _i1266;
++            for (_i1266 = 0; _i1266 < _size1262; ++_i1266)
              {
-               xfer += iprot->readString((*(this->success))[_i1242]);
 -              xfer += iprot->readString((*(this->success))[_i1247]);
++              xfer += iprot->readString((*(this->success))[_i1266]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -3715,17 -3715,17 +3715,17 @@@ uint32_t ThriftHiveMetastore_get_type_a
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              this->success.clear();
-             uint32_t _size1243;
-             ::apache::thrift::protocol::TType _ktype1244;
-             ::apache::thrift::protocol::TType _vtype1245;
-             xfer += iprot->readMapBegin(_ktype1244, _vtype1245, _size1243);
-             uint32_t _i1247;
-             for (_i1247 = 0; _i1247 < _size1243; ++_i1247)
 -            uint32_t _size1248;
 -            ::apache::thrift::protocol::TType _ktype1249;
 -            ::apache::thrift::protocol::TType _vtype1250;
 -            xfer += iprot->readMapBegin(_ktype1249, _vtype1250, _size1248);
 -            uint32_t _i1252;
 -            for (_i1252 = 0; _i1252 < _size1248; ++_i1252)
++            uint32_t _size1267;
++            ::apache::thrift::protocol::TType _ktype1268;
++            ::apache::thrift::protocol::TType _vtype1269;
++            xfer += iprot->readMapBegin(_ktype1268, _vtype1269, _size1267);
++            uint32_t _i1271;
++            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
              {
-               std::string _key1248;
-               xfer += iprot->readString(_key1248);
-               Type& _val1249 = this->success[_key1248];
-               xfer += _val1249.read(iprot);
 -              std::string _key1253;
 -              xfer += iprot->readString(_key1253);
 -              Type& _val1254 = this->success[_key1253];
 -              xfer += _val1254.read(iprot);
++              std::string _key1272;
++              xfer += iprot->readString(_key1272);
++              Type& _val1273 = this->success[_key1272];
++              xfer += _val1273.read(iprot);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -3764,11 -3764,11 +3764,11 @@@ uint32_t ThriftHiveMetastore_get_type_a
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
      {
        xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::map<std::string, Type> ::const_iterator _iter1250;
-       for (_iter1250 = this->success.begin(); _iter1250 != this->success.end(); ++_iter1250)
 -      std::map<std::string, Type> ::const_iterator _iter1255;
 -      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
++      std::map<std::string, Type> ::const_iterator _iter1274;
++      for (_iter1274 = this->success.begin(); _iter1274 != this->success.end(); ++_iter1274)
        {
-         xfer += oprot->writeString(_iter1250->first);
-         xfer += _iter1250->second.write(oprot);
 -        xfer += oprot->writeString(_iter1255->first);
 -        xfer += _iter1255->second.write(oprot);
++        xfer += oprot->writeString(_iter1274->first);
++        xfer += _iter1274->second.write(oprot);
        }
        xfer += oprot->writeMapEnd();
      }
@@@ -3813,17 -3813,17 +3813,17 @@@ uint32_t ThriftHiveMetastore_get_type_a
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              (*(this->success)).clear();
-             uint32_t _size1251;
-             ::apache::thrift::protocol::TType _ktype1252;
-             ::apache::thrift::protocol::TType _vtype1253;
-             xfer += iprot->readMapBegin(_ktype1252, _vtype1253, _size1251);
-             uint32_t _i1255;
-             for (_i1255 = 0; _i1255 < _size1251; ++_i1255)
 -            uint32_t _size1256;
 -            ::apache::thrift::protocol::TType _ktype1257;
 -            ::apache::thrift::protocol::TType _vtype1258;
 -            xfer += iprot->readMapBegin(_ktype1257, _vtype1258, _size1256);
 -            uint32_t _i1260;
 -            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
++            uint32_t _size1275;
++            ::apache::thrift::protocol::TType _ktype1276;
++            ::apache::thrift::protocol::TType _vtype1277;
++            xfer += iprot->readMapBegin(_ktype1276, _vtype1277, _size1275);
++            uint32_t _i1279;
++            for (_i1279 = 0; _i1279 < _size1275; ++_i1279)
              {
-               std::string _key1256;
-               xfer += iprot->readString(_key1256);
-               Type& _val1257 = (*(this->success))[_key1256];
-               xfer += _val1257.read(iprot);
 -              std::string _key1261;
 -              xfer += iprot->readString(_key1261);
 -              Type& _val1262 = (*(this->success))[_key1261];
 -              xfer += _val1262.read(iprot);
++              std::string _key1280;
++              xfer += iprot->readString(_key1280);
++              Type& _val1281 = (*(this->success))[_key1280];
++              xfer += _val1281.read(iprot);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -3977,14 -3977,14 +3977,14 @@@ uint32_t ThriftHiveMetastore_get_fields
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1258;
-             ::apache::thrift::protocol::TType _etype1261;
-             xfer += iprot->readListBegin(_etype1261, _size1258);
-             this->success.resize(_size1258);
-             uint32_t _i1262;
-             for (_i1262 = 0; _i1262 < _size1258; ++_i1262)
 -            uint32_t _size1263;
 -            ::apache::thrift::protocol::TType _etype1266;
 -            xfer += iprot->readListBegin(_etype1266, _size1263);
 -            this->success.resize(_size1263);
 -            uint32_t _i1267;
 -            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
++            uint32_t _size1282;
++            ::apache::thrift::protocol::TType _etype1285;
++            xfer += iprot->readListBegin(_etype1285, _size1282);
++            this->success.resize(_size1282);
++            uint32_t _i1286;
++            for (_i1286 = 0; _i1286 < _size1282; ++_i1286)
              {
-               xfer += this->success[_i1262].read(iprot);
 -              xfer += this->success[_i1267].read(iprot);
++              xfer += this->success[_i1286].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4039,10 -4039,10 +4039,10 @@@ uint32_t ThriftHiveMetastore_get_fields
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<FieldSchema> ::const_iterator _iter1263;
-       for (_iter1263 = this->success.begin(); _iter1263 != this->success.end(); ++_iter1263)
 -      std::vector<FieldSchema> ::const_iterator _iter1268;
 -      for (_iter1268 = this->success.begin(); _iter1268 != this->success.end(); ++_iter1268)
++      std::vector<FieldSchema> ::const_iterator _iter1287;
++      for (_iter1287 = this->success.begin(); _iter1287 != this->success.end(); ++_iter1287)
        {
-         xfer += (*_iter1263).write(oprot);
 -        xfer += (*_iter1268).write(oprot);
++        xfer += (*_iter1287).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -4095,14 -4095,14 +4095,14 @@@ uint32_t ThriftHiveMetastore_get_fields
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1264;
-             ::apache::thrift::protocol::TType _etype1267;
-             xfer += iprot->readListBegin(_etype1267, _size1264);
-             (*(this->success)).resize(_size1264);
-             uint32_t _i1268;
-             for (_i1268 = 0; _i1268 < _size1264; ++_i1268)
 -            uint32_t _size1269;
 -            ::apache::thrift::protocol::TType _etype1272;
 -            xfer += iprot->readListBegin(_etype1272, _size1269);
 -            (*(this->success)).resize(_size1269);
 -            uint32_t _i1273;
 -            for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
++            uint32_t _size1288;
++            ::apache::thrift::protocol::TType _etype1291;
++            xfer += iprot->readListBegin(_etype1291, _size1288);
++            (*(this->success)).resize(_size1288);
++            uint32_t _i1292;
++            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
              {
-               xfer += (*(this->success))[_i1268].read(iprot);
 -              xfer += (*(this->success))[_i1273].read(iprot);
++              xfer += (*(this->success))[_i1292].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4288,14 -4288,14 +4288,14 @@@ uint32_t ThriftHiveMetastore_get_fields
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1269;
-             ::apache::thrift::protocol::TType _etype1272;
-             xfer += iprot->readListBegin(_etype1272, _size1269);
-             this->success.resize(_size1269);
-             uint32_t _i1273;
-             for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
 -            uint32_t _size1274;
 -            ::apache::thrift::protocol::TType _etype1277;
 -            xfer += iprot->readListBegin(_etype1277, _size1274);
 -            this->success.resize(_size1274);
 -            uint32_t _i1278;
 -            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
++            uint32_t _size1293;
++            ::apache::thrift::protocol::TType _etype1296;
++            xfer += iprot->readListBegin(_etype1296, _size1293);
++            this->success.resize(_size1293);
++            uint32_t _i1297;
++            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
              {
-               xfer += this->success[_i1273].read(iprot);
 -              xfer += this->success[_i1278].read(iprot);
++              xfer += this->success[_i1297].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4350,10 -4350,10 +4350,10 @@@ uint32_t ThriftHiveMetastore_get_fields
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<FieldSchema> ::const_iterator _iter1274;
-       for (_iter1274 = this->success.begin(); _iter1274 != this->success.end(); ++_iter1274)
 -      std::vector<FieldSchema> ::const_iterator _iter1279;
 -      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
++      std::vector<FieldSchema> ::const_iterator _iter1298;
++      for (_iter1298 = this->success.begin(); _iter1298 != this->success.end(); ++_iter1298)
        {
-         xfer += (*_iter1274).write(oprot);
 -        xfer += (*_iter1279).write(oprot);
++        xfer += (*_iter1298).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -4406,14 -4406,14 +4406,14 @@@ uint32_t ThriftHiveMetastore_get_fields
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1275;
-             ::apache::thrift::protocol::TType _etype1278;
-             xfer += iprot->readListBegin(_etype1278, _size1275);
-             (*(this->success)).resize(_size1275);
-             uint32_t _i1279;
-             for (_i1279 = 0; _i1279 < _size1275; ++_i1279)
 -            uint32_t _size1280;
 -            ::apache::thrift::protocol::TType _etype1283;
 -            xfer += iprot->readListBegin(_etype1283, _size1280);
 -            (*(this->success)).resize(_size1280);
 -            uint32_t _i1284;
 -            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
++            uint32_t _size1299;
++            ::apache::thrift::protocol::TType _etype1302;
++            xfer += iprot->readListBegin(_etype1302, _size1299);
++            (*(this->success)).resize(_size1299);
++            uint32_t _i1303;
++            for (_i1303 = 0; _i1303 < _size1299; ++_i1303)
              {
-               xfer += (*(this->success))[_i1279].read(iprot);
 -              xfer += (*(this->success))[_i1284].read(iprot);
++              xfer += (*(this->success))[_i1303].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4583,14 -4583,14 +4583,14 @@@ uint32_t ThriftHiveMetastore_get_schema
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1280;
-             ::apache::thrift::protocol::TType _etype1283;
-             xfer += iprot->readListBegin(_etype1283, _size1280);
-             this->success.resize(_size1280);
-             uint32_t _i1284;
-             for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
 -            uint32_t _size1285;
 -            ::apache::thrift::protocol::TType _etype1288;
 -            xfer += iprot->readListBegin(_etype1288, _size1285);
 -            this->success.resize(_size1285);
 -            uint32_t _i1289;
 -            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
++            uint32_t _size1304;
++            ::apache::thrift::protocol::TType _etype1307;
++            xfer += iprot->readListBegin(_etype1307, _size1304);
++            this->success.resize(_size1304);
++            uint32_t _i1308;
++            for (_i1308 = 0; _i1308 < _size1304; ++_i1308)
              {
-               xfer += this->success[_i1284].read(iprot);
 -              xfer += this->success[_i1289].read(iprot);
++              xfer += this->success[_i1308].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4645,10 -4645,10 +4645,10 @@@ uint32_t ThriftHiveMetastore_get_schema
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<FieldSchema> ::const_iterator _iter1285;
-       for (_iter1285 = this->success.begin(); _iter1285 != this->success.end(); ++_iter1285)
 -      std::vector<FieldSchema> ::const_iterator _iter1290;
 -      for (_iter1290 = this->success.begin(); _iter1290 != this->success.end(); ++_iter1290)
++      std::vector<FieldSchema> ::const_iterator _iter1309;
++      for (_iter1309 = this->success.begin(); _iter1309 != this->success.end(); ++_iter1309)
        {
-         xfer += (*_iter1285).write(oprot);
 -        xfer += (*_iter1290).write(oprot);
++        xfer += (*_iter1309).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -4701,14 -4701,14 +4701,14 @@@ uint32_t ThriftHiveMetastore_get_schema
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1286;
-             ::apache::thrift::protocol::TType _etype1289;
-             xfer += iprot->readListBegin(_etype1289, _size1286);
-             (*(this->success)).resize(_size1286);
-             uint32_t _i1290;
-             for (_i1290 = 0; _i1290 < _size1286; ++_i1290)
 -            uint32_t _size1291;
 -            ::apache::thrift::protocol::TType _etype1294;
 -            xfer += iprot->readListBegin(_etype1294, _size1291);
 -            (*(this->success)).resize(_size1291);
 -            uint32_t _i1295;
 -            for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
++            uint32_t _size1310;
++            ::apache::thrift::protocol::TType _etype1313;
++            xfer += iprot->readListBegin(_etype1313, _size1310);
++            (*(this->success)).resize(_size1310);
++            uint32_t _i1314;
++            for (_i1314 = 0; _i1314 < _size1310; ++_i1314)
              {
-               xfer += (*(this->success))[_i1290].read(iprot);
 -              xfer += (*(this->success))[_i1295].read(iprot);
++              xfer += (*(this->success))[_i1314].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4894,14 -4894,14 +4894,14 @@@ uint32_t ThriftHiveMetastore_get_schema
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1291;
-             ::apache::thrift::protocol::TType _etype1294;
-             xfer += iprot->readListBegin(_etype1294, _size1291);
-             this->success.resize(_size1291);
-             uint32_t _i1295;
-             for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
 -            uint32_t _size1296;
 -            ::apache::thrift::protocol::TType _etype1299;
 -            xfer += iprot->readListBegin(_etype1299, _size1296);
 -            this->success.resize(_size1296);
 -            uint32_t _i1300;
 -            for (_i1300 = 0; _i1300 < _size1296; ++_i1300)
++            uint32_t _size1315;
++            ::apache::thrift::protocol::TType _etype1318;
++            xfer += iprot->readListBegin(_etype1318, _size1315);
++            this->success.resize(_size1315);
++            uint32_t _i1319;
++            for (_i1319 = 0; _i1319 < _size1315; ++_i1319)
              {
-               xfer += this->success[_i1295].read(iprot);
 -              xfer += this->success[_i1300].read(iprot);
++              xfer += this->success[_i1319].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -4956,10 -4956,10 +4956,10 @@@ uint32_t ThriftHiveMetastore_get_schema
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<FieldSchema> ::const_iterator _iter1296;
-       for (_iter1296 = this->success.begin(); _iter1296 != this->success.end(); ++_iter1296)
 -      std::vector<FieldSchema> ::const_iterator _iter1301;
 -      for (_iter1301 = this->success.begin(); _iter1301 != this->success.end(); ++_iter1301)
++      std::vector<FieldSchema> ::const_iterator _iter1320;
++      for (_iter1320 = this->success.begin(); _iter1320 != this->success.end(); ++_iter1320)
        {
-         xfer += (*_iter1296).write(oprot);
 -        xfer += (*_iter1301).write(oprot);
++        xfer += (*_iter1320).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -5012,14 -5012,14 +5012,14 @@@ uint32_t ThriftHiveMetastore_get_schema
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1297;
-             ::apache::thrift::protocol::TType _etype1300;
-             xfer += iprot->readListBegin(_etype1300, _size1297);
-             (*(this->success)).resize(_size1297);
-             uint32_t _i1301;
-             for (_i1301 = 0; _i1301 < _size1297; ++_i1301)
 -            uint32_t _size1302;
 -            ::apache::thrift::protocol::TType _etype1305;
 -            xfer += iprot->readListBegin(_etype1305, _size1302);
 -            (*(this->success)).resize(_size1302);
 -            uint32_t _i1306;
 -            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
++            uint32_t _size1321;
++            ::apache::thrift::protocol::TType _etype1324;
++            xfer += iprot->readListBegin(_etype1324, _size1321);
++            (*(this->success)).resize(_size1321);
++            uint32_t _i1325;
++            for (_i1325 = 0; _i1325 < _size1321; ++_i1325)
              {
-               xfer += (*(this->success))[_i1301].read(iprot);
 -              xfer += (*(this->success))[_i1306].read(iprot);
++              xfer += (*(this->success))[_i1325].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5612,14 -5612,14 +5612,14 @@@ uint32_t ThriftHiveMetastore_create_tab
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->primaryKeys.clear();
-             uint32_t _size1302;
-             ::apache::thrift::protocol::TType _etype1305;
-             xfer += iprot->readListBegin(_etype1305, _size1302);
-             this->primaryKeys.resize(_size1302);
-             uint32_t _i1306;
-             for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
 -            uint32_t _size1307;
 -            ::apache::thrift::protocol::TType _etype1310;
 -            xfer += iprot->readListBegin(_etype1310, _size1307);
 -            this->primaryKeys.resize(_size1307);
 -            uint32_t _i1311;
 -            for (_i1311 = 0; _i1311 < _size1307; ++_i1311)
++            uint32_t _size1326;
++            ::apache::thrift::protocol::TType _etype1329;
++            xfer += iprot->readListBegin(_etype1329, _size1326);
++            this->primaryKeys.resize(_size1326);
++            uint32_t _i1330;
++            for (_i1330 = 0; _i1330 < _size1326; ++_i1330)
              {
-               xfer += this->primaryKeys[_i1306].read(iprot);
 -              xfer += this->primaryKeys[_i1311].read(iprot);
++              xfer += this->primaryKeys[_i1330].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5632,14 -5632,14 +5632,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->foreignKeys.clear();
-             uint32_t _size1307;
-             ::apache::thrift::protocol::TType _etype1310;
-             xfer += iprot->readListBegin(_etype1310, _size1307);
-             this->foreignKeys.resize(_size1307);
-             uint32_t _i1311;
-             for (_i1311 = 0; _i1311 < _size1307; ++_i1311)
 -            uint32_t _size1312;
 -            ::apache::thrift::protocol::TType _etype1315;
 -            xfer += iprot->readListBegin(_etype1315, _size1312);
 -            this->foreignKeys.resize(_size1312);
 -            uint32_t _i1316;
 -            for (_i1316 = 0; _i1316 < _size1312; ++_i1316)
++            uint32_t _size1331;
++            ::apache::thrift::protocol::TType _etype1334;
++            xfer += iprot->readListBegin(_etype1334, _size1331);
++            this->foreignKeys.resize(_size1331);
++            uint32_t _i1335;
++            for (_i1335 = 0; _i1335 < _size1331; ++_i1335)
              {
-               xfer += this->foreignKeys[_i1311].read(iprot);
 -              xfer += this->foreignKeys[_i1316].read(iprot);
++              xfer += this->foreignKeys[_i1335].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5652,14 -5652,14 +5652,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->uniqueConstraints.clear();
-             uint32_t _size1312;
-             ::apache::thrift::protocol::TType _etype1315;
-             xfer += iprot->readListBegin(_etype1315, _size1312);
-             this->uniqueConstraints.resize(_size1312);
-             uint32_t _i1316;
-             for (_i1316 = 0; _i1316 < _size1312; ++_i1316)
 -            uint32_t _size1317;
 -            ::apache::thrift::protocol::TType _etype1320;
 -            xfer += iprot->readListBegin(_etype1320, _size1317);
 -            this->uniqueConstraints.resize(_size1317);
 -            uint32_t _i1321;
 -            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
++            uint32_t _size1336;
++            ::apache::thrift::protocol::TType _etype1339;
++            xfer += iprot->readListBegin(_etype1339, _size1336);
++            this->uniqueConstraints.resize(_size1336);
++            uint32_t _i1340;
++            for (_i1340 = 0; _i1340 < _size1336; ++_i1340)
              {
-               xfer += this->uniqueConstraints[_i1316].read(iprot);
 -              xfer += this->uniqueConstraints[_i1321].read(iprot);
++              xfer += this->uniqueConstraints[_i1340].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5672,14 -5672,14 +5672,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->notNullConstraints.clear();
-             uint32_t _size1317;
-             ::apache::thrift::protocol::TType _etype1320;
-             xfer += iprot->readListBegin(_etype1320, _size1317);
-             this->notNullConstraints.resize(_size1317);
-             uint32_t _i1321;
-             for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
 -            uint32_t _size1322;
 -            ::apache::thrift::protocol::TType _etype1325;
 -            xfer += iprot->readListBegin(_etype1325, _size1322);
 -            this->notNullConstraints.resize(_size1322);
 -            uint32_t _i1326;
 -            for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
++            uint32_t _size1341;
++            ::apache::thrift::protocol::TType _etype1344;
++            xfer += iprot->readListBegin(_etype1344, _size1341);
++            this->notNullConstraints.resize(_size1341);
++            uint32_t _i1345;
++            for (_i1345 = 0; _i1345 < _size1341; ++_i1345)
              {
-               xfer += this->notNullConstraints[_i1321].read(iprot);
 -              xfer += this->notNullConstraints[_i1326].read(iprot);
++              xfer += this->notNullConstraints[_i1345].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5692,14 -5692,14 +5692,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->defaultConstraints.clear();
-             uint32_t _size1322;
-             ::apache::thrift::protocol::TType _etype1325;
-             xfer += iprot->readListBegin(_etype1325, _size1322);
-             this->defaultConstraints.resize(_size1322);
-             uint32_t _i1326;
-             for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
 -            uint32_t _size1327;
 -            ::apache::thrift::protocol::TType _etype1330;
 -            xfer += iprot->readListBegin(_etype1330, _size1327);
 -            this->defaultConstraints.resize(_size1327);
 -            uint32_t _i1331;
 -            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
++            uint32_t _size1346;
++            ::apache::thrift::protocol::TType _etype1349;
++            xfer += iprot->readListBegin(_etype1349, _size1346);
++            this->defaultConstraints.resize(_size1346);
++            uint32_t _i1350;
++            for (_i1350 = 0; _i1350 < _size1346; ++_i1350)
              {
-               xfer += this->defaultConstraints[_i1326].read(iprot);
 -              xfer += this->defaultConstraints[_i1331].read(iprot);
++              xfer += this->defaultConstraints[_i1350].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5712,14 -5712,14 +5712,14 @@@
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->checkConstraints.clear();
-             uint32_t _size1327;
-             ::apache::thrift::protocol::TType _etype1330;
-             xfer += iprot->readListBegin(_etype1330, _size1327);
-             this->checkConstraints.resize(_size1327);
-             uint32_t _i1331;
-             for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
 -            uint32_t _size1332;
 -            ::apache::thrift::protocol::TType _etype1335;
 -            xfer += iprot->readListBegin(_etype1335, _size1332);
 -            this->checkConstraints.resize(_size1332);
 -            uint32_t _i1336;
 -            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
++            uint32_t _size1351;
++            ::apache::thrift::protocol::TType _etype1354;
++            xfer += iprot->readListBegin(_etype1354, _size1351);
++            this->checkConstraints.resize(_size1351);
++            uint32_t _i1355;
++            for (_i1355 = 0; _i1355 < _size1351; ++_i1355)
              {
-               xfer += this->checkConstraints[_i1331].read(iprot);
 -              xfer += this->checkConstraints[_i1336].read(iprot);
++              xfer += this->checkConstraints[_i1355].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -5752,10 -5752,10 +5752,10 @@@ uint32_t ThriftHiveMetastore_create_tab
    xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-     std::vector<SQLPrimaryKey> ::const_iterator _iter1332;
-     for (_iter1332 = this->primaryKeys.begin(); _iter1332 != this->primaryKeys.end(); ++_iter1332)
 -    std::vector<SQLPrimaryKey> ::const_iterator _iter1337;
 -    for (_iter1337 = this->primaryKeys.begin(); _iter1337 != this->primaryKeys.end(); ++_iter1337)
++    std::vector<SQLPrimaryKey> ::const_iterator _iter1356;
++    for (_iter1356 = this->primaryKeys.begin(); _iter1356 != this->primaryKeys.end(); ++_iter1356)
      {
-       xfer += (*_iter1332).write(oprot);
 -      xfer += (*_iter1337).write(oprot);
++      xfer += (*_iter1356).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5764,10 -5764,10 +5764,10 @@@
    xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-     std::vector<SQLForeignKey> ::const_iterator _iter1333;
-     for (_iter1333 = this->foreignKeys.begin(); _iter1333 != this->foreignKeys.end(); ++_iter1333)
 -    std::vector<SQLForeignKey> ::const_iterator _iter1338;
 -    for (_iter1338 = this->foreignKeys.begin(); _iter1338 != this->foreignKeys.end(); ++_iter1338)
++    std::vector<SQLForeignKey> ::const_iterator _iter1357;
++    for (_iter1357 = this->foreignKeys.begin(); _iter1357 != this->foreignKeys.end(); ++_iter1357)
      {
-       xfer += (*_iter1333).write(oprot);
 -      xfer += (*_iter1338).write(oprot);
++      xfer += (*_iter1357).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5776,10 -5776,10 +5776,10 @@@
    xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-     std::vector<SQLUniqueConstraint> ::const_iterator _iter1334;
-     for (_iter1334 = this->uniqueConstraints.begin(); _iter1334 != this->uniqueConstraints.end(); ++_iter1334)
 -    std::vector<SQLUniqueConstraint> ::const_iterator _iter1339;
 -    for (_iter1339 = this->uniqueConstraints.begin(); _iter1339 != this->uniqueConstraints.end(); ++_iter1339)
++    std::vector<SQLUniqueConstraint> ::const_iterator _iter1358;
++    for (_iter1358 = this->uniqueConstraints.begin(); _iter1358 != this->uniqueConstraints.end(); ++_iter1358)
      {
-       xfer += (*_iter1334).write(oprot);
 -      xfer += (*_iter1339).write(oprot);
++      xfer += (*_iter1358).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5788,10 -5788,10 +5788,10 @@@
    xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-     std::vector<SQLNotNullConstraint> ::const_iterator _iter1335;
-     for (_iter1335 = this->notNullConstraints.begin(); _iter1335 != this->notNullConstraints.end(); ++_iter1335)
 -    std::vector<SQLNotNullConstraint> ::const_iterator _iter1340;
 -    for (_iter1340 = this->notNullConstraints.begin(); _iter1340 != this->notNullConstraints.end(); ++_iter1340)
++    std::vector<SQLNotNullConstraint> ::const_iterator _iter1359;
++    for (_iter1359 = this->notNullConstraints.begin(); _iter1359 != this->notNullConstraints.end(); ++_iter1359)
      {
-       xfer += (*_iter1335).write(oprot);
 -      xfer += (*_iter1340).write(oprot);
++      xfer += (*_iter1359).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5800,10 -5800,10 +5800,10 @@@
    xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-     std::vector<SQLDefaultConstraint> ::const_iterator _iter1336;
-     for (_iter1336 = this->defaultConstraints.begin(); _iter1336 != this->defaultConstraints.end(); ++_iter1336)
 -    std::vector<SQLDefaultConstraint> ::const_iterator _iter1341;
 -    for (_iter1341 = this->defaultConstraints.begin(); _iter1341 != this->defaultConstraints.end(); ++_iter1341)
++    std::vector<SQLDefaultConstraint> ::const_iterator _iter1360;
++    for (_iter1360 = this->defaultConstraints.begin(); _iter1360 != this->defaultConstraints.end(); ++_iter1360)
      {
-       xfer += (*_iter1336).write(oprot);
 -      xfer += (*_iter1341).write(oprot);
++      xfer += (*_iter1360).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5812,10 -5812,10 +5812,10 @@@
    xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-     std::vector<SQLCheckConstraint> ::const_iterator _iter1337;
-     for (_iter1337 = this->checkConstraints.begin(); _iter1337 != this->checkConstraints.end(); ++_iter1337)
 -    std::vector<SQLCheckConstraint> ::const_iterator _iter1342;
 -    for (_iter1342 = this->checkConstraints.begin(); _iter1342 != this->checkConstraints.end(); ++_iter1342)
++    std::vector<SQLCheckConstraint> ::const_iterator _iter1361;
++    for (_iter1361 = this->checkConstraints.begin(); _iter1361 != this->checkConstraints.end(); ++_iter1361)
      {
-       xfer += (*_iter1337).write(oprot);
 -      xfer += (*_iter1342).write(oprot);
++      xfer += (*_iter1361).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5843,10 -5843,10 +5843,10 @@@ uint32_t ThriftHiveMetastore_create_tab
    xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-     std::vector<SQLPrimaryKey> ::const_iterator _iter1338;
-     for (_iter1338 = (*(this->primaryKeys)).begin(); _iter1338 != (*(this->primaryKeys)).end(); ++_iter1338)
 -    std::vector<SQLPrimaryKey> ::const_iterator _iter1343;
 -    for (_iter1343 = (*(this->primaryKeys)).begin(); _iter1343 != (*(this->primaryKeys)).end(); ++_iter1343)
++    std::vector<SQLPrimaryKey> ::const_iterator _iter1362;
++    for (_iter1362 = (*(this->primaryKeys)).begin(); _iter1362 != (*(this->primaryKeys)).end(); ++_iter1362)
      {
-       xfer += (*_iter1338).write(oprot);
 -      xfer += (*_iter1343).write(oprot);
++      xfer += (*_iter1362).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5855,10 -5855,10 +5855,10 @@@
    xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-     std::vector<SQLForeignKey> ::const_iterator _iter1339;
-     for (_iter1339 = (*(this->foreignKeys)).begin(); _iter1339 != (*(this->foreignKeys)).end(); ++_iter1339)
 -    std::vector<SQLForeignKey> ::const_iterator _iter1344;
 -    for (_iter1344 = (*(this->foreignKeys)).begin(); _iter1344 != (*(this->foreignKeys)).end(); ++_iter1344)
++    std::vector<SQLForeignKey> ::const_iterator _iter1363;
++    for (_iter1363 = (*(this->foreignKeys)).begin(); _iter1363 != (*(this->foreignKeys)).end(); ++_iter1363)
      {
-       xfer += (*_iter1339).write(oprot);
 -      xfer += (*_iter1344).write(oprot);
++      xfer += (*_iter1363).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5867,10 -5867,10 +5867,10 @@@
    xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-     std::vector<SQLUniqueConstraint> ::const_iterator _iter1340;
-     for (_iter1340 = (*(this->uniqueConstraints)).begin(); _iter1340 != (*(this->uniqueConstraints)).end(); ++_iter1340)
 -    std::vector<SQLUniqueConstraint> ::const_iterator _iter1345;
 -    for (_iter1345 = (*(this->uniqueConstraints)).begin(); _iter1345 != (*(this->uniqueConstraints)).end(); ++_iter1345)
++    std::vector<SQLUniqueConstraint> ::const_iterator _iter1364;
++    for (_iter1364 = (*(this->uniqueConstraints)).begin(); _iter1364 != (*(this->uniqueConstraints)).end(); ++_iter1364)
      {
-       xfer += (*_iter1340).write(oprot);
 -      xfer += (*_iter1345).write(oprot);
++      xfer += (*_iter1364).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5879,10 -5879,10 +5879,10 @@@
    xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-     std::vector<SQLNotNullConstraint> ::const_iterator _iter1341;
-     for (_iter1341 = (*(this->notNullConstraints)).begin(); _iter1341 != (*(this->notNullConstraints)).end(); ++_iter1341)
 -    std::vector<SQLNotNullConstraint> ::const_iterator _iter1346;
 -    for (_iter1346 = (*(this->notNullConstraints)).begin(); _iter1346 != (*(this->notNullConstraints)).end(); ++_iter1346)
++    std::vector<SQLNotNullConstraint> ::const_iterator _iter1365;
++    for (_iter1365 = (*(this->notNullConstraints)).begin(); _iter1365 != (*(this->notNullConstraints)).end(); ++_iter1365)
      {
-       xfer += (*_iter1341).write(oprot);
 -      xfer += (*_iter1346).write(oprot);
++      xfer += (*_iter1365).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5891,10 -5891,10 +5891,10 @@@
    xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-     std::vector<SQLDefaultConstraint> ::const_iterator _iter1342;
-     for (_iter1342 = (*(this->defaultConstraints)).begin(); _iter1342 != (*(this->defaultConstraints)).end(); ++_iter1342)
 -    std::vector<SQLDefaultConstraint> ::const_iterator _iter1347;
 -    for (_iter1347 = (*(this->defaultConstraints)).begin(); _iter1347 != (*(this->defaultConstraints)).end(); ++_iter1347)
++    std::vector<SQLDefaultConstraint> ::const_iterator _iter1366;
++    for (_iter1366 = (*(this->defaultConstraints)).begin(); _iter1366 != (*(this->defaultConstraints)).end(); ++_iter1366)
      {
-       xfer += (*_iter1342).write(oprot);
 -      xfer += (*_iter1347).write(oprot);
++      xfer += (*_iter1366).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -5903,10 -5903,10 +5903,10 @@@
    xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-     std::vector<SQLCheckConstraint> ::const_iterator _iter1343;
-     for (_iter1343 = (*(this->checkConstraints)).begin(); _iter1343 != (*(this->checkConstraints)).end(); ++_iter1343)
 -    std::vector<SQLCheckConstraint> ::const_iterator _iter1348;
 -    for (_iter1348 = (*(this->checkConstraints)).begin(); _iter1348 != (*(this->checkConstraints)).end(); ++_iter1348)
++    std::vector<SQLCheckConstraint> ::const_iterator _iter1367;
++    for (_iter1367 = (*(this->checkConstraints)).begin(); _iter1367 != (*(this->checkConstraints)).end(); ++_iter1367)
      {
-       xfer += (*_iter1343).write(oprot);
 -      xfer += (*_iter1348).write(oprot);
++      xfer += (*_iter1367).write(oprot);
      }
      xfer += oprot->writeListEnd();
    }
@@@ -8074,14 -8074,14 +8074,14 @@@ uint32_t ThriftHiveMetastore_truncate_t
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->partNames.clear();
-             uint32_t _size1344;
-             ::apache::thrift::protocol::TType _etype1347;
-             xfer += iprot->readListBegin(_etype1347, _size1344);
-             this->partNames.resize(_size1344);
-             uint32_t _i1348;
-             for (_i1348 = 0; _i1348 < _size1344; ++_i1348)
 -            uint32_t _size1349;
 -            ::apache::thrift::protocol::TType _etype1352;
 -            xfer += iprot->readListBegin(_etype1352, _size1349);
 -            this->partNames.resize(_size1349);
 -            uint32_t _i1353;
 -            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
++            uint32_t _size1368;
++            ::apache::thrift::protocol::TType _etype1371;
++            xfer += iprot->readListBegin(_etype1371, _size1368);
++            this->partNames.resize(_size1368);
++            uint32_t _i1372;
++            for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
              {
-               xfer += iprot->readString(this->partNames[_i1348]);
 -              xfer += iprot->readString(this->partNames[_i1353]);
++              xfer += iprot->readString(this->partNames[_i1372]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8118,10 -8118,10 +8118,10 @@@ uint32_t ThriftHiveMetastore_truncate_t
    xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-     std::vector<std::string> ::const_iterator _iter1349;
-     for (_iter1349 = this->partNames.begin(); _iter1349 != this->partNames.end(); ++_iter1349)
 -    std::vector<std::string> ::const_iterator _iter1354;
 -    for (_iter1354 = this->partNames.begin(); _iter1354 != this->partNames.end(); ++_iter1354)
++    std::vector<std::string> ::const_iterator _iter1373;
++    for (_iter1373 = this->partNames.begin(); _iter1373 != this->partNames.end(); ++_iter1373)
      {
-       xfer += oprot->writeString((*_iter1349));
 -      xfer += oprot->writeString((*_iter1354));
++      xfer += oprot->writeString((*_iter1373));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -8153,10 -8153,10 +8153,10 @@@ uint32_t ThriftHiveMetastore_truncate_t
    xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-     std::vector<std::string> ::const_iterator _iter1350;
-     for (_iter1350 = (*(this->partNames)).begin(); _iter1350 != (*(this->partNames)).end(); ++_iter1350)
 -    std::vector<std::string> ::const_iterator _iter1355;
 -    for (_iter1355 = (*(this->partNames)).begin(); _iter1355 != (*(this->partNames)).end(); ++_iter1355)
++    std::vector<std::string> ::const_iterator _iter1374;
++    for (_iter1374 = (*(this->partNames)).begin(); _iter1374 != (*(this->partNames)).end(); ++_iter1374)
      {
-       xfer += oprot->writeString((*_iter1350));
 -      xfer += oprot->writeString((*_iter1355));
++      xfer += oprot->writeString((*_iter1374));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -8400,14 -8400,14 +8400,14 @@@ uint32_t ThriftHiveMetastore_get_tables
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1351;
-             ::apache::thrift::protocol::TType _etype1354;
-             xfer += iprot->readListBegin(_etype1354, _size1351);
-             this->success.resize(_size1351);
-             uint32_t _i1355;
-             for (_i1355 = 0; _i1355 < _size1351; ++_i1355)
 -            uint32_t _size1356;
 -            ::apache::thrift::protocol::TType _etype1359;
 -            xfer += iprot->readListBegin(_etype1359, _size1356);
 -            this->success.resize(_size1356);
 -            uint32_t _i1360;
 -            for (_i1360 = 0; _i1360 < _size1356; ++_i1360)
++            uint32_t _size1375;
++            ::apache::thrift::protocol::TType _etype1378;
++            xfer += iprot->readListBegin(_etype1378, _size1375);
++            this->success.resize(_size1375);
++            uint32_t _i1379;
++            for (_i1379 = 0; _i1379 < _size1375; ++_i1379)
              {
-               xfer += iprot->readString(this->success[_i1355]);
 -              xfer += iprot->readString(this->success[_i1360]);
++              xfer += iprot->readString(this->success[_i1379]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8446,10 -8446,10 +8446,10 @@@ uint32_t ThriftHiveMetastore_get_tables
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-       std::vector<std::string> ::const_iterator _iter1356;
-       for (_iter1356 = this->success.begin(); _iter1356 != this->success.end(); ++_iter1356)
 -      std::vector<std::string> ::const_iterator _iter1361;
 -      for (_iter1361 = this->success.begin(); _iter1361 != this->success.end(); ++_iter1361)
++      std::vector<std::string> ::const_iterator _iter1380;
++      for (_iter1380 = this->success.begin(); _iter1380 != this->success.end(); ++_iter1380)
        {
-         xfer += oprot->writeString((*_iter1356));
 -        xfer += oprot->writeString((*_iter1361));
++        xfer += oprot->writeString((*_iter1380));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -8494,14 -8494,14 +8494,14 @@@ uint32_t ThriftHiveMetastore_get_tables
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1357;
-             ::apache::thrift::protocol::TType _etype1360;
-             xfer += iprot->readListBegin(_etype1360, _size1357);
-             (*(this->success)).resize(_size1357);
-             uint32_t _i1361;
-             for (_i1361 = 0; _i1361 < _size1357; ++_i1361)
 -            uint32_t _size1362;
 -            ::apache::thrift::protocol::TType _etype1365;
 -            xfer += iprot->readListBegin(_etype1365, _size1362);
 -            (*(this->success)).resize(_size1362);
 -            uint32_t _i1366;
 -            for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
++            uint32_t _size1381;
++            ::apache::thrift::protocol::TType _etype1384;
++            xfer += iprot->readListBegin(_etype1384, _size1381);
++            (*(this->success)).resize(_size1381);
++            uint32_t _i1385;
++            for (_i1385 = 0; _i1385 < _size1381; ++_i1385)
              {
-               xfer += iprot->readString((*(this->success))[_i1361]);
 -              xfer += iprot->readString((*(this->success))[_i1366]);
++              xfer += iprot->readString((*(this->success))[_i1385]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8671,14 -8671,14 +8671,14 @@@ uint32_t ThriftHiveMetastore_get_tables
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1362;
-             ::apache::thrift::protocol::TType _etype1365;
-             xfer += iprot->readListBegin(_etype1365, _size1362);
-             this->success.resize(_size1362);
-             uint32_t _i1366;
-             for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
 -            uint32_t _size1367;
 -            ::apache::thrift::protocol::TType _etype1370;
 -            xfer += iprot->readListBegin(_etype1370, _size1367);
 -            this->success.resize(_size1367);
 -            uint32_t _i1371;
 -            for (_i1371 = 0; _i1371 < _size1367; ++_i1371)
++            uint32_t _size1386;
++            ::apache::thrift::protocol::TType _etype1389;
++            xfer += iprot->readListBegin(_etype1389, _size1386);
++            this->success.resize(_size1386);
++            uint32_t _i1390;
++            for (_i1390 = 0; _i1390 < _size1386; ++_i1390)
              {
-               xfer += iprot->readString(this->success[_i1366]);
 -              xfer += iprot->readString(this->success[_i1371]);
++              xfer += iprot->readString(this->success[_i1390]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8717,10 -8717,10 +8717,10 @@@ uint32_t ThriftHiveMetastore_get_tables
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-       std::vector<std::string> ::const_iterator _iter1367;
-       for (_iter1367 = this->success.begin(); _iter1367 != this->success.end(); ++_iter1367)
 -      std::vector<std::string> ::const_iterator _iter1372;
 -      for (_iter1372 = this->success.begin(); _iter1372 != this->success.end(); ++_iter1372)
++      std::vector<std::string> ::const_iterator _iter1391;
++      for (_iter1391 = this->success.begin(); _iter1391 != this->success.end(); ++_iter1391)
        {
-         xfer += oprot->writeString((*_iter1367));
 -        xfer += oprot->writeString((*_iter1372));
++        xfer += oprot->writeString((*_iter1391));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -8765,14 -8765,14 +8765,14 @@@ uint32_t ThriftHiveMetastore_get_tables
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1368;
-             ::apache::thrift::protocol::TType _etype1371;
-             xfer += iprot->readListBegin(_etype1371, _size1368);
-             (*(this->success)).resize(_size1368);
-             uint32_t _i1372;
-             for (_i1372 = 0; _i1372 < _size1368; ++_i1372)
 -            uint32_t _size1373;
 -            ::apache::thrift::protocol::TType _etype1376;
 -            xfer += iprot->readListBegin(_etype1376, _size1373);
 -            (*(this->success)).resize(_size1373);
 -            uint32_t _i1377;
 -            for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
++            uint32_t _size1392;
++            ::apache::thrift::protocol::TType _etype1395;
++            xfer += iprot->readListBegin(_etype1395, _size1392);
++            (*(this->success)).resize(_size1392);
++            uint32_t _i1396;
++            for (_i1396 = 0; _i1396 < _size1392; ++_i1396)
              {
-               xfer += iprot->readString((*(this->success))[_i1372]);
 -              xfer += iprot->readString((*(this->success))[_i1377]);
++              xfer += iprot->readString((*(this->success))[_i1396]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8910,14 -8910,14 +8910,14 @@@ uint32_t ThriftHiveMetastore_get_materi
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1373;
-             ::apache::thrift::protocol::TType _etype1376;
-             xfer += iprot->readListBegin(_etype1376, _size1373);
-             this->success.resize(_size1373);
-             uint32_t _i1377;
-             for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
 -            uint32_t _size1378;
 -            ::apache::thrift::protocol::TType _etype1381;
 -            xfer += iprot->readListBegin(_etype1381, _size1378);
 -            this->success.resize(_size1378);
 -            uint32_t _i1382;
 -            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
++            uint32_t _size1397;
++            ::apache::thrift::protocol::TType _etype1400;
++            xfer += iprot->readListBegin(_etype1400, _size1397);
++            this->success.resize(_size1397);
++            uint32_t _i1401;
++            for (_i1401 = 0; _i1401 < _size1397; ++_i1401)
              {
-               xfer += iprot->readString(this->success[_i1377]);
 -              xfer += iprot->readString(this->success[_i1382]);
++              xfer += iprot->readString(this->success[_i1401]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -8956,10 -8956,10 +8956,10 @@@ uint32_t ThriftHiveMetastore_get_materi
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-       std::vector<std::string> ::const_iterator _iter1378;
-       for (_iter1378 = this->success.begin(); _iter1378 != this->success.end(); ++_iter1378)
 -      std::vector<std::string> ::const_iterator _iter1383;
 -      for (_iter1383 = this->success.begin(); _iter1383 != this->success.end(); ++_iter1383)
++      std::vector<std::string> ::const_iterator _iter1402;
++      for (_iter1402 = this->success.begin(); _iter1402 != this->success.end(); ++_iter1402)
        {
-         xfer += oprot->writeString((*_iter1378));
 -        xfer += oprot->writeString((*_iter1383));
++        xfer += oprot->writeString((*_iter1402));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -9004,14 -9004,14 +9004,14 @@@ uint32_t ThriftHiveMetastore_get_materi
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1379;
-             ::apache::thrift::protocol::TType _etype1382;
-             xfer += iprot->readListBegin(_etype1382, _size1379);
-             (*(this->success)).resize(_size1379);
-             uint32_t _i1383;
-             for (_i1383 = 0; _i1383 < _size1379; ++_i1383)
 -            uint32_t _size1384;
 -            ::apache::thrift::protocol::TType _etype1387;
 -            xfer += iprot->readListBegin(_etype1387, _size1384);
 -            (*(this->success)).resize(_size1384);
 -            uint32_t _i1388;
 -            for (_i1388 = 0; _i1388 < _size1384; ++_i1388)
++            uint32_t _size1403;
++            ::apache::thrift::protocol::TType _etype1406;
++            xfer += iprot->readListBegin(_etype1406, _size1403);
++            (*(this->success)).resize(_size1403);
++            uint32_t _i1407;
++            for (_i1407 = 0; _i1407 < _size1403; ++_i1407)
              {
-               xfer += iprot->readString((*(this->success))[_i1383]);
 -              xfer += iprot->readString((*(this->success))[_i1388]);
++              xfer += iprot->readString((*(this->success))[_i1407]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9086,14 -9086,14 +9086,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->tbl_types.clear();
-             uint32_t _size1384;
-             ::apache::thrift::protocol::TType _etype1387;
-             xfer += iprot->readListBegin(_etype1387, _size1384);
-             this->tbl_types.resize(_size1384);
-             uint32_t _i1388;
-             for (_i1388 = 0; _i1388 < _size1384; ++_i1388)
 -            uint32_t _size1389;
 -            ::apache::thrift::protocol::TType _etype1392;
 -            xfer += iprot->readListBegin(_etype1392, _size1389);
 -            this->tbl_types.resize(_size1389);
 -            uint32_t _i1393;
 -            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
++            uint32_t _size1408;
++            ::apache::thrift::protocol::TType _etype1411;
++            xfer += iprot->readListBegin(_etype1411, _size1408);
++            this->tbl_types.resize(_size1408);
++            uint32_t _i1412;
++            for (_i1412 = 0; _i1412 < _size1408; ++_i1412)
              {
-               xfer += iprot->readString(this->tbl_types[_i1388]);
 -              xfer += iprot->readString(this->tbl_types[_i1393]);
++              xfer += iprot->readString(this->tbl_types[_i1412]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9130,10 -9130,10 +9130,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-     std::vector<std::string> ::const_iterator _iter1389;
-     for (_iter1389 = this->tbl_types.begin(); _iter1389 != this->tbl_types.end(); ++_iter1389)
 -    std::vector<std::string> ::const_iterator _iter1394;
 -    for (_iter1394 = this->tbl_types.begin(); _iter1394 != this->tbl_types.end(); ++_iter1394)
++    std::vector<std::string> ::const_iterator _iter1413;
++    for (_iter1413 = this->tbl_types.begin(); _iter1413 != this->tbl_types.end(); ++_iter1413)
      {
-       xfer += oprot->writeString((*_iter1389));
 -      xfer += oprot->writeString((*_iter1394));
++      xfer += oprot->writeString((*_iter1413));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9165,10 -9165,10 +9165,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-     std::vector<std::string> ::const_iterator _iter1390;
-     for (_iter1390 = (*(this->tbl_types)).begin(); _iter1390 != (*(this->tbl_types)).end(); ++_iter1390)
 -    std::vector<std::string> ::const_iterator _iter1395;
 -    for (_iter1395 = (*(this->tbl_types)).begin(); _iter1395 != (*(this->tbl_types)).end(); ++_iter1395)
++    std::vector<std::string> ::const_iterator _iter1414;
++    for (_iter1414 = (*(this->tbl_types)).begin(); _iter1414 != (*(this->tbl_types)).end(); ++_iter1414)
      {
-       xfer += oprot->writeString((*_iter1390));
 -      xfer += oprot->writeString((*_iter1395));
++      xfer += oprot->writeString((*_iter1414));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9209,14 -9209,14 +9209,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1391;
-             ::apache::thrift::protocol::TType _etype1394;
-             xfer += iprot->readListBegin(_etype1394, _size1391);
-             this->success.resize(_size1391);
-             uint32_t _i1395;
-             for (_i1395 = 0; _i1395 < _size1391; ++_i1395)
 -            uint32_t _size1396;
 -            ::apache::thrift::protocol::TType _etype1399;
 -            xfer += iprot->readListBegin(_etype1399, _size1396);
 -            this->success.resize(_size1396);
 -            uint32_t _i1400;
 -            for (_i1400 = 0; _i1400 < _size1396; ++_i1400)
++            uint32_t _size1415;
++            ::apache::thrift::protocol::TType _etype1418;
++            xfer += iprot->readListBegin(_etype1418, _size1415);
++            this->success.resize(_size1415);
++            uint32_t _i1419;
++            for (_i1419 = 0; _i1419 < _size1415; ++_i1419)
              {
-               xfer += this->success[_i1395].read(iprot);
 -              xfer += this->success[_i1400].read(iprot);
++              xfer += this->success[_i1419].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9255,10 -9255,10 +9255,10 @@@ uint32_t ThriftHiveMetastore_get_table_
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<TableMeta> ::const_iterator _iter1396;
-       for (_iter1396 = this->success.begin(); _iter1396 != this->success.end(); ++_iter1396)
 -      std::vector<TableMeta> ::const_iterator _iter1401;
 -      for (_iter1401 = this->success.begin(); _iter1401 != this->success.end(); ++_iter1401)
++      std::vector<TableMeta> ::const_iterator _iter1420;
++      for (_iter1420 = this->success.begin(); _iter1420 != this->success.end(); ++_iter1420)
        {
-         xfer += (*_iter1396).write(oprot);
 -        xfer += (*_iter1401).write(oprot);
++        xfer += (*_iter1420).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -9303,14 -9303,14 +9303,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1397;
-             ::apache::thrift::protocol::TType _etype1400;
-             xfer += iprot->readListBegin(_etype1400, _size1397);
-             (*(this->success)).resize(_size1397);
-             uint32_t _i1401;
-             for (_i1401 = 0; _i1401 < _size1397; ++_i1401)
 -            uint32_t _size1402;
 -            ::apache::thrift::protocol::TType _etype1405;
 -            xfer += iprot->readListBegin(_etype1405, _size1402);
 -            (*(this->success)).resize(_size1402);
 -            uint32_t _i1406;
 -            for (_i1406 = 0; _i1406 < _size1402; ++_i1406)
++            uint32_t _size1421;
++            ::apache::thrift::protocol::TType _etype1424;
++            xfer += iprot->readListBegin(_etype1424, _size1421);
++            (*(this->success)).resize(_size1421);
++            uint32_t _i1425;
++            for (_i1425 = 0; _i1425 < _size1421; ++_i1425)
              {
-               xfer += (*(this->success))[_i1401].read(iprot);
 -              xfer += (*(this->success))[_i1406].read(iprot);
++              xfer += (*(this->success))[_i1425].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9448,14 -9448,14 +9448,14 @@@ uint32_t ThriftHiveMetastore_get_all_ta
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1402;
-             ::apache::thrift::protocol::TType _etype1405;
-             xfer += iprot->readListBegin(_etype1405, _size1402);
-             this->success.resize(_size1402);
-             uint32_t _i1406;
-             for (_i1406 = 0; _i1406 < _size1402; ++_i1406)
 -            uint32_t _size1407;
 -            ::apache::thrift::protocol::TType _etype1410;
 -            xfer += iprot->readListBegin(_etype1410, _size1407);
 -            this->success.resize(_size1407);
 -            uint32_t _i1411;
 -            for (_i1411 = 0; _i1411 < _size1407; ++_i1411)
++            uint32_t _size1426;
++            ::apache::thrift::protocol::TType _etype1429;
++            xfer += iprot->readListBegin(_etype1429, _size1426);
++            this->success.resize(_size1426);
++            uint32_t _i1430;
++            for (_i1430 = 0; _i1430 < _size1426; ++_i1430)
              {
-               xfer += iprot->readString(this->success[_i1406]);
 -              xfer += iprot->readString(this->success[_i1411]);
++              xfer += iprot->readString(this->success[_i1430]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9494,10 -9494,10 +9494,10 @@@ uint32_t ThriftHiveMetastore_get_all_ta
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-       std::vector<std::string> ::const_iterator _iter1407;
-       for (_iter1407 = this->success.begin(); _iter1407 != this->success.end(); ++_iter1407)
 -      std::vector<std::string> ::const_iterator _iter1412;
 -      for (_iter1412 = this->success.begin(); _iter1412 != this->success.end(); ++_iter1412)
++      std::vector<std::string> ::const_iterator _iter1431;
++      for (_iter1431 = this->success.begin(); _iter1431 != this->success.end(); ++_iter1431)
        {
-         xfer += oprot->writeString((*_iter1407));
 -        xfer += oprot->writeString((*_iter1412));
++        xfer += oprot->writeString((*_iter1431));
        }
        xfer += oprot->writeListEnd();
      }
@@@ -9542,14 -9542,14 +9542,14 @@@ uint32_t ThriftHiveMetastore_get_all_ta
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1408;
-             ::apache::thrift::protocol::TType _etype1411;
-             xfer += iprot->readListBegin(_etype1411, _size1408);
-             (*(this->success)).resize(_size1408);
-             uint32_t _i1412;
-             for (_i1412 = 0; _i1412 < _size1408; ++_i1412)
 -            uint32_t _size1413;
 -            ::apache::thrift::protocol::TType _etype1416;
 -            xfer += iprot->readListBegin(_etype1416, _size1413);
 -            (*(this->success)).resize(_size1413);
 -            uint32_t _i1417;
 -            for (_i1417 = 0; _i1417 < _size1413; ++_i1417)
++            uint32_t _size1432;
++            ::apache::thrift::protocol::TType _etype1435;
++            xfer += iprot->readListBegin(_etype1435, _size1432);
++            (*(this->success)).resize(_size1432);
++            uint32_t _i1436;
++            for (_i1436 = 0; _i1436 < _size1432; ++_i1436)
              {
-               xfer += iprot->readString((*(this->success))[_i1412]);
 -              xfer += iprot->readString((*(this->success))[_i1417]);
++              xfer += iprot->readString((*(this->success))[_i1436]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9859,14 -9859,14 +9859,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->tbl_names.clear();
-             uint32_t _size1413;
-             ::apache::thrift::protocol::TType _etype1416;
-             xfer += iprot->readListBegin(_etype1416, _size1413);
-             this->tbl_names.resize(_size1413);
-             uint32_t _i1417;
-             for (_i1417 = 0; _i1417 < _size1413; ++_i1417)
 -            uint32_t _size1418;
 -            ::apache::thrift::protocol::TType _etype1421;
 -            xfer += iprot->readListBegin(_etype1421, _size1418);
 -            this->tbl_names.resize(_size1418);
 -            uint32_t _i1422;
 -            for (_i1422 = 0; _i1422 < _size1418; ++_i1422)
++            uint32_t _size1437;
++            ::apache::thrift::protocol::TType _etype1440;
++            xfer += iprot->readListBegin(_etype1440, _size1437);
++            this->tbl_names.resize(_size1437);
++            uint32_t _i1441;
++            for (_i1441 = 0; _i1441 < _size1437; ++_i1441)
              {
-               xfer += iprot->readString(this->tbl_names[_i1417]);
 -              xfer += iprot->readString(this->tbl_names[_i1422]);
++              xfer += iprot->readString(this->tbl_names[_i1441]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -9899,10 -9899,10 +9899,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-     std::vector<std::string> ::const_iterator _iter1418;
-     for (_iter1418 = this->tbl_names.begin(); _iter1418 != this->tbl_names.end(); ++_iter1418)
 -    std::vector<std::string> ::const_iterator _iter1423;
 -    for (_iter1423 = this->tbl_names.begin(); _iter1423 != this->tbl_names.end(); ++_iter1423)
++    std::vector<std::string> ::const_iterator _iter1442;
++    for (_iter1442 = this->tbl_names.begin(); _iter1442 != this->tbl_names.end(); ++_iter1442)
      {
-       xfer += oprot->writeString((*_iter1418));
 -      xfer += oprot->writeString((*_iter1423));
++      xfer += oprot->writeString((*_iter1442));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9930,10 -9930,10 +9930,10 @@@ uint32_t ThriftHiveMetastore_get_table_
    xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-     std::vector<std::string> ::const_iterator _iter1419;
-     for (_iter1419 = (*(this->tbl_names)).begin(); _iter1419 != (*(this->tbl_names)).end(); ++_iter1419)
 -    std::vector<std::string> ::const_iterator _iter1424;
 -    for (_iter1424 = (*(this->tbl_names)).begin(); _iter1424 != (*(this->tbl_names)).end(); ++_iter1424)
++    std::vector<std::string> ::const_iterator _iter1443;
++    for (_iter1443 = (*(this->tbl_names)).begin(); _iter1443 != (*(this->tbl_names)).end(); ++_iter1443)
      {
-       xfer += oprot->writeString((*_iter1419));
 -      xfer += oprot->writeString((*_iter1424));
++      xfer += oprot->writeString((*_iter1443));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -9974,14 -9974,14 +9974,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1420;
-             ::apache::thrift::protocol::TType _etype1423;
-             xfer += iprot->readListBegin(_etype1423, _size1420);
-             this->success.resize(_size1420);
-             uint32_t _i1424;
-             for (_i1424 = 0; _i1424 < _size1420; ++_i1424)
 -            uint32_t _size1425;
 -            ::apache::thrift::protocol::TType _etype1428;
 -            xfer += iprot->readListBegin(_etype1428, _size1425);
 -            this->success.resize(_size1425);
 -            uint32_t _i1429;
 -            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
++            uint32_t _size1444;
++            ::apache::thrift::protocol::TType _etype1447;
++            xfer += iprot->readListBegin(_etype1447, _size1444);
++            this->success.resize(_size1444);
++            uint32_t _i1448;
++            for (_i1448 = 0; _i1448 < _size1444; ++_i1448)
              {
-               xfer += this->success[_i1424].read(iprot);
 -              xfer += this->success[_i1429].read(iprot);
++              xfer += this->success[_i1448].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -10012,10 -10012,10 +10012,10 @@@ uint32_t ThriftHiveMetastore_get_table_
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
      {
        xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::vector<Table> ::const_iterator _iter1425;
-       for (_iter1425 = this->success.begin(); _iter1425 != this->success.end(); ++_iter1425)
 -      std::vector<Table> ::const_iterator _iter1430;
 -      for (_iter1430 = this->success.begin(); _iter1430 != this->success.end(); ++_iter1430)
++      std::vector<Table> ::const_iterator _iter1449;
++      for (_iter1449 = this->success.begin(); _iter1449 != this->success.end(); ++_iter1449)
        {
-         xfer += (*_iter1425).write(oprot);
 -        xfer += (*_iter1430).write(oprot);
++        xfer += (*_iter1449).write(oprot);
        }
        xfer += oprot->writeListEnd();
      }
@@@ -10056,14 -10056,14 +10056,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              (*(this->success)).clear();
-             uint32_t _size1426;
-             ::apache::thrift::protocol::TType _etype1429;
-             xfer += iprot->readListBegin(_etype1429, _size1426);
-             (*(this->success)).resize(_size1426);
-             uint32_t _i1430;
-             for (_i1430 = 0; _i1430 < _size1426; ++_i1430)
 -            uint32_t _size1431;
 -            ::apache::thrift::protocol::TType _etype1434;
 -            xfer += iprot->readListBegin(_etype1434, _size1431);
 -            (*(this->success)).resize(_size1431);
 -            uint32_t _i1435;
 -            for (_i1435 = 0; _i1435 < _size1431; ++_i1435)
++            uint32_t _size1450;
++            ::apache::thrift::protocol::TType _etype1453;
++            xfer += iprot->readListBegin(_etype1453, _size1450);
++            (*(this->success)).resize(_size1450);
++            uint32_t _i1454;
++            for (_i1454 = 0; _i1454 < _size1450; ++_i1454)
              {
-               xfer += (*(this->success))[_i1430].read(iprot);
 -              xfer += (*(this->success))[_i1435].read(iprot);
++              xfer += (*(this->success))[_i1454].read(iprot);
              }
              xfer += iprot->readListEnd();
            }
@@@ -10596,14 -10596,14 +10596,14 @@@ uint32_t ThriftHiveMetastore_get_materi
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->tbl_names.clear();
-             uint32_t _size1431;
-             ::apache::thrift::protocol::TType _etype1434;
-             xfer += iprot->readListBegin(_etype1434, _size1431);
-             this->tbl_names.resize(_size1431);
-             uint32_t _i1435;
-             for (_i1435 = 0; _i1435 < _size1431; ++_i1435)
 -            uint32_t _size1436;
 -            ::apache::thrift::protocol::TType _etype1439;
 -            xfer += iprot->readListBegin(_etype1439, _size1436);
 -            this->tbl_names.resize(_size1436);
 -            uint32_t _i1440;
 -            for (_i1440 = 0; _i1440 < _size1436; ++_i1440)
++            uint32_t _size1455;
++            ::apache::thrift::protocol::TType _etype1458;
++            xfer += iprot->readListBegin(_etype1458, _size1455);
++            this->tbl_names.resize(_size1455);
++            uint32_t _i1459;
++            for (_i1459 = 0; _i1459 < _size1455; ++_i1459)
              {
-               xfer += iprot->readString(this->tbl_names[_i1435]);
 -              xfer += iprot->readString(this->tbl_names[_i1440]);
++              xfer += iprot->readString(this->tbl_names[_i1459]);
              }
              xfer += iprot->readListEnd();
            }
@@@ -10636,10 -10636,10 +10636,10 @@@ uint32_t ThriftHiveMetastore_get_materi
    xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-     std::vector<std::string> ::const_iterator _iter1436;
-     for (_iter1436 = this->tbl_names.begin(); _iter1436 != this->tbl_names.end(); ++_iter1436)
 -    std::vector<std::string> ::const_iterator _iter1441;
 -    for (_iter1441 = this->tbl_names.begin(); _iter1441 != this->tbl_names.end(); ++_iter1441)
++    std::vector<std::string> ::const_iterator _iter1460;
++    for (_iter1460 = this->tbl_names.begin(); _iter1460 != this->tbl_names.end(); ++_iter1460)
      {
-       xfer += oprot->writeString((*_iter1436));
 -      xfer += oprot->writeString((*_iter1441));
++      xfer += oprot->writeString((*_iter1460));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -10667,10 -10667,10 +10667,10 @@@ uint32_t ThriftHiveMetastore_get_materi
    xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
    {
      xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-     std::vector<std::string> ::const_iterator _iter1437;
-     for (_iter1437 = (*(this->tbl_names)).begin(); _iter1437 != (*(this->tbl_names)).end(); ++_iter1437)
 -    std::vector<std::string> ::const_iterator _iter1442;
 -    for (_iter1442 = (*(this->tbl_names)).begin(); _iter1442 != (*(this->tbl_names)).end(); ++_iter1442)
++    std::vector<std::string> ::const_iterator _iter1461;
++    for (_iter1461 = (*(this->tbl_names)).begin(); _iter1461 != (*(this->tbl_names)).end(); ++_iter1461)
      {
-       xfer += oprot->writeString((*_iter1437));
 -      xfer += oprot->writeString((*_iter1442));
++      xfer += oprot->writeString((*_iter1461));
      }
      xfer += oprot->writeListEnd();
    }
@@@ -10711,17 -10711,17 +10711,17 @@@ uint32_t ThriftHiveMetastore_get_materi
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              this->success.clear();
-             uint32_t _size1438;
-             ::apache::thrift::protocol::TType _ktype1439;
-             ::apache::thrift::protocol::TType _vtype1440;
-             xfer += iprot->readMapBegin(_ktype1439, _vtype1440, _size1438);
-             uint32_t _i1442;
-             for (_i1442 = 0; _i1442 < _size1438; ++_i1442)
 -            uint32_t _size1443;
 -            ::apache::thrift::protocol::TType _ktype1444;
 -            ::apache::thrift::protocol::TType _vtype1445;
 -            xfer += iprot->readMapBegin(_ktype1444, _vtype1445, _size1443);
 -            uint32_t _i1447;
 -            for (_i1447 = 0; _i1447 < _size1443; ++_i1447)
++            uint32_t _size1462;
++            ::apache::thrift::protocol::TType _ktype1463;
++            ::apache::thrift::protocol::TType _vtype1464;
++            xfer += iprot->readMapBegin(_ktype1463, _vtype1464, _size1462);
++            uint32_t _i1466;
++            for (_i1466 = 0; _i1466 < _size1462; ++_i1466)
              {
-               std::string _key1443;
-               xfer += iprot->readString(_key1443);
-               Materialization& _val1444 = this->success[_key1443];
-               xfer += _val1444.read(iprot);
 -              std::string _key1448;
 -              xfer += iprot->readString(_key1448);
 -              Materialization& _val1449 = this->success[_key1448];
 -              xfer += _val1449.read(iprot);
++              std::string _key1467;
++              xfer += iprot->readString(_key1467);
++              Materialization& _val1468 = this->success[_key1467];
++              xfer += _val1468.read(iprot);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -10776,11 -10776,11 +10776,11 @@@ uint32_t ThriftHiveMetastore_get_materi
      xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
      {
        xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-       std::map<std::string, Materialization> ::const_iterator _iter1445;
-       for (_iter1445 = this->success.begin(); _iter1445 != this->success.end(); ++_iter1445)
 -      std::map<std::string, Materialization> ::const_iterator _iter1450;
 -      for (_iter1450 = this->success.begin(); _iter1450 != this->success.end(); ++_iter1450)
++      std::map<std::string, Materialization> ::const_iterator _iter1469;
++      for (_iter1469 = this->success.begin(); _iter1469 != this->success.end(); ++_iter1469)
        {
-         xfer += oprot->writeString(_iter1445->first);
-         xfer += _iter1445->second.write(oprot);
 -        xfer += oprot->writeString(_iter1450->first);
 -        xfer += _iter1450->second.write(oprot);
++        xfer += oprot->writeString(_iter1469->first);
++        xfer += _iter1469->second.write(oprot);
        }
        xfer += oprot->writeMapEnd();
      }
@@@ -10833,17 -10833,17 +10833,17 @@@ uint32_t ThriftHiveMetastore_get_materi
          if (ftype == ::apache::thrift::protocol::T_MAP) {
            {
              (*(this->success)).clear();
-             uint32_t _size1446;
-             ::apache::thrift::protocol::TType _ktype1447;
-             ::apache::thrift::protocol::TType _vtype1448;
-             xfer += iprot->readMapBegin(_ktype1447, _vtype1448, _size1446);
-             uint32_t _i1450;
-             for (_i1450 = 0; _i1450 < _size1446; ++_i1450)
 -            uint32_t _size1451;
 -            ::apache::thrift::protocol::TType _ktype1452;
 -            ::apache::thrift::protocol::TType _vtype1453;
 -            xfer += iprot->readMapBegin(_ktype1452, _vtype1453, _size1451);
 -            uint32_t _i1455;
 -            for (_i1455 = 0; _i1455 < _size1451; ++_i1455)
++            uint32_t _size1470;
++            ::apache::thrift::protocol::TType _ktype1471;
++            ::apache::thrift::protocol::TType _vtype1472;
++            xfer += iprot->readMapBegin(_ktype1471, _vtype1472, _size1470);
++            uint32_t _i1474;
++            for (_i1474 = 0; _i1474 < _size1470; ++_i1474)
              {
-               std::string _key1451;
-               xfer += iprot->readString(_key1451);
-               Materialization& _val1452 = (*(this->success))[_key1451];
-               xfer += _val1452.read(iprot);
 -              std::string _key1456;
 -              xfer += iprot->readString(_key1456);
 -              Materialization& _val1457 = (*(this->success))[_key1456];
 -              xfer += _val1457.read(iprot);
++              std::string _key1475;
++              xfer += iprot->readString(_key1475);
++              Materialization& _val1476 = (*(this->success))[_key1475];
++              xfer += _val1476.read(iprot);
              }
              xfer += iprot->readMapEnd();
            }
@@@ -11304,14 -11304,14 +11304,14 @@@ uint32_t ThriftHiveMetastore_get_table_
          if (ftype == ::apache::thrift::protocol::T_LIST) {
            {
              this->success.clear();
-             uint32_t _size1453;
-             ::apache::thrift::protocol::TType _etype1456;
-             xfer += iprot->readListBegin(_etype1456, _size1453);
-             this->success.resize(_size1453);
-             uint32_t _i1457;
-             for (_i1457 = 0; _i1457 < _size1453; ++_i1457)
 -            uint32_t _size1458;
 -            ::apache::thrift::protocol::TType _etype1461;
 -            xfer += iprot->readListBegin(_etype1461, _si

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------


[23/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
index ec26cca..a83017b 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/ThriftHiveMetastore.php
@@ -1271,6 +1271,11 @@ interface ThriftHiveMetastoreIf extends \FacebookServiceIf {
    */
   public function flushCache();
   /**
+   * @param \metastore\WriteNotificationLogRequest $rqst
+   * @return \metastore\WriteNotificationLogResponse
+   */
+  public function add_write_notification_log(\metastore\WriteNotificationLogRequest $rqst);
+  /**
    * @param \metastore\CmRecycleRequest $request
    * @return \metastore\CmRecycleResponse
    * @throws \metastore\MetaException
@@ -10933,6 +10938,57 @@ class ThriftHiveMetastoreClient extends \FacebookServiceClient implements \metas
     return;
   }
 
+  public function add_write_notification_log(\metastore\WriteNotificationLogRequest $rqst)
+  {
+    $this->send_add_write_notification_log($rqst);
+    return $this->recv_add_write_notification_log();
+  }
+
+  public function send_add_write_notification_log(\metastore\WriteNotificationLogRequest $rqst)
+  {
+    $args = new \metastore\ThriftHiveMetastore_add_write_notification_log_args();
+    $args->rqst = $rqst;
+    $bin_accel = ($this->output_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_write_binary');
+    if ($bin_accel)
+    {
+      thrift_protocol_write_binary($this->output_, 'add_write_notification_log', TMessageType::CALL, $args, $this->seqid_, $this->output_->isStrictWrite());
+    }
+    else
+    {
+      $this->output_->writeMessageBegin('add_write_notification_log', TMessageType::CALL, $this->seqid_);
+      $args->write($this->output_);
+      $this->output_->writeMessageEnd();
+      $this->output_->getTransport()->flush();
+    }
+  }
+
+  public function recv_add_write_notification_log()
+  {
+    $bin_accel = ($this->input_ instanceof TBinaryProtocolAccelerated) && function_exists('thrift_protocol_read_binary');
+    if ($bin_accel) $result = thrift_protocol_read_binary($this->input_, '\metastore\ThriftHiveMetastore_add_write_notification_log_result', $this->input_->isStrictRead());
+    else
+    {
+      $rseqid = 0;
+      $fname = null;
+      $mtype = 0;
+
+      $this->input_->readMessageBegin($fname, $mtype, $rseqid);
+      if ($mtype == TMessageType::EXCEPTION) {
+        $x = new TApplicationException();
+        $x->read($this->input_);
+        $this->input_->readMessageEnd();
+        throw $x;
+      }
+      $result = new \metastore\ThriftHiveMetastore_add_write_notification_log_result();
+      $result->read($this->input_);
+      $this->input_->readMessageEnd();
+    }
+    if ($result->success !== null) {
+      return $result->success;
+    }
+    throw new \Exception("add_write_notification_log failed: unknown result");
+  }
+
   public function cm_recycle(\metastore\CmRecycleRequest $request)
   {
     $this->send_cm_recycle($request);
@@ -15440,14 +15496,14 @@ class ThriftHiveMetastore_get_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size820 = 0;
-            $_etype823 = 0;
-            $xfer += $input->readListBegin($_etype823, $_size820);
-            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
+            $_size841 = 0;
+            $_etype844 = 0;
+            $xfer += $input->readListBegin($_etype844, $_size841);
+            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
             {
-              $elem825 = null;
-              $xfer += $input->readString($elem825);
-              $this->success []= $elem825;
+              $elem846 = null;
+              $xfer += $input->readString($elem846);
+              $this->success []= $elem846;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15483,9 +15539,9 @@ class ThriftHiveMetastore_get_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter826)
+          foreach ($this->success as $iter847)
           {
-            $xfer += $output->writeString($iter826);
+            $xfer += $output->writeString($iter847);
           }
         }
         $output->writeListEnd();
@@ -15616,14 +15672,14 @@ class ThriftHiveMetastore_get_all_databases_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size827 = 0;
-            $_etype830 = 0;
-            $xfer += $input->readListBegin($_etype830, $_size827);
-            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
+            $_size848 = 0;
+            $_etype851 = 0;
+            $xfer += $input->readListBegin($_etype851, $_size848);
+            for ($_i852 = 0; $_i852 < $_size848; ++$_i852)
             {
-              $elem832 = null;
-              $xfer += $input->readString($elem832);
-              $this->success []= $elem832;
+              $elem853 = null;
+              $xfer += $input->readString($elem853);
+              $this->success []= $elem853;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -15659,9 +15715,9 @@ class ThriftHiveMetastore_get_all_databases_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter833)
+          foreach ($this->success as $iter854)
           {
-            $xfer += $output->writeString($iter833);
+            $xfer += $output->writeString($iter854);
           }
         }
         $output->writeListEnd();
@@ -16662,18 +16718,18 @@ class ThriftHiveMetastore_get_type_all_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size834 = 0;
-            $_ktype835 = 0;
-            $_vtype836 = 0;
-            $xfer += $input->readMapBegin($_ktype835, $_vtype836, $_size834);
-            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
+            $_size855 = 0;
+            $_ktype856 = 0;
+            $_vtype857 = 0;
+            $xfer += $input->readMapBegin($_ktype856, $_vtype857, $_size855);
+            for ($_i859 = 0; $_i859 < $_size855; ++$_i859)
             {
-              $key839 = '';
-              $val840 = new \metastore\Type();
-              $xfer += $input->readString($key839);
-              $val840 = new \metastore\Type();
-              $xfer += $val840->read($input);
-              $this->success[$key839] = $val840;
+              $key860 = '';
+              $val861 = new \metastore\Type();
+              $xfer += $input->readString($key860);
+              $val861 = new \metastore\Type();
+              $xfer += $val861->read($input);
+              $this->success[$key860] = $val861;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -16709,10 +16765,10 @@ class ThriftHiveMetastore_get_type_all_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter841 => $viter842)
+          foreach ($this->success as $kiter862 => $viter863)
           {
-            $xfer += $output->writeString($kiter841);
-            $xfer += $viter842->write($output);
+            $xfer += $output->writeString($kiter862);
+            $xfer += $viter863->write($output);
           }
         }
         $output->writeMapEnd();
@@ -16916,15 +16972,15 @@ class ThriftHiveMetastore_get_fields_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size843 = 0;
-            $_etype846 = 0;
-            $xfer += $input->readListBegin($_etype846, $_size843);
-            for ($_i847 = 0; $_i847 < $_size843; ++$_i847)
+            $_size864 = 0;
+            $_etype867 = 0;
+            $xfer += $input->readListBegin($_etype867, $_size864);
+            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
             {
-              $elem848 = null;
-              $elem848 = new \metastore\FieldSchema();
-              $xfer += $elem848->read($input);
-              $this->success []= $elem848;
+              $elem869 = null;
+              $elem869 = new \metastore\FieldSchema();
+              $xfer += $elem869->read($input);
+              $this->success []= $elem869;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16976,9 +17032,9 @@ class ThriftHiveMetastore_get_fields_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter849)
+          foreach ($this->success as $iter870)
           {
-            $xfer += $iter849->write($output);
+            $xfer += $iter870->write($output);
           }
         }
         $output->writeListEnd();
@@ -17220,15 +17276,15 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size850 = 0;
-            $_etype853 = 0;
-            $xfer += $input->readListBegin($_etype853, $_size850);
-            for ($_i854 = 0; $_i854 < $_size850; ++$_i854)
+            $_size871 = 0;
+            $_etype874 = 0;
+            $xfer += $input->readListBegin($_etype874, $_size871);
+            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
             {
-              $elem855 = null;
-              $elem855 = new \metastore\FieldSchema();
-              $xfer += $elem855->read($input);
-              $this->success []= $elem855;
+              $elem876 = null;
+              $elem876 = new \metastore\FieldSchema();
+              $xfer += $elem876->read($input);
+              $this->success []= $elem876;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17280,9 +17336,9 @@ class ThriftHiveMetastore_get_fields_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter856)
+          foreach ($this->success as $iter877)
           {
-            $xfer += $iter856->write($output);
+            $xfer += $iter877->write($output);
           }
         }
         $output->writeListEnd();
@@ -17496,15 +17552,15 @@ class ThriftHiveMetastore_get_schema_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size857 = 0;
-            $_etype860 = 0;
-            $xfer += $input->readListBegin($_etype860, $_size857);
-            for ($_i861 = 0; $_i861 < $_size857; ++$_i861)
+            $_size878 = 0;
+            $_etype881 = 0;
+            $xfer += $input->readListBegin($_etype881, $_size878);
+            for ($_i882 = 0; $_i882 < $_size878; ++$_i882)
             {
-              $elem862 = null;
-              $elem862 = new \metastore\FieldSchema();
-              $xfer += $elem862->read($input);
-              $this->success []= $elem862;
+              $elem883 = null;
+              $elem883 = new \metastore\FieldSchema();
+              $xfer += $elem883->read($input);
+              $this->success []= $elem883;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17556,9 +17612,9 @@ class ThriftHiveMetastore_get_schema_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter863)
+          foreach ($this->success as $iter884)
           {
-            $xfer += $iter863->write($output);
+            $xfer += $iter884->write($output);
           }
         }
         $output->writeListEnd();
@@ -17800,15 +17856,15 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size864 = 0;
-            $_etype867 = 0;
-            $xfer += $input->readListBegin($_etype867, $_size864);
-            for ($_i868 = 0; $_i868 < $_size864; ++$_i868)
+            $_size885 = 0;
+            $_etype888 = 0;
+            $xfer += $input->readListBegin($_etype888, $_size885);
+            for ($_i889 = 0; $_i889 < $_size885; ++$_i889)
             {
-              $elem869 = null;
-              $elem869 = new \metastore\FieldSchema();
-              $xfer += $elem869->read($input);
-              $this->success []= $elem869;
+              $elem890 = null;
+              $elem890 = new \metastore\FieldSchema();
+              $xfer += $elem890->read($input);
+              $this->success []= $elem890;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17860,9 +17916,9 @@ class ThriftHiveMetastore_get_schema_with_environment_context_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter870)
+          foreach ($this->success as $iter891)
           {
-            $xfer += $iter870->write($output);
+            $xfer += $iter891->write($output);
           }
         }
         $output->writeListEnd();
@@ -18534,15 +18590,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->primaryKeys = array();
-            $_size871 = 0;
-            $_etype874 = 0;
-            $xfer += $input->readListBegin($_etype874, $_size871);
-            for ($_i875 = 0; $_i875 < $_size871; ++$_i875)
+            $_size892 = 0;
+            $_etype895 = 0;
+            $xfer += $input->readListBegin($_etype895, $_size892);
+            for ($_i896 = 0; $_i896 < $_size892; ++$_i896)
             {
-              $elem876 = null;
-              $elem876 = new \metastore\SQLPrimaryKey();
-              $xfer += $elem876->read($input);
-              $this->primaryKeys []= $elem876;
+              $elem897 = null;
+              $elem897 = new \metastore\SQLPrimaryKey();
+              $xfer += $elem897->read($input);
+              $this->primaryKeys []= $elem897;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18552,15 +18608,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->foreignKeys = array();
-            $_size877 = 0;
-            $_etype880 = 0;
-            $xfer += $input->readListBegin($_etype880, $_size877);
-            for ($_i881 = 0; $_i881 < $_size877; ++$_i881)
+            $_size898 = 0;
+            $_etype901 = 0;
+            $xfer += $input->readListBegin($_etype901, $_size898);
+            for ($_i902 = 0; $_i902 < $_size898; ++$_i902)
             {
-              $elem882 = null;
-              $elem882 = new \metastore\SQLForeignKey();
-              $xfer += $elem882->read($input);
-              $this->foreignKeys []= $elem882;
+              $elem903 = null;
+              $elem903 = new \metastore\SQLForeignKey();
+              $xfer += $elem903->read($input);
+              $this->foreignKeys []= $elem903;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18570,15 +18626,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 4:
           if ($ftype == TType::LST) {
             $this->uniqueConstraints = array();
-            $_size883 = 0;
-            $_etype886 = 0;
-            $xfer += $input->readListBegin($_etype886, $_size883);
-            for ($_i887 = 0; $_i887 < $_size883; ++$_i887)
+            $_size904 = 0;
+            $_etype907 = 0;
+            $xfer += $input->readListBegin($_etype907, $_size904);
+            for ($_i908 = 0; $_i908 < $_size904; ++$_i908)
             {
-              $elem888 = null;
-              $elem888 = new \metastore\SQLUniqueConstraint();
-              $xfer += $elem888->read($input);
-              $this->uniqueConstraints []= $elem888;
+              $elem909 = null;
+              $elem909 = new \metastore\SQLUniqueConstraint();
+              $xfer += $elem909->read($input);
+              $this->uniqueConstraints []= $elem909;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18588,15 +18644,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->notNullConstraints = array();
-            $_size889 = 0;
-            $_etype892 = 0;
-            $xfer += $input->readListBegin($_etype892, $_size889);
-            for ($_i893 = 0; $_i893 < $_size889; ++$_i893)
+            $_size910 = 0;
+            $_etype913 = 0;
+            $xfer += $input->readListBegin($_etype913, $_size910);
+            for ($_i914 = 0; $_i914 < $_size910; ++$_i914)
             {
-              $elem894 = null;
-              $elem894 = new \metastore\SQLNotNullConstraint();
-              $xfer += $elem894->read($input);
-              $this->notNullConstraints []= $elem894;
+              $elem915 = null;
+              $elem915 = new \metastore\SQLNotNullConstraint();
+              $xfer += $elem915->read($input);
+              $this->notNullConstraints []= $elem915;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18606,15 +18662,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->defaultConstraints = array();
-            $_size895 = 0;
-            $_etype898 = 0;
-            $xfer += $input->readListBegin($_etype898, $_size895);
-            for ($_i899 = 0; $_i899 < $_size895; ++$_i899)
+            $_size916 = 0;
+            $_etype919 = 0;
+            $xfer += $input->readListBegin($_etype919, $_size916);
+            for ($_i920 = 0; $_i920 < $_size916; ++$_i920)
             {
-              $elem900 = null;
-              $elem900 = new \metastore\SQLDefaultConstraint();
-              $xfer += $elem900->read($input);
-              $this->defaultConstraints []= $elem900;
+              $elem921 = null;
+              $elem921 = new \metastore\SQLDefaultConstraint();
+              $xfer += $elem921->read($input);
+              $this->defaultConstraints []= $elem921;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18624,15 +18680,15 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
         case 7:
           if ($ftype == TType::LST) {
             $this->checkConstraints = array();
-            $_size901 = 0;
-            $_etype904 = 0;
-            $xfer += $input->readListBegin($_etype904, $_size901);
-            for ($_i905 = 0; $_i905 < $_size901; ++$_i905)
+            $_size922 = 0;
+            $_etype925 = 0;
+            $xfer += $input->readListBegin($_etype925, $_size922);
+            for ($_i926 = 0; $_i926 < $_size922; ++$_i926)
             {
-              $elem906 = null;
-              $elem906 = new \metastore\SQLCheckConstraint();
-              $xfer += $elem906->read($input);
-              $this->checkConstraints []= $elem906;
+              $elem927 = null;
+              $elem927 = new \metastore\SQLCheckConstraint();
+              $xfer += $elem927->read($input);
+              $this->checkConstraints []= $elem927;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18668,9 +18724,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->primaryKeys));
         {
-          foreach ($this->primaryKeys as $iter907)
+          foreach ($this->primaryKeys as $iter928)
           {
-            $xfer += $iter907->write($output);
+            $xfer += $iter928->write($output);
           }
         }
         $output->writeListEnd();
@@ -18685,9 +18741,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->foreignKeys));
         {
-          foreach ($this->foreignKeys as $iter908)
+          foreach ($this->foreignKeys as $iter929)
           {
-            $xfer += $iter908->write($output);
+            $xfer += $iter929->write($output);
           }
         }
         $output->writeListEnd();
@@ -18702,9 +18758,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->uniqueConstraints));
         {
-          foreach ($this->uniqueConstraints as $iter909)
+          foreach ($this->uniqueConstraints as $iter930)
           {
-            $xfer += $iter909->write($output);
+            $xfer += $iter930->write($output);
           }
         }
         $output->writeListEnd();
@@ -18719,9 +18775,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->notNullConstraints));
         {
-          foreach ($this->notNullConstraints as $iter910)
+          foreach ($this->notNullConstraints as $iter931)
           {
-            $xfer += $iter910->write($output);
+            $xfer += $iter931->write($output);
           }
         }
         $output->writeListEnd();
@@ -18736,9 +18792,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->defaultConstraints));
         {
-          foreach ($this->defaultConstraints as $iter911)
+          foreach ($this->defaultConstraints as $iter932)
           {
-            $xfer += $iter911->write($output);
+            $xfer += $iter932->write($output);
           }
         }
         $output->writeListEnd();
@@ -18753,9 +18809,9 @@ class ThriftHiveMetastore_create_table_with_constraints_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->checkConstraints));
         {
-          foreach ($this->checkConstraints as $iter912)
+          foreach ($this->checkConstraints as $iter933)
           {
-            $xfer += $iter912->write($output);
+            $xfer += $iter933->write($output);
           }
         }
         $output->writeListEnd();
@@ -20755,14 +20811,14 @@ class ThriftHiveMetastore_truncate_table_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size913 = 0;
-            $_etype916 = 0;
-            $xfer += $input->readListBegin($_etype916, $_size913);
-            for ($_i917 = 0; $_i917 < $_size913; ++$_i917)
+            $_size934 = 0;
+            $_etype937 = 0;
+            $xfer += $input->readListBegin($_etype937, $_size934);
+            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
             {
-              $elem918 = null;
-              $xfer += $input->readString($elem918);
-              $this->partNames []= $elem918;
+              $elem939 = null;
+              $xfer += $input->readString($elem939);
+              $this->partNames []= $elem939;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20800,9 +20856,9 @@ class ThriftHiveMetastore_truncate_table_args {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter919)
+          foreach ($this->partNames as $iter940)
           {
-            $xfer += $output->writeString($iter919);
+            $xfer += $output->writeString($iter940);
           }
         }
         $output->writeListEnd();
@@ -21053,14 +21109,14 @@ class ThriftHiveMetastore_get_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size920 = 0;
-            $_etype923 = 0;
-            $xfer += $input->readListBegin($_etype923, $_size920);
-            for ($_i924 = 0; $_i924 < $_size920; ++$_i924)
+            $_size941 = 0;
+            $_etype944 = 0;
+            $xfer += $input->readListBegin($_etype944, $_size941);
+            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
             {
-              $elem925 = null;
-              $xfer += $input->readString($elem925);
-              $this->success []= $elem925;
+              $elem946 = null;
+              $xfer += $input->readString($elem946);
+              $this->success []= $elem946;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21096,9 +21152,9 @@ class ThriftHiveMetastore_get_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter926)
+          foreach ($this->success as $iter947)
           {
-            $xfer += $output->writeString($iter926);
+            $xfer += $output->writeString($iter947);
           }
         }
         $output->writeListEnd();
@@ -21300,14 +21356,14 @@ class ThriftHiveMetastore_get_tables_by_type_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size927 = 0;
-            $_etype930 = 0;
-            $xfer += $input->readListBegin($_etype930, $_size927);
-            for ($_i931 = 0; $_i931 < $_size927; ++$_i931)
+            $_size948 = 0;
+            $_etype951 = 0;
+            $xfer += $input->readListBegin($_etype951, $_size948);
+            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
             {
-              $elem932 = null;
-              $xfer += $input->readString($elem932);
-              $this->success []= $elem932;
+              $elem953 = null;
+              $xfer += $input->readString($elem953);
+              $this->success []= $elem953;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21343,9 +21399,9 @@ class ThriftHiveMetastore_get_tables_by_type_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter933)
+          foreach ($this->success as $iter954)
           {
-            $xfer += $output->writeString($iter933);
+            $xfer += $output->writeString($iter954);
           }
         }
         $output->writeListEnd();
@@ -21501,14 +21557,14 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size934 = 0;
-            $_etype937 = 0;
-            $xfer += $input->readListBegin($_etype937, $_size934);
-            for ($_i938 = 0; $_i938 < $_size934; ++$_i938)
+            $_size955 = 0;
+            $_etype958 = 0;
+            $xfer += $input->readListBegin($_etype958, $_size955);
+            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
             {
-              $elem939 = null;
-              $xfer += $input->readString($elem939);
-              $this->success []= $elem939;
+              $elem960 = null;
+              $xfer += $input->readString($elem960);
+              $this->success []= $elem960;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21544,9 +21600,9 @@ class ThriftHiveMetastore_get_materialized_views_for_rewriting_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter940)
+          foreach ($this->success as $iter961)
           {
-            $xfer += $output->writeString($iter940);
+            $xfer += $output->writeString($iter961);
           }
         }
         $output->writeListEnd();
@@ -21651,14 +21707,14 @@ class ThriftHiveMetastore_get_table_meta_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->tbl_types = array();
-            $_size941 = 0;
-            $_etype944 = 0;
-            $xfer += $input->readListBegin($_etype944, $_size941);
-            for ($_i945 = 0; $_i945 < $_size941; ++$_i945)
+            $_size962 = 0;
+            $_etype965 = 0;
+            $xfer += $input->readListBegin($_etype965, $_size962);
+            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
             {
-              $elem946 = null;
-              $xfer += $input->readString($elem946);
-              $this->tbl_types []= $elem946;
+              $elem967 = null;
+              $xfer += $input->readString($elem967);
+              $this->tbl_types []= $elem967;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21696,9 +21752,9 @@ class ThriftHiveMetastore_get_table_meta_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_types));
         {
-          foreach ($this->tbl_types as $iter947)
+          foreach ($this->tbl_types as $iter968)
           {
-            $xfer += $output->writeString($iter947);
+            $xfer += $output->writeString($iter968);
           }
         }
         $output->writeListEnd();
@@ -21775,15 +21831,15 @@ class ThriftHiveMetastore_get_table_meta_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size948 = 0;
-            $_etype951 = 0;
-            $xfer += $input->readListBegin($_etype951, $_size948);
-            for ($_i952 = 0; $_i952 < $_size948; ++$_i952)
+            $_size969 = 0;
+            $_etype972 = 0;
+            $xfer += $input->readListBegin($_etype972, $_size969);
+            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
             {
-              $elem953 = null;
-              $elem953 = new \metastore\TableMeta();
-              $xfer += $elem953->read($input);
-              $this->success []= $elem953;
+              $elem974 = null;
+              $elem974 = new \metastore\TableMeta();
+              $xfer += $elem974->read($input);
+              $this->success []= $elem974;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21819,9 +21875,9 @@ class ThriftHiveMetastore_get_table_meta_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter954)
+          foreach ($this->success as $iter975)
           {
-            $xfer += $iter954->write($output);
+            $xfer += $iter975->write($output);
           }
         }
         $output->writeListEnd();
@@ -21977,14 +22033,14 @@ class ThriftHiveMetastore_get_all_tables_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size955 = 0;
-            $_etype958 = 0;
-            $xfer += $input->readListBegin($_etype958, $_size955);
-            for ($_i959 = 0; $_i959 < $_size955; ++$_i959)
+            $_size976 = 0;
+            $_etype979 = 0;
+            $xfer += $input->readListBegin($_etype979, $_size976);
+            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
             {
-              $elem960 = null;
-              $xfer += $input->readString($elem960);
-              $this->success []= $elem960;
+              $elem981 = null;
+              $xfer += $input->readString($elem981);
+              $this->success []= $elem981;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22020,9 +22076,9 @@ class ThriftHiveMetastore_get_all_tables_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter961)
+          foreach ($this->success as $iter982)
           {
-            $xfer += $output->writeString($iter961);
+            $xfer += $output->writeString($iter982);
           }
         }
         $output->writeListEnd();
@@ -22337,14 +22393,14 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size962 = 0;
-            $_etype965 = 0;
-            $xfer += $input->readListBegin($_etype965, $_size962);
-            for ($_i966 = 0; $_i966 < $_size962; ++$_i966)
+            $_size983 = 0;
+            $_etype986 = 0;
+            $xfer += $input->readListBegin($_etype986, $_size983);
+            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
             {
-              $elem967 = null;
-              $xfer += $input->readString($elem967);
-              $this->tbl_names []= $elem967;
+              $elem988 = null;
+              $xfer += $input->readString($elem988);
+              $this->tbl_names []= $elem988;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22377,9 +22433,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter968)
+          foreach ($this->tbl_names as $iter989)
           {
-            $xfer += $output->writeString($iter968);
+            $xfer += $output->writeString($iter989);
           }
         }
         $output->writeListEnd();
@@ -22444,15 +22500,15 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size969 = 0;
-            $_etype972 = 0;
-            $xfer += $input->readListBegin($_etype972, $_size969);
-            for ($_i973 = 0; $_i973 < $_size969; ++$_i973)
+            $_size990 = 0;
+            $_etype993 = 0;
+            $xfer += $input->readListBegin($_etype993, $_size990);
+            for ($_i994 = 0; $_i994 < $_size990; ++$_i994)
             {
-              $elem974 = null;
-              $elem974 = new \metastore\Table();
-              $xfer += $elem974->read($input);
-              $this->success []= $elem974;
+              $elem995 = null;
+              $elem995 = new \metastore\Table();
+              $xfer += $elem995->read($input);
+              $this->success []= $elem995;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22480,9 +22536,9 @@ class ThriftHiveMetastore_get_table_objects_by_name_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter975)
+          foreach ($this->success as $iter996)
           {
-            $xfer += $iter975->write($output);
+            $xfer += $iter996->write($output);
           }
         }
         $output->writeListEnd();
@@ -23009,14 +23065,14 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
         case 2:
           if ($ftype == TType::LST) {
             $this->tbl_names = array();
-            $_size976 = 0;
-            $_etype979 = 0;
-            $xfer += $input->readListBegin($_etype979, $_size976);
-            for ($_i980 = 0; $_i980 < $_size976; ++$_i980)
+            $_size997 = 0;
+            $_etype1000 = 0;
+            $xfer += $input->readListBegin($_etype1000, $_size997);
+            for ($_i1001 = 0; $_i1001 < $_size997; ++$_i1001)
             {
-              $elem981 = null;
-              $xfer += $input->readString($elem981);
-              $this->tbl_names []= $elem981;
+              $elem1002 = null;
+              $xfer += $input->readString($elem1002);
+              $this->tbl_names []= $elem1002;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23049,9 +23105,9 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_args {
       {
         $output->writeListBegin(TType::STRING, count($this->tbl_names));
         {
-          foreach ($this->tbl_names as $iter982)
+          foreach ($this->tbl_names as $iter1003)
           {
-            $xfer += $output->writeString($iter982);
+            $xfer += $output->writeString($iter1003);
           }
         }
         $output->writeListEnd();
@@ -23156,18 +23212,18 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size983 = 0;
-            $_ktype984 = 0;
-            $_vtype985 = 0;
-            $xfer += $input->readMapBegin($_ktype984, $_vtype985, $_size983);
-            for ($_i987 = 0; $_i987 < $_size983; ++$_i987)
+            $_size1004 = 0;
+            $_ktype1005 = 0;
+            $_vtype1006 = 0;
+            $xfer += $input->readMapBegin($_ktype1005, $_vtype1006, $_size1004);
+            for ($_i1008 = 0; $_i1008 < $_size1004; ++$_i1008)
             {
-              $key988 = '';
-              $val989 = new \metastore\Materialization();
-              $xfer += $input->readString($key988);
-              $val989 = new \metastore\Materialization();
-              $xfer += $val989->read($input);
-              $this->success[$key988] = $val989;
+              $key1009 = '';
+              $val1010 = new \metastore\Materialization();
+              $xfer += $input->readString($key1009);
+              $val1010 = new \metastore\Materialization();
+              $xfer += $val1010->read($input);
+              $this->success[$key1009] = $val1010;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -23219,10 +23275,10 @@ class ThriftHiveMetastore_get_materialization_invalidation_info_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $kiter990 => $viter991)
+          foreach ($this->success as $kiter1011 => $viter1012)
           {
-            $xfer += $output->writeString($kiter990);
-            $xfer += $viter991->write($output);
+            $xfer += $output->writeString($kiter1011);
+            $xfer += $viter1012->write($output);
           }
         }
         $output->writeMapEnd();
@@ -23734,14 +23790,14 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size992 = 0;
-            $_etype995 = 0;
-            $xfer += $input->readListBegin($_etype995, $_size992);
-            for ($_i996 = 0; $_i996 < $_size992; ++$_i996)
+            $_size1013 = 0;
+            $_etype1016 = 0;
+            $xfer += $input->readListBegin($_etype1016, $_size1013);
+            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
             {
-              $elem997 = null;
-              $xfer += $input->readString($elem997);
-              $this->success []= $elem997;
+              $elem1018 = null;
+              $xfer += $input->readString($elem1018);
+              $this->success []= $elem1018;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23793,9 +23849,9 @@ class ThriftHiveMetastore_get_table_names_by_filter_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter998)
+          foreach ($this->success as $iter1019)
           {
-            $xfer += $output->writeString($iter998);
+            $xfer += $output->writeString($iter1019);
           }
         }
         $output->writeListEnd();
@@ -25108,15 +25164,15 @@ class ThriftHiveMetastore_add_partitions_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size999 = 0;
-            $_etype1002 = 0;
-            $xfer += $input->readListBegin($_etype1002, $_size999);
-            for ($_i1003 = 0; $_i1003 < $_size999; ++$_i1003)
+            $_size1020 = 0;
+            $_etype1023 = 0;
+            $xfer += $input->readListBegin($_etype1023, $_size1020);
+            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
             {
-              $elem1004 = null;
-              $elem1004 = new \metastore\Partition();
-              $xfer += $elem1004->read($input);
-              $this->new_parts []= $elem1004;
+              $elem1025 = null;
+              $elem1025 = new \metastore\Partition();
+              $xfer += $elem1025->read($input);
+              $this->new_parts []= $elem1025;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25144,9 +25200,9 @@ class ThriftHiveMetastore_add_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1005)
+          foreach ($this->new_parts as $iter1026)
           {
-            $xfer += $iter1005->write($output);
+            $xfer += $iter1026->write($output);
           }
         }
         $output->writeListEnd();
@@ -25361,15 +25417,15 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1006 = 0;
-            $_etype1009 = 0;
-            $xfer += $input->readListBegin($_etype1009, $_size1006);
-            for ($_i1010 = 0; $_i1010 < $_size1006; ++$_i1010)
+            $_size1027 = 0;
+            $_etype1030 = 0;
+            $xfer += $input->readListBegin($_etype1030, $_size1027);
+            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
             {
-              $elem1011 = null;
-              $elem1011 = new \metastore\PartitionSpec();
-              $xfer += $elem1011->read($input);
-              $this->new_parts []= $elem1011;
+              $elem1032 = null;
+              $elem1032 = new \metastore\PartitionSpec();
+              $xfer += $elem1032->read($input);
+              $this->new_parts []= $elem1032;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25397,9 +25453,9 @@ class ThriftHiveMetastore_add_partitions_pspec_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1012)
+          foreach ($this->new_parts as $iter1033)
           {
-            $xfer += $iter1012->write($output);
+            $xfer += $iter1033->write($output);
           }
         }
         $output->writeListEnd();
@@ -25649,14 +25705,14 @@ class ThriftHiveMetastore_append_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1013 = 0;
-            $_etype1016 = 0;
-            $xfer += $input->readListBegin($_etype1016, $_size1013);
-            for ($_i1017 = 0; $_i1017 < $_size1013; ++$_i1017)
+            $_size1034 = 0;
+            $_etype1037 = 0;
+            $xfer += $input->readListBegin($_etype1037, $_size1034);
+            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
             {
-              $elem1018 = null;
-              $xfer += $input->readString($elem1018);
-              $this->part_vals []= $elem1018;
+              $elem1039 = null;
+              $xfer += $input->readString($elem1039);
+              $this->part_vals []= $elem1039;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25694,9 +25750,9 @@ class ThriftHiveMetastore_append_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1019)
+          foreach ($this->part_vals as $iter1040)
           {
-            $xfer += $output->writeString($iter1019);
+            $xfer += $output->writeString($iter1040);
           }
         }
         $output->writeListEnd();
@@ -26198,14 +26254,14 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1020 = 0;
-            $_etype1023 = 0;
-            $xfer += $input->readListBegin($_etype1023, $_size1020);
-            for ($_i1024 = 0; $_i1024 < $_size1020; ++$_i1024)
+            $_size1041 = 0;
+            $_etype1044 = 0;
+            $xfer += $input->readListBegin($_etype1044, $_size1041);
+            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
             {
-              $elem1025 = null;
-              $xfer += $input->readString($elem1025);
-              $this->part_vals []= $elem1025;
+              $elem1046 = null;
+              $xfer += $input->readString($elem1046);
+              $this->part_vals []= $elem1046;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26251,9 +26307,9 @@ class ThriftHiveMetastore_append_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1026)
+          foreach ($this->part_vals as $iter1047)
           {
-            $xfer += $output->writeString($iter1026);
+            $xfer += $output->writeString($iter1047);
           }
         }
         $output->writeListEnd();
@@ -27107,14 +27163,14 @@ class ThriftHiveMetastore_drop_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1027 = 0;
-            $_etype1030 = 0;
-            $xfer += $input->readListBegin($_etype1030, $_size1027);
-            for ($_i1031 = 0; $_i1031 < $_size1027; ++$_i1031)
+            $_size1048 = 0;
+            $_etype1051 = 0;
+            $xfer += $input->readListBegin($_etype1051, $_size1048);
+            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
             {
-              $elem1032 = null;
-              $xfer += $input->readString($elem1032);
-              $this->part_vals []= $elem1032;
+              $elem1053 = null;
+              $xfer += $input->readString($elem1053);
+              $this->part_vals []= $elem1053;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27159,9 +27215,9 @@ class ThriftHiveMetastore_drop_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1033)
+          foreach ($this->part_vals as $iter1054)
           {
-            $xfer += $output->writeString($iter1033);
+            $xfer += $output->writeString($iter1054);
           }
         }
         $output->writeListEnd();
@@ -27414,14 +27470,14 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1034 = 0;
-            $_etype1037 = 0;
-            $xfer += $input->readListBegin($_etype1037, $_size1034);
-            for ($_i1038 = 0; $_i1038 < $_size1034; ++$_i1038)
+            $_size1055 = 0;
+            $_etype1058 = 0;
+            $xfer += $input->readListBegin($_etype1058, $_size1055);
+            for ($_i1059 = 0; $_i1059 < $_size1055; ++$_i1059)
             {
-              $elem1039 = null;
-              $xfer += $input->readString($elem1039);
-              $this->part_vals []= $elem1039;
+              $elem1060 = null;
+              $xfer += $input->readString($elem1060);
+              $this->part_vals []= $elem1060;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27474,9 +27530,9 @@ class ThriftHiveMetastore_drop_partition_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1040)
+          foreach ($this->part_vals as $iter1061)
           {
-            $xfer += $output->writeString($iter1040);
+            $xfer += $output->writeString($iter1061);
           }
         }
         $output->writeListEnd();
@@ -28490,14 +28546,14 @@ class ThriftHiveMetastore_get_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1041 = 0;
-            $_etype1044 = 0;
-            $xfer += $input->readListBegin($_etype1044, $_size1041);
-            for ($_i1045 = 0; $_i1045 < $_size1041; ++$_i1045)
+            $_size1062 = 0;
+            $_etype1065 = 0;
+            $xfer += $input->readListBegin($_etype1065, $_size1062);
+            for ($_i1066 = 0; $_i1066 < $_size1062; ++$_i1066)
             {
-              $elem1046 = null;
-              $xfer += $input->readString($elem1046);
-              $this->part_vals []= $elem1046;
+              $elem1067 = null;
+              $xfer += $input->readString($elem1067);
+              $this->part_vals []= $elem1067;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -28535,9 +28591,9 @@ class ThriftHiveMetastore_get_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1047)
+          foreach ($this->part_vals as $iter1068)
           {
-            $xfer += $output->writeString($iter1047);
+            $xfer += $output->writeString($iter1068);
           }
         }
         $output->writeListEnd();
@@ -28779,17 +28835,17 @@ class ThriftHiveMetastore_exchange_partition_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1048 = 0;
-            $_ktype1049 = 0;
-            $_vtype1050 = 0;
-            $xfer += $input->readMapBegin($_ktype1049, $_vtype1050, $_size1048);
-            for ($_i1052 = 0; $_i1052 < $_size1048; ++$_i1052)
+            $_size1069 = 0;
+            $_ktype1070 = 0;
+            $_vtype1071 = 0;
+            $xfer += $input->readMapBegin($_ktype1070, $_vtype1071, $_size1069);
+            for ($_i1073 = 0; $_i1073 < $_size1069; ++$_i1073)
             {
-              $key1053 = '';
-              $val1054 = '';
-              $xfer += $input->readString($key1053);
-              $xfer += $input->readString($val1054);
-              $this->partitionSpecs[$key1053] = $val1054;
+              $key1074 = '';
+              $val1075 = '';
+              $xfer += $input->readString($key1074);
+              $xfer += $input->readString($val1075);
+              $this->partitionSpecs[$key1074] = $val1075;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -28845,10 +28901,10 @@ class ThriftHiveMetastore_exchange_partition_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1055 => $viter1056)
+          foreach ($this->partitionSpecs as $kiter1076 => $viter1077)
           {
-            $xfer += $output->writeString($kiter1055);
-            $xfer += $output->writeString($viter1056);
+            $xfer += $output->writeString($kiter1076);
+            $xfer += $output->writeString($viter1077);
           }
         }
         $output->writeMapEnd();
@@ -29160,17 +29216,17 @@ class ThriftHiveMetastore_exchange_partitions_args {
         case 1:
           if ($ftype == TType::MAP) {
             $this->partitionSpecs = array();
-            $_size1057 = 0;
-            $_ktype1058 = 0;
-            $_vtype1059 = 0;
-            $xfer += $input->readMapBegin($_ktype1058, $_vtype1059, $_size1057);
-            for ($_i1061 = 0; $_i1061 < $_size1057; ++$_i1061)
+            $_size1078 = 0;
+            $_ktype1079 = 0;
+            $_vtype1080 = 0;
+            $xfer += $input->readMapBegin($_ktype1079, $_vtype1080, $_size1078);
+            for ($_i1082 = 0; $_i1082 < $_size1078; ++$_i1082)
             {
-              $key1062 = '';
-              $val1063 = '';
-              $xfer += $input->readString($key1062);
-              $xfer += $input->readString($val1063);
-              $this->partitionSpecs[$key1062] = $val1063;
+              $key1083 = '';
+              $val1084 = '';
+              $xfer += $input->readString($key1083);
+              $xfer += $input->readString($val1084);
+              $this->partitionSpecs[$key1083] = $val1084;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -29226,10 +29282,10 @@ class ThriftHiveMetastore_exchange_partitions_args {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->partitionSpecs));
         {
-          foreach ($this->partitionSpecs as $kiter1064 => $viter1065)
+          foreach ($this->partitionSpecs as $kiter1085 => $viter1086)
           {
-            $xfer += $output->writeString($kiter1064);
-            $xfer += $output->writeString($viter1065);
+            $xfer += $output->writeString($kiter1085);
+            $xfer += $output->writeString($viter1086);
           }
         }
         $output->writeMapEnd();
@@ -29362,15 +29418,15 @@ class ThriftHiveMetastore_exchange_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1066 = 0;
-            $_etype1069 = 0;
-            $xfer += $input->readListBegin($_etype1069, $_size1066);
-            for ($_i1070 = 0; $_i1070 < $_size1066; ++$_i1070)
+            $_size1087 = 0;
+            $_etype1090 = 0;
+            $xfer += $input->readListBegin($_etype1090, $_size1087);
+            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
             {
-              $elem1071 = null;
-              $elem1071 = new \metastore\Partition();
-              $xfer += $elem1071->read($input);
-              $this->success []= $elem1071;
+              $elem1092 = null;
+              $elem1092 = new \metastore\Partition();
+              $xfer += $elem1092->read($input);
+              $this->success []= $elem1092;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29430,9 +29486,9 @@ class ThriftHiveMetastore_exchange_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1072)
+          foreach ($this->success as $iter1093)
           {
-            $xfer += $iter1072->write($output);
+            $xfer += $iter1093->write($output);
           }
         }
         $output->writeListEnd();
@@ -29578,14 +29634,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1073 = 0;
-            $_etype1076 = 0;
-            $xfer += $input->readListBegin($_etype1076, $_size1073);
-            for ($_i1077 = 0; $_i1077 < $_size1073; ++$_i1077)
+            $_size1094 = 0;
+            $_etype1097 = 0;
+            $xfer += $input->readListBegin($_etype1097, $_size1094);
+            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
             {
-              $elem1078 = null;
-              $xfer += $input->readString($elem1078);
-              $this->part_vals []= $elem1078;
+              $elem1099 = null;
+              $xfer += $input->readString($elem1099);
+              $this->part_vals []= $elem1099;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29602,14 +29658,14 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1079 = 0;
-            $_etype1082 = 0;
-            $xfer += $input->readListBegin($_etype1082, $_size1079);
-            for ($_i1083 = 0; $_i1083 < $_size1079; ++$_i1083)
+            $_size1100 = 0;
+            $_etype1103 = 0;
+            $xfer += $input->readListBegin($_etype1103, $_size1100);
+            for ($_i1104 = 0; $_i1104 < $_size1100; ++$_i1104)
             {
-              $elem1084 = null;
-              $xfer += $input->readString($elem1084);
-              $this->group_names []= $elem1084;
+              $elem1105 = null;
+              $xfer += $input->readString($elem1105);
+              $this->group_names []= $elem1105;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29647,9 +29703,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1085)
+          foreach ($this->part_vals as $iter1106)
           {
-            $xfer += $output->writeString($iter1085);
+            $xfer += $output->writeString($iter1106);
           }
         }
         $output->writeListEnd();
@@ -29669,9 +29725,9 @@ class ThriftHiveMetastore_get_partition_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1086)
+          foreach ($this->group_names as $iter1107)
           {
-            $xfer += $output->writeString($iter1086);
+            $xfer += $output->writeString($iter1107);
           }
         }
         $output->writeListEnd();
@@ -30262,15 +30318,15 @@ class ThriftHiveMetastore_get_partitions_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1087 = 0;
-            $_etype1090 = 0;
-            $xfer += $input->readListBegin($_etype1090, $_size1087);
-            for ($_i1091 = 0; $_i1091 < $_size1087; ++$_i1091)
+            $_size1108 = 0;
+            $_etype1111 = 0;
+            $xfer += $input->readListBegin($_etype1111, $_size1108);
+            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
             {
-              $elem1092 = null;
-              $elem1092 = new \metastore\Partition();
-              $xfer += $elem1092->read($input);
-              $this->success []= $elem1092;
+              $elem1113 = null;
+              $elem1113 = new \metastore\Partition();
+              $xfer += $elem1113->read($input);
+              $this->success []= $elem1113;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30314,9 +30370,9 @@ class ThriftHiveMetastore_get_partitions_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1093)
+          foreach ($this->success as $iter1114)
           {
-            $xfer += $iter1093->write($output);
+            $xfer += $iter1114->write($output);
           }
         }
         $output->writeListEnd();
@@ -30462,14 +30518,14 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
         case 5:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1094 = 0;
-            $_etype1097 = 0;
-            $xfer += $input->readListBegin($_etype1097, $_size1094);
-            for ($_i1098 = 0; $_i1098 < $_size1094; ++$_i1098)
+            $_size1115 = 0;
+            $_etype1118 = 0;
+            $xfer += $input->readListBegin($_etype1118, $_size1115);
+            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
             {
-              $elem1099 = null;
-              $xfer += $input->readString($elem1099);
-              $this->group_names []= $elem1099;
+              $elem1120 = null;
+              $xfer += $input->readString($elem1120);
+              $this->group_names []= $elem1120;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30517,9 +30573,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1100)
+          foreach ($this->group_names as $iter1121)
           {
-            $xfer += $output->writeString($iter1100);
+            $xfer += $output->writeString($iter1121);
           }
         }
         $output->writeListEnd();
@@ -30608,15 +30664,15 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1101 = 0;
-            $_etype1104 = 0;
-            $xfer += $input->readListBegin($_etype1104, $_size1101);
-            for ($_i1105 = 0; $_i1105 < $_size1101; ++$_i1105)
+            $_size1122 = 0;
+            $_etype1125 = 0;
+            $xfer += $input->readListBegin($_etype1125, $_size1122);
+            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
             {
-              $elem1106 = null;
-              $elem1106 = new \metastore\Partition();
-              $xfer += $elem1106->read($input);
-              $this->success []= $elem1106;
+              $elem1127 = null;
+              $elem1127 = new \metastore\Partition();
+              $xfer += $elem1127->read($input);
+              $this->success []= $elem1127;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30660,9 +30716,9 @@ class ThriftHiveMetastore_get_partitions_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1107)
+          foreach ($this->success as $iter1128)
           {
-            $xfer += $iter1107->write($output);
+            $xfer += $iter1128->write($output);
           }
         }
         $output->writeListEnd();
@@ -30882,15 +30938,15 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1108 = 0;
-            $_etype1111 = 0;
-            $xfer += $input->readListBegin($_etype1111, $_size1108);
-            for ($_i1112 = 0; $_i1112 < $_size1108; ++$_i1112)
+            $_size1129 = 0;
+            $_etype1132 = 0;
+            $xfer += $input->readListBegin($_etype1132, $_size1129);
+            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
             {
-              $elem1113 = null;
-              $elem1113 = new \metastore\PartitionSpec();
-              $xfer += $elem1113->read($input);
-              $this->success []= $elem1113;
+              $elem1134 = null;
+              $elem1134 = new \metastore\PartitionSpec();
+              $xfer += $elem1134->read($input);
+              $this->success []= $elem1134;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -30934,9 +30990,9 @@ class ThriftHiveMetastore_get_partitions_pspec_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1114)
+          foreach ($this->success as $iter1135)
           {
-            $xfer += $iter1114->write($output);
+            $xfer += $iter1135->write($output);
           }
         }
         $output->writeListEnd();
@@ -31155,14 +31211,14 @@ class ThriftHiveMetastore_get_partition_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1115 = 0;
-            $_etype1118 = 0;
-            $xfer += $input->readListBegin($_etype1118, $_size1115);
-            for ($_i1119 = 0; $_i1119 < $_size1115; ++$_i1119)
+            $_size1136 = 0;
+            $_etype1139 = 0;
+            $xfer += $input->readListBegin($_etype1139, $_size1136);
+            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
             {
-              $elem1120 = null;
-              $xfer += $input->readString($elem1120);
-              $this->success []= $elem1120;
+              $elem1141 = null;
+              $xfer += $input->readString($elem1141);
+              $this->success []= $elem1141;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31206,9 +31262,9 @@ class ThriftHiveMetastore_get_partition_names_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1121)
+          foreach ($this->success as $iter1142)
           {
-            $xfer += $output->writeString($iter1121);
+            $xfer += $output->writeString($iter1142);
           }
         }
         $output->writeListEnd();
@@ -31539,14 +31595,14 @@ class ThriftHiveMetastore_get_partitions_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1122 = 0;
-            $_etype1125 = 0;
-            $xfer += $input->readListBegin($_etype1125, $_size1122);
-            for ($_i1126 = 0; $_i1126 < $_size1122; ++$_i1126)
+            $_size1143 = 0;
+            $_etype1146 = 0;
+            $xfer += $input->readListBegin($_etype1146, $_size1143);
+            for ($_i1147 = 0; $_i1147 < $_size1143; ++$_i1147)
             {
-              $elem1127 = null;
-              $xfer += $input->readString($elem1127);
-              $this->part_vals []= $elem1127;
+              $elem1148 = null;
+              $xfer += $input->readString($elem1148);
+              $this->part_vals []= $elem1148;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31591,9 +31647,9 @@ class ThriftHiveMetastore_get_partitions_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1128)
+          foreach ($this->part_vals as $iter1149)
           {
-            $xfer += $output->writeString($iter1128);
+            $xfer += $output->writeString($iter1149);
           }
         }
         $output->writeListEnd();
@@ -31687,15 +31743,15 @@ class ThriftHiveMetastore_get_partitions_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1129 = 0;
-            $_etype1132 = 0;
-            $xfer += $input->readListBegin($_etype1132, $_size1129);
-            for ($_i1133 = 0; $_i1133 < $_size1129; ++$_i1133)
+            $_size1150 = 0;
+            $_etype1153 = 0;
+            $xfer += $input->readListBegin($_etype1153, $_size1150);
+            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
             {
-              $elem1134 = null;
-              $elem1134 = new \metastore\Partition();
-              $xfer += $elem1134->read($input);
-              $this->success []= $elem1134;
+              $elem1155 = null;
+              $elem1155 = new \metastore\Partition();
+              $xfer += $elem1155->read($input);
+              $this->success []= $elem1155;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31739,9 +31795,9 @@ class ThriftHiveMetastore_get_partitions_ps_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1135)
+          foreach ($this->success as $iter1156)
           {
-            $xfer += $iter1135->write($output);
+            $xfer += $iter1156->write($output);
           }
         }
         $output->writeListEnd();
@@ -31888,14 +31944,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1136 = 0;
-            $_etype1139 = 0;
-            $xfer += $input->readListBegin($_etype1139, $_size1136);
-            for ($_i1140 = 0; $_i1140 < $_size1136; ++$_i1140)
+            $_size1157 = 0;
+            $_etype1160 = 0;
+            $xfer += $input->readListBegin($_etype1160, $_size1157);
+            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
             {
-              $elem1141 = null;
-              $xfer += $input->readString($elem1141);
-              $this->part_vals []= $elem1141;
+              $elem1162 = null;
+              $xfer += $input->readString($elem1162);
+              $this->part_vals []= $elem1162;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31919,14 +31975,14 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
         case 6:
           if ($ftype == TType::LST) {
             $this->group_names = array();
-            $_size1142 = 0;
-            $_etype1145 = 0;
-            $xfer += $input->readListBegin($_etype1145, $_size1142);
-            for ($_i1146 = 0; $_i1146 < $_size1142; ++$_i1146)
+            $_size1163 = 0;
+            $_etype1166 = 0;
+            $xfer += $input->readListBegin($_etype1166, $_size1163);
+            for ($_i1167 = 0; $_i1167 < $_size1163; ++$_i1167)
             {
-              $elem1147 = null;
-              $xfer += $input->readString($elem1147);
-              $this->group_names []= $elem1147;
+              $elem1168 = null;
+              $xfer += $input->readString($elem1168);
+              $this->group_names []= $elem1168;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -31964,9 +32020,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1148)
+          foreach ($this->part_vals as $iter1169)
           {
-            $xfer += $output->writeString($iter1148);
+            $xfer += $output->writeString($iter1169);
           }
         }
         $output->writeListEnd();
@@ -31991,9 +32047,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_args {
       {
         $output->writeListBegin(TType::STRING, count($this->group_names));
         {
-          foreach ($this->group_names as $iter1149)
+          foreach ($this->group_names as $iter1170)
           {
-            $xfer += $output->writeString($iter1149);
+            $xfer += $output->writeString($iter1170);
           }
         }
         $output->writeListEnd();
@@ -32082,15 +32138,15 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1150 = 0;
-            $_etype1153 = 0;
-            $xfer += $input->readListBegin($_etype1153, $_size1150);
-            for ($_i1154 = 0; $_i1154 < $_size1150; ++$_i1154)
+            $_size1171 = 0;
+            $_etype1174 = 0;
+            $xfer += $input->readListBegin($_etype1174, $_size1171);
+            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
             {
-              $elem1155 = null;
-              $elem1155 = new \metastore\Partition();
-              $xfer += $elem1155->read($input);
-              $this->success []= $elem1155;
+              $elem1176 = null;
+              $elem1176 = new \metastore\Partition();
+              $xfer += $elem1176->read($input);
+              $this->success []= $elem1176;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32134,9 +32190,9 @@ class ThriftHiveMetastore_get_partitions_ps_with_auth_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1156)
+          foreach ($this->success as $iter1177)
           {
-            $xfer += $iter1156->write($output);
+            $xfer += $iter1177->write($output);
           }
         }
         $output->writeListEnd();
@@ -32257,14 +32313,14 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1157 = 0;
-            $_etype1160 = 0;
-            $xfer += $input->readListBegin($_etype1160, $_size1157);
-            for ($_i1161 = 0; $_i1161 < $_size1157; ++$_i1161)
+            $_size1178 = 0;
+            $_etype1181 = 0;
+            $xfer += $input->readListBegin($_etype1181, $_size1178);
+            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
             {
-              $elem1162 = null;
-              $xfer += $input->readString($elem1162);
-              $this->part_vals []= $elem1162;
+              $elem1183 = null;
+              $xfer += $input->readString($elem1183);
+              $this->part_vals []= $elem1183;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32309,9 +32365,9 @@ class ThriftHiveMetastore_get_partition_names_ps_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1163)
+          foreach ($this->part_vals as $iter1184)
           {
-            $xfer += $output->writeString($iter1163);
+            $xfer += $output->writeString($iter1184);
           }
         }
         $output->writeListEnd();
@@ -32404,14 +32460,14 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1164 = 0;
-            $_etype1167 = 0;
-            $xfer += $input->readListBegin($_etype1167, $_size1164);
-            for ($_i1168 = 0; $_i1168 < $_size1164; ++$_i1168)
+            $_size1185 = 0;
+            $_etype1188 = 0;
+            $xfer += $input->readListBegin($_etype1188, $_size1185);
+            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
             {
-              $elem1169 = null;
-              $xfer += $input->readString($elem1169);
-              $this->success []= $elem1169;
+              $elem1190 = null;
+              $xfer += $input->readString($elem1190);
+              $this->success []= $elem1190;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32455,9 +32511,9 @@ class ThriftHiveMetastore_get_partition_names_ps_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1170)
+          foreach ($this->success as $iter1191)
           {
-            $xfer += $output->writeString($iter1170);
+            $xfer += $output->writeString($iter1191);
           }
         }
         $output->writeListEnd();
@@ -32700,15 +32756,15 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1171 = 0;
-            $_etype1174 = 0;
-            $xfer += $input->readListBegin($_etype1174, $_size1171);
-            for ($_i1175 = 0; $_i1175 < $_size1171; ++$_i1175)
+            $_size1192 = 0;
+            $_etype1195 = 0;
+            $xfer += $input->readListBegin($_etype1195, $_size1192);
+            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
             {
-              $elem1176 = null;
-              $elem1176 = new \metastore\Partition();
-              $xfer += $elem1176->read($input);
-              $this->success []= $elem1176;
+              $elem1197 = null;
+              $elem1197 = new \metastore\Partition();
+              $xfer += $elem1197->read($input);
+              $this->success []= $elem1197;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -32752,9 +32808,9 @@ class ThriftHiveMetastore_get_partitions_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1177)
+          foreach ($this->success as $iter1198)
           {
-            $xfer += $iter1177->write($output);
+            $xfer += $iter1198->write($output);
           }
         }
         $output->writeListEnd();
@@ -32997,15 +33053,15 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1178 = 0;
-            $_etype1181 = 0;
-            $xfer += $input->readListBegin($_etype1181, $_size1178);
-            for ($_i1182 = 0; $_i1182 < $_size1178; ++$_i1182)
+            $_size1199 = 0;
+            $_etype1202 = 0;
+            $xfer += $input->readListBegin($_etype1202, $_size1199);
+            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
             {
-              $elem1183 = null;
-              $elem1183 = new \metastore\PartitionSpec();
-              $xfer += $elem1183->read($input);
-              $this->success []= $elem1183;
+              $elem1204 = null;
+              $elem1204 = new \metastore\PartitionSpec();
+              $xfer += $elem1204->read($input);
+              $this->success []= $elem1204;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33049,9 +33105,9 @@ class ThriftHiveMetastore_get_part_specs_by_filter_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1184)
+          foreach ($this->success as $iter1205)
           {
-            $xfer += $iter1184->write($output);
+            $xfer += $iter1205->write($output);
           }
         }
         $output->writeListEnd();
@@ -33617,14 +33673,14 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->names = array();
-            $_size1185 = 0;
-            $_etype1188 = 0;
-            $xfer += $input->readListBegin($_etype1188, $_size1185);
-            for ($_i1189 = 0; $_i1189 < $_size1185; ++$_i1189)
+            $_size1206 = 0;
+            $_etype1209 = 0;
+            $xfer += $input->readListBegin($_etype1209, $_size1206);
+            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
             {
-              $elem1190 = null;
-              $xfer += $input->readString($elem1190);
-              $this->names []= $elem1190;
+              $elem1211 = null;
+              $xfer += $input->readString($elem1211);
+              $this->names []= $elem1211;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33662,9 +33718,9 @@ class ThriftHiveMetastore_get_partitions_by_names_args {
       {
         $output->writeListBegin(TType::STRING, count($this->names));
         {
-          foreach ($this->names as $iter1191)
+          foreach ($this->names as $iter1212)
           {
-            $xfer += $output->writeString($iter1191);
+            $xfer += $output->writeString($iter1212);
           }
         }
         $output->writeListEnd();
@@ -33753,15 +33809,15 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1192 = 0;
-            $_etype1195 = 0;
-            $xfer += $input->readListBegin($_etype1195, $_size1192);
-            for ($_i1196 = 0; $_i1196 < $_size1192; ++$_i1196)
+            $_size1213 = 0;
+            $_etype1216 = 0;
+            $xfer += $input->readListBegin($_etype1216, $_size1213);
+            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
             {
-              $elem1197 = null;
-              $elem1197 = new \metastore\Partition();
-              $xfer += $elem1197->read($input);
-              $this->success []= $elem1197;
+              $elem1218 = null;
+              $elem1218 = new \metastore\Partition();
+              $xfer += $elem1218->read($input);
+              $this->success []= $elem1218;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -33805,9 +33861,9 @@ class ThriftHiveMetastore_get_partitions_by_names_result {
       {
         $output->writeListBegin(TType::STRUCT, count($this->success));
         {
-          foreach ($this->success as $iter1198)
+          foreach ($this->success as $iter1219)
           {
-            $xfer += $iter1198->write($output);
+            $xfer += $iter1219->write($output);
           }
         }
         $output->writeListEnd();
@@ -34146,15 +34202,15 @@ class ThriftHiveMetastore_alter_partitions_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1199 = 0;
-            $_etype1202 = 0;
-            $xfer += $input->readListBegin($_etype1202, $_size1199);
-            for ($_i1203 = 0; $_i1203 < $_size1199; ++$_i1203)
+            $_size1220 = 0;
+            $_etype1223 = 0;
+            $xfer += $input->readListBegin($_etype1223, $_size1220);
+            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
             {
-              $elem1204 = null;
-              $elem1204 = new \metastore\Partition();
-              $xfer += $elem1204->read($input);
-              $this->new_parts []= $elem1204;
+              $elem1225 = null;
+              $elem1225 = new \metastore\Partition();
+              $xfer += $elem1225->read($input);
+              $this->new_parts []= $elem1225;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34192,9 +34248,9 @@ class ThriftHiveMetastore_alter_partitions_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1205)
+          foreach ($this->new_parts as $iter1226)
           {
-            $xfer += $iter1205->write($output);
+            $xfer += $iter1226->write($output);
           }
         }
         $output->writeListEnd();
@@ -34409,15 +34465,15 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->new_parts = array();
-            $_size1206 = 0;
-            $_etype1209 = 0;
-            $xfer += $input->readListBegin($_etype1209, $_size1206);
-            for ($_i1210 = 0; $_i1210 < $_size1206; ++$_i1210)
+            $_size1227 = 0;
+            $_etype1230 = 0;
+            $xfer += $input->readListBegin($_etype1230, $_size1227);
+            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
             {
-              $elem1211 = null;
-              $elem1211 = new \metastore\Partition();
-              $xfer += $elem1211->read($input);
-              $this->new_parts []= $elem1211;
+              $elem1232 = null;
+              $elem1232 = new \metastore\Partition();
+              $xfer += $elem1232->read($input);
+              $this->new_parts []= $elem1232;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34463,9 +34519,9 @@ class ThriftHiveMetastore_alter_partitions_with_environment_context_args {
       {
         $output->writeListBegin(TType::STRUCT, count($this->new_parts));
         {
-          foreach ($this->new_parts as $iter1212)
+          foreach ($this->new_parts as $iter1233)
           {
-            $xfer += $iter1212->write($output);
+            $xfer += $iter1233->write($output);
           }
         }
         $output->writeListEnd();
@@ -34943,14 +34999,14 @@ class ThriftHiveMetastore_rename_partition_args {
         case 3:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1213 = 0;
-            $_etype1216 = 0;
-            $xfer += $input->readListBegin($_etype1216, $_size1213);
-            for ($_i1217 = 0; $_i1217 < $_size1213; ++$_i1217)
+            $_size1234 = 0;
+            $_etype1237 = 0;
+            $xfer += $input->readListBegin($_etype1237, $_size1234);
+            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
             {
-              $elem1218 = null;
-              $xfer += $input->readString($elem1218);
-              $this->part_vals []= $elem1218;
+              $elem1239 = null;
+              $xfer += $input->readString($elem1239);
+              $this->part_vals []= $elem1239;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -34996,9 +35052,9 @@ class ThriftHiveMetastore_rename_partition_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1219)
+          foreach ($this->part_vals as $iter1240)
           {
-            $xfer += $output->writeString($iter1219);
+            $xfer += $output->writeString($iter1240);
           }
         }
         $output->writeListEnd();
@@ -35183,14 +35239,14 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
         case 1:
           if ($ftype == TType::LST) {
             $this->part_vals = array();
-            $_size1220 = 0;
-            $_etype1223 = 0;
-            $xfer += $input->readListBegin($_etype1223, $_size1220);
-            for ($_i1224 = 0; $_i1224 < $_size1220; ++$_i1224)
+            $_size1241 = 0;
+            $_etype1244 = 0;
+            $xfer += $input->readListBegin($_etype1244, $_size1241);
+            for ($_i1245 = 0; $_i1245 < $_size1241; ++$_i1245)
             {
-              $elem1225 = null;
-              $xfer += $input->readString($elem1225);
-              $this->part_vals []= $elem1225;
+              $elem1246 = null;
+              $xfer += $input->readString($elem1246);
+              $this->part_vals []= $elem1246;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35225,9 +35281,9 @@ class ThriftHiveMetastore_partition_name_has_valid_characters_args {
       {
         $output->writeListBegin(TType::STRING, count($this->part_vals));
         {
-          foreach ($this->part_vals as $iter1226)
+          foreach ($this->part_vals as $iter1247)
           {
-            $xfer += $output->writeString($iter1226);
+            $xfer += $output->writeString($iter1247);
           }
         }
         $output->writeListEnd();
@@ -35681,14 +35737,14 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
         case 0:
           if ($ftype == TType::LST) {
             $this->success = array();
-            $_size1227 = 0;
-            $_etype1230 = 0;
-            $xfer += $input->readListBegin($_etype1230, $_size1227);
-            for ($_i1231 = 0; $_i1231 < $_size1227; ++$_i1231)
+            $_size1248 = 0;
+            $_etype1251 = 0;
+            $xfer += $input->readListBegin($_etype1251, $_size1248);
+            for ($_i1252 = 0; $_i1252 < $_size1248; ++$_i1252)
             {
-              $elem1232 = null;
-              $xfer += $input->readString($elem1232);
-              $this->success []= $elem1232;
+              $elem1253 = null;
+              $xfer += $input->readString($elem1253);
+              $this->success []= $elem1253;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -35724,9 +35780,9 @@ class ThriftHiveMetastore_partition_name_to_vals_result {
       {
         $output->writeListBegin(TType::STRING, count($this->success));
         {
-          foreach ($this->success as $iter1233)
+          foreach ($this->success as $iter1254)
           {
-            $xfer += $output->writeString($iter1233);
+            $xfer += $output->writeString($iter1254);
           }
         }
         $output->writeListEnd();
@@ -35886,17 +35942,17 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
         case 0:
           if ($ftype == TType::MAP) {
             $this->success = array();
-            $_size1234 = 0;
-            $_ktype1235 = 0;
-            $_vtype1236 = 0;
-            $xfer += $input->readMapBegin($_ktype1235, $_vtype1236, $_size1234);
-            for ($_i1238 = 0; $_i1238 < $_size1234; ++$_i1238)
+            $_size1255 = 0;
+            $_ktype1256 = 0;
+            $_vtype1257 = 0;
+            $xfer += $input->readMapBegin($_ktype1256, $_vtype1257, $_size1255);
+            for ($_i1259 = 0; $_i1259 < $_size1255; ++$_i1259)
             {
-              $key1239 = '';
-              $val1240 = '';
-              $xfer += $input->readString($key1239);
-              $xfer += $input->readString($val1240);
-              $this->success[$key1239] = $val1240;
+              $key1260 = '';
+              $val1261 = '';
+              $xfer += $input->readString($key1260);
+              $xfer += $input->readString($val1261);
+              $this->success[$key1260] = $val1261;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -35932,10 +35988,10 @@ class ThriftHiveMetastore_partition_name_to_spec_result {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->success));
         {
-          foreach ($this->success as $kiter1241 => $viter1242)
+          foreach ($this->success as $kiter1262 => $viter1263)
           {
-            $xfer += $output->writeString($kiter1241);
-            $xfer += $output->writeString($viter1242);
+            $xfer += $output->writeString($kiter1262);
+            $xfer += $output->writeString($viter1263);
           }
         }
         $output->writeMapEnd();
@@ -36055,17 +36111,17 @@ class ThriftHiveMetastore_markPartitionForEvent_args {
         case 3:
           if ($ftype == TType::MAP) {
             $this->part_vals = array();
-            $_size1243 = 0;

<TRUNCATED>

[31/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index e459bc2..a816ae7 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -2334,14 +2334,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1202;
-            ::apache::thrift::protocol::TType _etype1205;
-            xfer += iprot->readListBegin(_etype1205, _size1202);
-            this->success.resize(_size1202);
-            uint32_t _i1206;
-            for (_i1206 = 0; _i1206 < _size1202; ++_i1206)
+            uint32_t _size1226;
+            ::apache::thrift::protocol::TType _etype1229;
+            xfer += iprot->readListBegin(_etype1229, _size1226);
+            this->success.resize(_size1226);
+            uint32_t _i1230;
+            for (_i1230 = 0; _i1230 < _size1226; ++_i1230)
             {
-              xfer += iprot->readString(this->success[_i1206]);
+              xfer += iprot->readString(this->success[_i1230]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2380,10 +2380,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1207;
-      for (_iter1207 = this->success.begin(); _iter1207 != this->success.end(); ++_iter1207)
+      std::vector<std::string> ::const_iterator _iter1231;
+      for (_iter1231 = this->success.begin(); _iter1231 != this->success.end(); ++_iter1231)
       {
-        xfer += oprot->writeString((*_iter1207));
+        xfer += oprot->writeString((*_iter1231));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2428,14 +2428,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1208;
-            ::apache::thrift::protocol::TType _etype1211;
-            xfer += iprot->readListBegin(_etype1211, _size1208);
-            (*(this->success)).resize(_size1208);
-            uint32_t _i1212;
-            for (_i1212 = 0; _i1212 < _size1208; ++_i1212)
+            uint32_t _size1232;
+            ::apache::thrift::protocol::TType _etype1235;
+            xfer += iprot->readListBegin(_etype1235, _size1232);
+            (*(this->success)).resize(_size1232);
+            uint32_t _i1236;
+            for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
             {
-              xfer += iprot->readString((*(this->success))[_i1212]);
+              xfer += iprot->readString((*(this->success))[_i1236]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2552,14 +2552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1213;
-            ::apache::thrift::protocol::TType _etype1216;
-            xfer += iprot->readListBegin(_etype1216, _size1213);
-            this->success.resize(_size1213);
-            uint32_t _i1217;
-            for (_i1217 = 0; _i1217 < _size1213; ++_i1217)
+            uint32_t _size1237;
+            ::apache::thrift::protocol::TType _etype1240;
+            xfer += iprot->readListBegin(_etype1240, _size1237);
+            this->success.resize(_size1237);
+            uint32_t _i1241;
+            for (_i1241 = 0; _i1241 < _size1237; ++_i1241)
             {
-              xfer += iprot->readString(this->success[_i1217]);
+              xfer += iprot->readString(this->success[_i1241]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2598,10 +2598,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1218;
-      for (_iter1218 = this->success.begin(); _iter1218 != this->success.end(); ++_iter1218)
+      std::vector<std::string> ::const_iterator _iter1242;
+      for (_iter1242 = this->success.begin(); _iter1242 != this->success.end(); ++_iter1242)
       {
-        xfer += oprot->writeString((*_iter1218));
+        xfer += oprot->writeString((*_iter1242));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2646,14 +2646,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1219;
-            ::apache::thrift::protocol::TType _etype1222;
-            xfer += iprot->readListBegin(_etype1222, _size1219);
-            (*(this->success)).resize(_size1219);
-            uint32_t _i1223;
-            for (_i1223 = 0; _i1223 < _size1219; ++_i1223)
+            uint32_t _size1243;
+            ::apache::thrift::protocol::TType _etype1246;
+            xfer += iprot->readListBegin(_etype1246, _size1243);
+            (*(this->success)).resize(_size1243);
+            uint32_t _i1247;
+            for (_i1247 = 0; _i1247 < _size1243; ++_i1247)
             {
-              xfer += iprot->readString((*(this->success))[_i1223]);
+              xfer += iprot->readString((*(this->success))[_i1247]);
             }
             xfer += iprot->readListEnd();
           }
@@ -3715,17 +3715,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1224;
-            ::apache::thrift::protocol::TType _ktype1225;
-            ::apache::thrift::protocol::TType _vtype1226;
-            xfer += iprot->readMapBegin(_ktype1225, _vtype1226, _size1224);
-            uint32_t _i1228;
-            for (_i1228 = 0; _i1228 < _size1224; ++_i1228)
+            uint32_t _size1248;
+            ::apache::thrift::protocol::TType _ktype1249;
+            ::apache::thrift::protocol::TType _vtype1250;
+            xfer += iprot->readMapBegin(_ktype1249, _vtype1250, _size1248);
+            uint32_t _i1252;
+            for (_i1252 = 0; _i1252 < _size1248; ++_i1252)
             {
-              std::string _key1229;
-              xfer += iprot->readString(_key1229);
-              Type& _val1230 = this->success[_key1229];
-              xfer += _val1230.read(iprot);
+              std::string _key1253;
+              xfer += iprot->readString(_key1253);
+              Type& _val1254 = this->success[_key1253];
+              xfer += _val1254.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3764,11 +3764,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter1231;
-      for (_iter1231 = this->success.begin(); _iter1231 != this->success.end(); ++_iter1231)
+      std::map<std::string, Type> ::const_iterator _iter1255;
+      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
       {
-        xfer += oprot->writeString(_iter1231->first);
-        xfer += _iter1231->second.write(oprot);
+        xfer += oprot->writeString(_iter1255->first);
+        xfer += _iter1255->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -3813,17 +3813,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1232;
-            ::apache::thrift::protocol::TType _ktype1233;
-            ::apache::thrift::protocol::TType _vtype1234;
-            xfer += iprot->readMapBegin(_ktype1233, _vtype1234, _size1232);
-            uint32_t _i1236;
-            for (_i1236 = 0; _i1236 < _size1232; ++_i1236)
+            uint32_t _size1256;
+            ::apache::thrift::protocol::TType _ktype1257;
+            ::apache::thrift::protocol::TType _vtype1258;
+            xfer += iprot->readMapBegin(_ktype1257, _vtype1258, _size1256);
+            uint32_t _i1260;
+            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
             {
-              std::string _key1237;
-              xfer += iprot->readString(_key1237);
-              Type& _val1238 = (*(this->success))[_key1237];
-              xfer += _val1238.read(iprot);
+              std::string _key1261;
+              xfer += iprot->readString(_key1261);
+              Type& _val1262 = (*(this->success))[_key1261];
+              xfer += _val1262.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3977,14 +3977,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1239;
-            ::apache::thrift::protocol::TType _etype1242;
-            xfer += iprot->readListBegin(_etype1242, _size1239);
-            this->success.resize(_size1239);
-            uint32_t _i1243;
-            for (_i1243 = 0; _i1243 < _size1239; ++_i1243)
+            uint32_t _size1263;
+            ::apache::thrift::protocol::TType _etype1266;
+            xfer += iprot->readListBegin(_etype1266, _size1263);
+            this->success.resize(_size1263);
+            uint32_t _i1267;
+            for (_i1267 = 0; _i1267 < _size1263; ++_i1267)
             {
-              xfer += this->success[_i1243].read(iprot);
+              xfer += this->success[_i1267].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4039,10 +4039,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1244;
-      for (_iter1244 = this->success.begin(); _iter1244 != this->success.end(); ++_iter1244)
+      std::vector<FieldSchema> ::const_iterator _iter1268;
+      for (_iter1268 = this->success.begin(); _iter1268 != this->success.end(); ++_iter1268)
       {
-        xfer += (*_iter1244).write(oprot);
+        xfer += (*_iter1268).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4095,14 +4095,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1245;
-            ::apache::thrift::protocol::TType _etype1248;
-            xfer += iprot->readListBegin(_etype1248, _size1245);
-            (*(this->success)).resize(_size1245);
-            uint32_t _i1249;
-            for (_i1249 = 0; _i1249 < _size1245; ++_i1249)
+            uint32_t _size1269;
+            ::apache::thrift::protocol::TType _etype1272;
+            xfer += iprot->readListBegin(_etype1272, _size1269);
+            (*(this->success)).resize(_size1269);
+            uint32_t _i1273;
+            for (_i1273 = 0; _i1273 < _size1269; ++_i1273)
             {
-              xfer += (*(this->success))[_i1249].read(iprot);
+              xfer += (*(this->success))[_i1273].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4288,14 +4288,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1250;
-            ::apache::thrift::protocol::TType _etype1253;
-            xfer += iprot->readListBegin(_etype1253, _size1250);
-            this->success.resize(_size1250);
-            uint32_t _i1254;
-            for (_i1254 = 0; _i1254 < _size1250; ++_i1254)
+            uint32_t _size1274;
+            ::apache::thrift::protocol::TType _etype1277;
+            xfer += iprot->readListBegin(_etype1277, _size1274);
+            this->success.resize(_size1274);
+            uint32_t _i1278;
+            for (_i1278 = 0; _i1278 < _size1274; ++_i1278)
             {
-              xfer += this->success[_i1254].read(iprot);
+              xfer += this->success[_i1278].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4350,10 +4350,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1255;
-      for (_iter1255 = this->success.begin(); _iter1255 != this->success.end(); ++_iter1255)
+      std::vector<FieldSchema> ::const_iterator _iter1279;
+      for (_iter1279 = this->success.begin(); _iter1279 != this->success.end(); ++_iter1279)
       {
-        xfer += (*_iter1255).write(oprot);
+        xfer += (*_iter1279).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4406,14 +4406,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1256;
-            ::apache::thrift::protocol::TType _etype1259;
-            xfer += iprot->readListBegin(_etype1259, _size1256);
-            (*(this->success)).resize(_size1256);
-            uint32_t _i1260;
-            for (_i1260 = 0; _i1260 < _size1256; ++_i1260)
+            uint32_t _size1280;
+            ::apache::thrift::protocol::TType _etype1283;
+            xfer += iprot->readListBegin(_etype1283, _size1280);
+            (*(this->success)).resize(_size1280);
+            uint32_t _i1284;
+            for (_i1284 = 0; _i1284 < _size1280; ++_i1284)
             {
-              xfer += (*(this->success))[_i1260].read(iprot);
+              xfer += (*(this->success))[_i1284].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4583,14 +4583,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1261;
-            ::apache::thrift::protocol::TType _etype1264;
-            xfer += iprot->readListBegin(_etype1264, _size1261);
-            this->success.resize(_size1261);
-            uint32_t _i1265;
-            for (_i1265 = 0; _i1265 < _size1261; ++_i1265)
+            uint32_t _size1285;
+            ::apache::thrift::protocol::TType _etype1288;
+            xfer += iprot->readListBegin(_etype1288, _size1285);
+            this->success.resize(_size1285);
+            uint32_t _i1289;
+            for (_i1289 = 0; _i1289 < _size1285; ++_i1289)
             {
-              xfer += this->success[_i1265].read(iprot);
+              xfer += this->success[_i1289].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4645,10 +4645,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1266;
-      for (_iter1266 = this->success.begin(); _iter1266 != this->success.end(); ++_iter1266)
+      std::vector<FieldSchema> ::const_iterator _iter1290;
+      for (_iter1290 = this->success.begin(); _iter1290 != this->success.end(); ++_iter1290)
       {
-        xfer += (*_iter1266).write(oprot);
+        xfer += (*_iter1290).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -4701,14 +4701,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1267;
-            ::apache::thrift::protocol::TType _etype1270;
-            xfer += iprot->readListBegin(_etype1270, _size1267);
-            (*(this->success)).resize(_size1267);
-            uint32_t _i1271;
-            for (_i1271 = 0; _i1271 < _size1267; ++_i1271)
+            uint32_t _size1291;
+            ::apache::thrift::protocol::TType _etype1294;
+            xfer += iprot->readListBegin(_etype1294, _size1291);
+            (*(this->success)).resize(_size1291);
+            uint32_t _i1295;
+            for (_i1295 = 0; _i1295 < _size1291; ++_i1295)
             {
-              xfer += (*(this->success))[_i1271].read(iprot);
+              xfer += (*(this->success))[_i1295].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4894,14 +4894,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1272;
-            ::apache::thrift::protocol::TType _etype1275;
-            xfer += iprot->readListBegin(_etype1275, _size1272);
-            this->success.resize(_size1272);
-            uint32_t _i1276;
-            for (_i1276 = 0; _i1276 < _size1272; ++_i1276)
+            uint32_t _size1296;
+            ::apache::thrift::protocol::TType _etype1299;
+            xfer += iprot->readListBegin(_etype1299, _size1296);
+            this->success.resize(_size1296);
+            uint32_t _i1300;
+            for (_i1300 = 0; _i1300 < _size1296; ++_i1300)
             {
-              xfer += this->success[_i1276].read(iprot);
+              xfer += this->success[_i1300].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -4956,10 +4956,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter1277;
-      for (_iter1277 = this->success.begin(); _iter1277 != this->success.end(); ++_iter1277)
+      std::vector<FieldSchema> ::const_iterator _iter1301;
+      for (_iter1301 = this->success.begin(); _iter1301 != this->success.end(); ++_iter1301)
       {
-        xfer += (*_iter1277).write(oprot);
+        xfer += (*_iter1301).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5012,14 +5012,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1278;
-            ::apache::thrift::protocol::TType _etype1281;
-            xfer += iprot->readListBegin(_etype1281, _size1278);
-            (*(this->success)).resize(_size1278);
-            uint32_t _i1282;
-            for (_i1282 = 0; _i1282 < _size1278; ++_i1282)
+            uint32_t _size1302;
+            ::apache::thrift::protocol::TType _etype1305;
+            xfer += iprot->readListBegin(_etype1305, _size1302);
+            (*(this->success)).resize(_size1302);
+            uint32_t _i1306;
+            for (_i1306 = 0; _i1306 < _size1302; ++_i1306)
             {
-              xfer += (*(this->success))[_i1282].read(iprot);
+              xfer += (*(this->success))[_i1306].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5612,14 +5612,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->primaryKeys.clear();
-            uint32_t _size1283;
-            ::apache::thrift::protocol::TType _etype1286;
-            xfer += iprot->readListBegin(_etype1286, _size1283);
-            this->primaryKeys.resize(_size1283);
-            uint32_t _i1287;
-            for (_i1287 = 0; _i1287 < _size1283; ++_i1287)
+            uint32_t _size1307;
+            ::apache::thrift::protocol::TType _etype1310;
+            xfer += iprot->readListBegin(_etype1310, _size1307);
+            this->primaryKeys.resize(_size1307);
+            uint32_t _i1311;
+            for (_i1311 = 0; _i1311 < _size1307; ++_i1311)
             {
-              xfer += this->primaryKeys[_i1287].read(iprot);
+              xfer += this->primaryKeys[_i1311].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5632,14 +5632,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->foreignKeys.clear();
-            uint32_t _size1288;
-            ::apache::thrift::protocol::TType _etype1291;
-            xfer += iprot->readListBegin(_etype1291, _size1288);
-            this->foreignKeys.resize(_size1288);
-            uint32_t _i1292;
-            for (_i1292 = 0; _i1292 < _size1288; ++_i1292)
+            uint32_t _size1312;
+            ::apache::thrift::protocol::TType _etype1315;
+            xfer += iprot->readListBegin(_etype1315, _size1312);
+            this->foreignKeys.resize(_size1312);
+            uint32_t _i1316;
+            for (_i1316 = 0; _i1316 < _size1312; ++_i1316)
             {
-              xfer += this->foreignKeys[_i1292].read(iprot);
+              xfer += this->foreignKeys[_i1316].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5652,14 +5652,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->uniqueConstraints.clear();
-            uint32_t _size1293;
-            ::apache::thrift::protocol::TType _etype1296;
-            xfer += iprot->readListBegin(_etype1296, _size1293);
-            this->uniqueConstraints.resize(_size1293);
-            uint32_t _i1297;
-            for (_i1297 = 0; _i1297 < _size1293; ++_i1297)
+            uint32_t _size1317;
+            ::apache::thrift::protocol::TType _etype1320;
+            xfer += iprot->readListBegin(_etype1320, _size1317);
+            this->uniqueConstraints.resize(_size1317);
+            uint32_t _i1321;
+            for (_i1321 = 0; _i1321 < _size1317; ++_i1321)
             {
-              xfer += this->uniqueConstraints[_i1297].read(iprot);
+              xfer += this->uniqueConstraints[_i1321].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5672,14 +5672,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->notNullConstraints.clear();
-            uint32_t _size1298;
-            ::apache::thrift::protocol::TType _etype1301;
-            xfer += iprot->readListBegin(_etype1301, _size1298);
-            this->notNullConstraints.resize(_size1298);
-            uint32_t _i1302;
-            for (_i1302 = 0; _i1302 < _size1298; ++_i1302)
+            uint32_t _size1322;
+            ::apache::thrift::protocol::TType _etype1325;
+            xfer += iprot->readListBegin(_etype1325, _size1322);
+            this->notNullConstraints.resize(_size1322);
+            uint32_t _i1326;
+            for (_i1326 = 0; _i1326 < _size1322; ++_i1326)
             {
-              xfer += this->notNullConstraints[_i1302].read(iprot);
+              xfer += this->notNullConstraints[_i1326].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5692,14 +5692,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->defaultConstraints.clear();
-            uint32_t _size1303;
-            ::apache::thrift::protocol::TType _etype1306;
-            xfer += iprot->readListBegin(_etype1306, _size1303);
-            this->defaultConstraints.resize(_size1303);
-            uint32_t _i1307;
-            for (_i1307 = 0; _i1307 < _size1303; ++_i1307)
+            uint32_t _size1327;
+            ::apache::thrift::protocol::TType _etype1330;
+            xfer += iprot->readListBegin(_etype1330, _size1327);
+            this->defaultConstraints.resize(_size1327);
+            uint32_t _i1331;
+            for (_i1331 = 0; _i1331 < _size1327; ++_i1331)
             {
-              xfer += this->defaultConstraints[_i1307].read(iprot);
+              xfer += this->defaultConstraints[_i1331].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5712,14 +5712,14 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::read(::apache::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->checkConstraints.clear();
-            uint32_t _size1308;
-            ::apache::thrift::protocol::TType _etype1311;
-            xfer += iprot->readListBegin(_etype1311, _size1308);
-            this->checkConstraints.resize(_size1308);
-            uint32_t _i1312;
-            for (_i1312 = 0; _i1312 < _size1308; ++_i1312)
+            uint32_t _size1332;
+            ::apache::thrift::protocol::TType _etype1335;
+            xfer += iprot->readListBegin(_etype1335, _size1332);
+            this->checkConstraints.resize(_size1332);
+            uint32_t _i1336;
+            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
             {
-              xfer += this->checkConstraints[_i1312].read(iprot);
+              xfer += this->checkConstraints[_i1336].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5752,10 +5752,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->primaryKeys.size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1313;
-    for (_iter1313 = this->primaryKeys.begin(); _iter1313 != this->primaryKeys.end(); ++_iter1313)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1337;
+    for (_iter1337 = this->primaryKeys.begin(); _iter1337 != this->primaryKeys.end(); ++_iter1337)
     {
-      xfer += (*_iter1313).write(oprot);
+      xfer += (*_iter1337).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5764,10 +5764,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->foreignKeys.size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1314;
-    for (_iter1314 = this->foreignKeys.begin(); _iter1314 != this->foreignKeys.end(); ++_iter1314)
+    std::vector<SQLForeignKey> ::const_iterator _iter1338;
+    for (_iter1338 = this->foreignKeys.begin(); _iter1338 != this->foreignKeys.end(); ++_iter1338)
     {
-      xfer += (*_iter1314).write(oprot);
+      xfer += (*_iter1338).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5776,10 +5776,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->uniqueConstraints.size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1315;
-    for (_iter1315 = this->uniqueConstraints.begin(); _iter1315 != this->uniqueConstraints.end(); ++_iter1315)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1339;
+    for (_iter1339 = this->uniqueConstraints.begin(); _iter1339 != this->uniqueConstraints.end(); ++_iter1339)
     {
-      xfer += (*_iter1315).write(oprot);
+      xfer += (*_iter1339).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5788,10 +5788,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->notNullConstraints.size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1316;
-    for (_iter1316 = this->notNullConstraints.begin(); _iter1316 != this->notNullConstraints.end(); ++_iter1316)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1340;
+    for (_iter1340 = this->notNullConstraints.begin(); _iter1340 != this->notNullConstraints.end(); ++_iter1340)
     {
-      xfer += (*_iter1316).write(oprot);
+      xfer += (*_iter1340).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5800,10 +5800,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->defaultConstraints.size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1317;
-    for (_iter1317 = this->defaultConstraints.begin(); _iter1317 != this->defaultConstraints.end(); ++_iter1317)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1341;
+    for (_iter1341 = this->defaultConstraints.begin(); _iter1341 != this->defaultConstraints.end(); ++_iter1341)
     {
-      xfer += (*_iter1317).write(oprot);
+      xfer += (*_iter1341).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5812,10 +5812,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_args::write(::apache:
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->checkConstraints.size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1318;
-    for (_iter1318 = this->checkConstraints.begin(); _iter1318 != this->checkConstraints.end(); ++_iter1318)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1342;
+    for (_iter1342 = this->checkConstraints.begin(); _iter1342 != this->checkConstraints.end(); ++_iter1342)
     {
-      xfer += (*_iter1318).write(oprot);
+      xfer += (*_iter1342).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5843,10 +5843,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("primaryKeys", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->primaryKeys)).size()));
-    std::vector<SQLPrimaryKey> ::const_iterator _iter1319;
-    for (_iter1319 = (*(this->primaryKeys)).begin(); _iter1319 != (*(this->primaryKeys)).end(); ++_iter1319)
+    std::vector<SQLPrimaryKey> ::const_iterator _iter1343;
+    for (_iter1343 = (*(this->primaryKeys)).begin(); _iter1343 != (*(this->primaryKeys)).end(); ++_iter1343)
     {
-      xfer += (*_iter1319).write(oprot);
+      xfer += (*_iter1343).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5855,10 +5855,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("foreignKeys", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->foreignKeys)).size()));
-    std::vector<SQLForeignKey> ::const_iterator _iter1320;
-    for (_iter1320 = (*(this->foreignKeys)).begin(); _iter1320 != (*(this->foreignKeys)).end(); ++_iter1320)
+    std::vector<SQLForeignKey> ::const_iterator _iter1344;
+    for (_iter1344 = (*(this->foreignKeys)).begin(); _iter1344 != (*(this->foreignKeys)).end(); ++_iter1344)
     {
-      xfer += (*_iter1320).write(oprot);
+      xfer += (*_iter1344).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5867,10 +5867,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("uniqueConstraints", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->uniqueConstraints)).size()));
-    std::vector<SQLUniqueConstraint> ::const_iterator _iter1321;
-    for (_iter1321 = (*(this->uniqueConstraints)).begin(); _iter1321 != (*(this->uniqueConstraints)).end(); ++_iter1321)
+    std::vector<SQLUniqueConstraint> ::const_iterator _iter1345;
+    for (_iter1345 = (*(this->uniqueConstraints)).begin(); _iter1345 != (*(this->uniqueConstraints)).end(); ++_iter1345)
     {
-      xfer += (*_iter1321).write(oprot);
+      xfer += (*_iter1345).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5879,10 +5879,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("notNullConstraints", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->notNullConstraints)).size()));
-    std::vector<SQLNotNullConstraint> ::const_iterator _iter1322;
-    for (_iter1322 = (*(this->notNullConstraints)).begin(); _iter1322 != (*(this->notNullConstraints)).end(); ++_iter1322)
+    std::vector<SQLNotNullConstraint> ::const_iterator _iter1346;
+    for (_iter1346 = (*(this->notNullConstraints)).begin(); _iter1346 != (*(this->notNullConstraints)).end(); ++_iter1346)
     {
-      xfer += (*_iter1322).write(oprot);
+      xfer += (*_iter1346).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5891,10 +5891,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("defaultConstraints", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->defaultConstraints)).size()));
-    std::vector<SQLDefaultConstraint> ::const_iterator _iter1323;
-    for (_iter1323 = (*(this->defaultConstraints)).begin(); _iter1323 != (*(this->defaultConstraints)).end(); ++_iter1323)
+    std::vector<SQLDefaultConstraint> ::const_iterator _iter1347;
+    for (_iter1347 = (*(this->defaultConstraints)).begin(); _iter1347 != (*(this->defaultConstraints)).end(); ++_iter1347)
     {
-      xfer += (*_iter1323).write(oprot);
+      xfer += (*_iter1347).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -5903,10 +5903,10 @@ uint32_t ThriftHiveMetastore_create_table_with_constraints_pargs::write(::apache
   xfer += oprot->writeFieldBegin("checkConstraints", ::apache::thrift::protocol::T_LIST, 7);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->checkConstraints)).size()));
-    std::vector<SQLCheckConstraint> ::const_iterator _iter1324;
-    for (_iter1324 = (*(this->checkConstraints)).begin(); _iter1324 != (*(this->checkConstraints)).end(); ++_iter1324)
+    std::vector<SQLCheckConstraint> ::const_iterator _iter1348;
+    for (_iter1348 = (*(this->checkConstraints)).begin(); _iter1348 != (*(this->checkConstraints)).end(); ++_iter1348)
     {
-      xfer += (*_iter1324).write(oprot);
+      xfer += (*_iter1348).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8074,14 +8074,14 @@ uint32_t ThriftHiveMetastore_truncate_table_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partNames.clear();
-            uint32_t _size1325;
-            ::apache::thrift::protocol::TType _etype1328;
-            xfer += iprot->readListBegin(_etype1328, _size1325);
-            this->partNames.resize(_size1325);
-            uint32_t _i1329;
-            for (_i1329 = 0; _i1329 < _size1325; ++_i1329)
+            uint32_t _size1349;
+            ::apache::thrift::protocol::TType _etype1352;
+            xfer += iprot->readListBegin(_etype1352, _size1349);
+            this->partNames.resize(_size1349);
+            uint32_t _i1353;
+            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
             {
-              xfer += iprot->readString(this->partNames[_i1329]);
+              xfer += iprot->readString(this->partNames[_i1353]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8118,10 +8118,10 @@ uint32_t ThriftHiveMetastore_truncate_table_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partNames.size()));
-    std::vector<std::string> ::const_iterator _iter1330;
-    for (_iter1330 = this->partNames.begin(); _iter1330 != this->partNames.end(); ++_iter1330)
+    std::vector<std::string> ::const_iterator _iter1354;
+    for (_iter1354 = this->partNames.begin(); _iter1354 != this->partNames.end(); ++_iter1354)
     {
-      xfer += oprot->writeString((*_iter1330));
+      xfer += oprot->writeString((*_iter1354));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8153,10 +8153,10 @@ uint32_t ThriftHiveMetastore_truncate_table_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("partNames", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partNames)).size()));
-    std::vector<std::string> ::const_iterator _iter1331;
-    for (_iter1331 = (*(this->partNames)).begin(); _iter1331 != (*(this->partNames)).end(); ++_iter1331)
+    std::vector<std::string> ::const_iterator _iter1355;
+    for (_iter1355 = (*(this->partNames)).begin(); _iter1355 != (*(this->partNames)).end(); ++_iter1355)
     {
-      xfer += oprot->writeString((*_iter1331));
+      xfer += oprot->writeString((*_iter1355));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8400,14 +8400,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1332;
-            ::apache::thrift::protocol::TType _etype1335;
-            xfer += iprot->readListBegin(_etype1335, _size1332);
-            this->success.resize(_size1332);
-            uint32_t _i1336;
-            for (_i1336 = 0; _i1336 < _size1332; ++_i1336)
+            uint32_t _size1356;
+            ::apache::thrift::protocol::TType _etype1359;
+            xfer += iprot->readListBegin(_etype1359, _size1356);
+            this->success.resize(_size1356);
+            uint32_t _i1360;
+            for (_i1360 = 0; _i1360 < _size1356; ++_i1360)
             {
-              xfer += iprot->readString(this->success[_i1336]);
+              xfer += iprot->readString(this->success[_i1360]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8446,10 +8446,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1337;
-      for (_iter1337 = this->success.begin(); _iter1337 != this->success.end(); ++_iter1337)
+      std::vector<std::string> ::const_iterator _iter1361;
+      for (_iter1361 = this->success.begin(); _iter1361 != this->success.end(); ++_iter1361)
       {
-        xfer += oprot->writeString((*_iter1337));
+        xfer += oprot->writeString((*_iter1361));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8494,14 +8494,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1338;
-            ::apache::thrift::protocol::TType _etype1341;
-            xfer += iprot->readListBegin(_etype1341, _size1338);
-            (*(this->success)).resize(_size1338);
-            uint32_t _i1342;
-            for (_i1342 = 0; _i1342 < _size1338; ++_i1342)
+            uint32_t _size1362;
+            ::apache::thrift::protocol::TType _etype1365;
+            xfer += iprot->readListBegin(_etype1365, _size1362);
+            (*(this->success)).resize(_size1362);
+            uint32_t _i1366;
+            for (_i1366 = 0; _i1366 < _size1362; ++_i1366)
             {
-              xfer += iprot->readString((*(this->success))[_i1342]);
+              xfer += iprot->readString((*(this->success))[_i1366]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8671,14 +8671,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1343;
-            ::apache::thrift::protocol::TType _etype1346;
-            xfer += iprot->readListBegin(_etype1346, _size1343);
-            this->success.resize(_size1343);
-            uint32_t _i1347;
-            for (_i1347 = 0; _i1347 < _size1343; ++_i1347)
+            uint32_t _size1367;
+            ::apache::thrift::protocol::TType _etype1370;
+            xfer += iprot->readListBegin(_etype1370, _size1367);
+            this->success.resize(_size1367);
+            uint32_t _i1371;
+            for (_i1371 = 0; _i1371 < _size1367; ++_i1371)
             {
-              xfer += iprot->readString(this->success[_i1347]);
+              xfer += iprot->readString(this->success[_i1371]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8717,10 +8717,10 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_result::write(::apache::thrift::
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1348;
-      for (_iter1348 = this->success.begin(); _iter1348 != this->success.end(); ++_iter1348)
+      std::vector<std::string> ::const_iterator _iter1372;
+      for (_iter1372 = this->success.begin(); _iter1372 != this->success.end(); ++_iter1372)
       {
-        xfer += oprot->writeString((*_iter1348));
+        xfer += oprot->writeString((*_iter1372));
       }
       xfer += oprot->writeListEnd();
     }
@@ -8765,14 +8765,14 @@ uint32_t ThriftHiveMetastore_get_tables_by_type_presult::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1349;
-            ::apache::thrift::protocol::TType _etype1352;
-            xfer += iprot->readListBegin(_etype1352, _size1349);
-            (*(this->success)).resize(_size1349);
-            uint32_t _i1353;
-            for (_i1353 = 0; _i1353 < _size1349; ++_i1353)
+            uint32_t _size1373;
+            ::apache::thrift::protocol::TType _etype1376;
+            xfer += iprot->readListBegin(_etype1376, _size1373);
+            (*(this->success)).resize(_size1373);
+            uint32_t _i1377;
+            for (_i1377 = 0; _i1377 < _size1373; ++_i1377)
             {
-              xfer += iprot->readString((*(this->success))[_i1353]);
+              xfer += iprot->readString((*(this->success))[_i1377]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8910,14 +8910,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1354;
-            ::apache::thrift::protocol::TType _etype1357;
-            xfer += iprot->readListBegin(_etype1357, _size1354);
-            this->success.resize(_size1354);
-            uint32_t _i1358;
-            for (_i1358 = 0; _i1358 < _size1354; ++_i1358)
+            uint32_t _size1378;
+            ::apache::thrift::protocol::TType _etype1381;
+            xfer += iprot->readListBegin(_etype1381, _size1378);
+            this->success.resize(_size1378);
+            uint32_t _i1382;
+            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
             {
-              xfer += iprot->readString(this->success[_i1358]);
+              xfer += iprot->readString(this->success[_i1382]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8956,10 +8956,10 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_result::write(
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1359;
-      for (_iter1359 = this->success.begin(); _iter1359 != this->success.end(); ++_iter1359)
+      std::vector<std::string> ::const_iterator _iter1383;
+      for (_iter1383 = this->success.begin(); _iter1383 != this->success.end(); ++_iter1383)
       {
-        xfer += oprot->writeString((*_iter1359));
+        xfer += oprot->writeString((*_iter1383));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9004,14 +9004,14 @@ uint32_t ThriftHiveMetastore_get_materialized_views_for_rewriting_presult::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1360;
-            ::apache::thrift::protocol::TType _etype1363;
-            xfer += iprot->readListBegin(_etype1363, _size1360);
-            (*(this->success)).resize(_size1360);
-            uint32_t _i1364;
-            for (_i1364 = 0; _i1364 < _size1360; ++_i1364)
+            uint32_t _size1384;
+            ::apache::thrift::protocol::TType _etype1387;
+            xfer += iprot->readListBegin(_etype1387, _size1384);
+            (*(this->success)).resize(_size1384);
+            uint32_t _i1388;
+            for (_i1388 = 0; _i1388 < _size1384; ++_i1388)
             {
-              xfer += iprot->readString((*(this->success))[_i1364]);
+              xfer += iprot->readString((*(this->success))[_i1388]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9086,14 +9086,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_types.clear();
-            uint32_t _size1365;
-            ::apache::thrift::protocol::TType _etype1368;
-            xfer += iprot->readListBegin(_etype1368, _size1365);
-            this->tbl_types.resize(_size1365);
-            uint32_t _i1369;
-            for (_i1369 = 0; _i1369 < _size1365; ++_i1369)
+            uint32_t _size1389;
+            ::apache::thrift::protocol::TType _etype1392;
+            xfer += iprot->readListBegin(_etype1392, _size1389);
+            this->tbl_types.resize(_size1389);
+            uint32_t _i1393;
+            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
             {
-              xfer += iprot->readString(this->tbl_types[_i1369]);
+              xfer += iprot->readString(this->tbl_types[_i1393]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9130,10 +9130,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_types.size()));
-    std::vector<std::string> ::const_iterator _iter1370;
-    for (_iter1370 = this->tbl_types.begin(); _iter1370 != this->tbl_types.end(); ++_iter1370)
+    std::vector<std::string> ::const_iterator _iter1394;
+    for (_iter1394 = this->tbl_types.begin(); _iter1394 != this->tbl_types.end(); ++_iter1394)
     {
-      xfer += oprot->writeString((*_iter1370));
+      xfer += oprot->writeString((*_iter1394));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9165,10 +9165,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("tbl_types", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_types)).size()));
-    std::vector<std::string> ::const_iterator _iter1371;
-    for (_iter1371 = (*(this->tbl_types)).begin(); _iter1371 != (*(this->tbl_types)).end(); ++_iter1371)
+    std::vector<std::string> ::const_iterator _iter1395;
+    for (_iter1395 = (*(this->tbl_types)).begin(); _iter1395 != (*(this->tbl_types)).end(); ++_iter1395)
     {
-      xfer += oprot->writeString((*_iter1371));
+      xfer += oprot->writeString((*_iter1395));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9209,14 +9209,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1372;
-            ::apache::thrift::protocol::TType _etype1375;
-            xfer += iprot->readListBegin(_etype1375, _size1372);
-            this->success.resize(_size1372);
-            uint32_t _i1376;
-            for (_i1376 = 0; _i1376 < _size1372; ++_i1376)
+            uint32_t _size1396;
+            ::apache::thrift::protocol::TType _etype1399;
+            xfer += iprot->readListBegin(_etype1399, _size1396);
+            this->success.resize(_size1396);
+            uint32_t _i1400;
+            for (_i1400 = 0; _i1400 < _size1396; ++_i1400)
             {
-              xfer += this->success[_i1376].read(iprot);
+              xfer += this->success[_i1400].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9255,10 +9255,10 @@ uint32_t ThriftHiveMetastore_get_table_meta_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<TableMeta> ::const_iterator _iter1377;
-      for (_iter1377 = this->success.begin(); _iter1377 != this->success.end(); ++_iter1377)
+      std::vector<TableMeta> ::const_iterator _iter1401;
+      for (_iter1401 = this->success.begin(); _iter1401 != this->success.end(); ++_iter1401)
       {
-        xfer += (*_iter1377).write(oprot);
+        xfer += (*_iter1401).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -9303,14 +9303,14 @@ uint32_t ThriftHiveMetastore_get_table_meta_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1378;
-            ::apache::thrift::protocol::TType _etype1381;
-            xfer += iprot->readListBegin(_etype1381, _size1378);
-            (*(this->success)).resize(_size1378);
-            uint32_t _i1382;
-            for (_i1382 = 0; _i1382 < _size1378; ++_i1382)
+            uint32_t _size1402;
+            ::apache::thrift::protocol::TType _etype1405;
+            xfer += iprot->readListBegin(_etype1405, _size1402);
+            (*(this->success)).resize(_size1402);
+            uint32_t _i1406;
+            for (_i1406 = 0; _i1406 < _size1402; ++_i1406)
             {
-              xfer += (*(this->success))[_i1382].read(iprot);
+              xfer += (*(this->success))[_i1406].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -9448,14 +9448,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1383;
-            ::apache::thrift::protocol::TType _etype1386;
-            xfer += iprot->readListBegin(_etype1386, _size1383);
-            this->success.resize(_size1383);
-            uint32_t _i1387;
-            for (_i1387 = 0; _i1387 < _size1383; ++_i1387)
+            uint32_t _size1407;
+            ::apache::thrift::protocol::TType _etype1410;
+            xfer += iprot->readListBegin(_etype1410, _size1407);
+            this->success.resize(_size1407);
+            uint32_t _i1411;
+            for (_i1411 = 0; _i1411 < _size1407; ++_i1411)
             {
-              xfer += iprot->readString(this->success[_i1387]);
+              xfer += iprot->readString(this->success[_i1411]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9494,10 +9494,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1388;
-      for (_iter1388 = this->success.begin(); _iter1388 != this->success.end(); ++_iter1388)
+      std::vector<std::string> ::const_iterator _iter1412;
+      for (_iter1412 = this->success.begin(); _iter1412 != this->success.end(); ++_iter1412)
       {
-        xfer += oprot->writeString((*_iter1388));
+        xfer += oprot->writeString((*_iter1412));
       }
       xfer += oprot->writeListEnd();
     }
@@ -9542,14 +9542,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1389;
-            ::apache::thrift::protocol::TType _etype1392;
-            xfer += iprot->readListBegin(_etype1392, _size1389);
-            (*(this->success)).resize(_size1389);
-            uint32_t _i1393;
-            for (_i1393 = 0; _i1393 < _size1389; ++_i1393)
+            uint32_t _size1413;
+            ::apache::thrift::protocol::TType _etype1416;
+            xfer += iprot->readListBegin(_etype1416, _size1413);
+            (*(this->success)).resize(_size1413);
+            uint32_t _i1417;
+            for (_i1417 = 0; _i1417 < _size1413; ++_i1417)
             {
-              xfer += iprot->readString((*(this->success))[_i1393]);
+              xfer += iprot->readString((*(this->success))[_i1417]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9859,14 +9859,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1394;
-            ::apache::thrift::protocol::TType _etype1397;
-            xfer += iprot->readListBegin(_etype1397, _size1394);
-            this->tbl_names.resize(_size1394);
-            uint32_t _i1398;
-            for (_i1398 = 0; _i1398 < _size1394; ++_i1398)
+            uint32_t _size1418;
+            ::apache::thrift::protocol::TType _etype1421;
+            xfer += iprot->readListBegin(_etype1421, _size1418);
+            this->tbl_names.resize(_size1418);
+            uint32_t _i1422;
+            for (_i1422 = 0; _i1422 < _size1418; ++_i1422)
             {
-              xfer += iprot->readString(this->tbl_names[_i1398]);
+              xfer += iprot->readString(this->tbl_names[_i1422]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9899,10 +9899,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1399;
-    for (_iter1399 = this->tbl_names.begin(); _iter1399 != this->tbl_names.end(); ++_iter1399)
+    std::vector<std::string> ::const_iterator _iter1423;
+    for (_iter1423 = this->tbl_names.begin(); _iter1423 != this->tbl_names.end(); ++_iter1423)
     {
-      xfer += oprot->writeString((*_iter1399));
+      xfer += oprot->writeString((*_iter1423));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9930,10 +9930,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1400;
-    for (_iter1400 = (*(this->tbl_names)).begin(); _iter1400 != (*(this->tbl_names)).end(); ++_iter1400)
+    std::vector<std::string> ::const_iterator _iter1424;
+    for (_iter1424 = (*(this->tbl_names)).begin(); _iter1424 != (*(this->tbl_names)).end(); ++_iter1424)
     {
-      xfer += oprot->writeString((*_iter1400));
+      xfer += oprot->writeString((*_iter1424));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9974,14 +9974,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1401;
-            ::apache::thrift::protocol::TType _etype1404;
-            xfer += iprot->readListBegin(_etype1404, _size1401);
-            this->success.resize(_size1401);
-            uint32_t _i1405;
-            for (_i1405 = 0; _i1405 < _size1401; ++_i1405)
+            uint32_t _size1425;
+            ::apache::thrift::protocol::TType _etype1428;
+            xfer += iprot->readListBegin(_etype1428, _size1425);
+            this->success.resize(_size1425);
+            uint32_t _i1429;
+            for (_i1429 = 0; _i1429 < _size1425; ++_i1429)
             {
-              xfer += this->success[_i1405].read(iprot);
+              xfer += this->success[_i1429].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10012,10 +10012,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter1406;
-      for (_iter1406 = this->success.begin(); _iter1406 != this->success.end(); ++_iter1406)
+      std::vector<Table> ::const_iterator _iter1430;
+      for (_iter1430 = this->success.begin(); _iter1430 != this->success.end(); ++_iter1430)
       {
-        xfer += (*_iter1406).write(oprot);
+        xfer += (*_iter1430).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -10056,14 +10056,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1407;
-            ::apache::thrift::protocol::TType _etype1410;
-            xfer += iprot->readListBegin(_etype1410, _size1407);
-            (*(this->success)).resize(_size1407);
-            uint32_t _i1411;
-            for (_i1411 = 0; _i1411 < _size1407; ++_i1411)
+            uint32_t _size1431;
+            ::apache::thrift::protocol::TType _etype1434;
+            xfer += iprot->readListBegin(_etype1434, _size1431);
+            (*(this->success)).resize(_size1431);
+            uint32_t _i1435;
+            for (_i1435 = 0; _i1435 < _size1431; ++_i1435)
             {
-              xfer += (*(this->success))[_i1411].read(iprot);
+              xfer += (*(this->success))[_i1435].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -10596,14 +10596,14 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size1412;
-            ::apache::thrift::protocol::TType _etype1415;
-            xfer += iprot->readListBegin(_etype1415, _size1412);
-            this->tbl_names.resize(_size1412);
-            uint32_t _i1416;
-            for (_i1416 = 0; _i1416 < _size1412; ++_i1416)
+            uint32_t _size1436;
+            ::apache::thrift::protocol::TType _etype1439;
+            xfer += iprot->readListBegin(_etype1439, _size1436);
+            this->tbl_names.resize(_size1436);
+            uint32_t _i1440;
+            for (_i1440 = 0; _i1440 < _size1436; ++_i1440)
             {
-              xfer += iprot->readString(this->tbl_names[_i1416]);
+              xfer += iprot->readString(this->tbl_names[_i1440]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10636,10 +10636,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_args::write(:
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter1417;
-    for (_iter1417 = this->tbl_names.begin(); _iter1417 != this->tbl_names.end(); ++_iter1417)
+    std::vector<std::string> ::const_iterator _iter1441;
+    for (_iter1441 = this->tbl_names.begin(); _iter1441 != this->tbl_names.end(); ++_iter1441)
     {
-      xfer += oprot->writeString((*_iter1417));
+      xfer += oprot->writeString((*_iter1441));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10667,10 +10667,10 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_pargs::write(
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter1418;
-    for (_iter1418 = (*(this->tbl_names)).begin(); _iter1418 != (*(this->tbl_names)).end(); ++_iter1418)
+    std::vector<std::string> ::const_iterator _iter1442;
+    for (_iter1442 = (*(this->tbl_names)).begin(); _iter1442 != (*(this->tbl_names)).end(); ++_iter1442)
     {
-      xfer += oprot->writeString((*_iter1418));
+      xfer += oprot->writeString((*_iter1442));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10711,17 +10711,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::read(
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size1419;
-            ::apache::thrift::protocol::TType _ktype1420;
-            ::apache::thrift::protocol::TType _vtype1421;
-            xfer += iprot->readMapBegin(_ktype1420, _vtype1421, _size1419);
-            uint32_t _i1423;
-            for (_i1423 = 0; _i1423 < _size1419; ++_i1423)
+            uint32_t _size1443;
+            ::apache::thrift::protocol::TType _ktype1444;
+            ::apache::thrift::protocol::TType _vtype1445;
+            xfer += iprot->readMapBegin(_ktype1444, _vtype1445, _size1443);
+            uint32_t _i1447;
+            for (_i1447 = 0; _i1447 < _size1443; ++_i1447)
             {
-              std::string _key1424;
-              xfer += iprot->readString(_key1424);
-              Materialization& _val1425 = this->success[_key1424];
-              xfer += _val1425.read(iprot);
+              std::string _key1448;
+              xfer += iprot->readString(_key1448);
+              Materialization& _val1449 = this->success[_key1448];
+              xfer += _val1449.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -10776,11 +10776,11 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_result::write
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Materialization> ::const_iterator _iter1426;
-      for (_iter1426 = this->success.begin(); _iter1426 != this->success.end(); ++_iter1426)
+      std::map<std::string, Materialization> ::const_iterator _iter1450;
+      for (_iter1450 = this->success.begin(); _iter1450 != this->success.end(); ++_iter1450)
       {
-        xfer += oprot->writeString(_iter1426->first);
-        xfer += _iter1426->second.write(oprot);
+        xfer += oprot->writeString(_iter1450->first);
+        xfer += _iter1450->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -10833,17 +10833,17 @@ uint32_t ThriftHiveMetastore_get_materialization_invalidation_info_presult::read
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size1427;
-            ::apache::thrift::protocol::TType _ktype1428;
-            ::apache::thrift::protocol::TType _vtype1429;
-            xfer += iprot->readMapBegin(_ktype1428, _vtype1429, _size1427);
-            uint32_t _i1431;
-            for (_i1431 = 0; _i1431 < _size1427; ++_i1431)
+            uint32_t _size1451;
+            ::apache::thrift::protocol::TType _ktype1452;
+            ::apache::thrift::protocol::TType _vtype1453;
+            xfer += iprot->readMapBegin(_ktype1452, _vtype1453, _size1451);
+            uint32_t _i1455;
+            for (_i1455 = 0; _i1455 < _size1451; ++_i1455)
             {
-              std::string _key1432;
-              xfer += iprot->readString(_key1432);
-              Materialization& _val1433 = (*(this->success))[_key1432];
-              xfer += _val1433.read(iprot);
+              std::string _key1456;
+              xfer += iprot->readString(_key1456);
+              Materialization& _val1457 = (*(this->success))[_key1456];
+              xfer += _val1457.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11304,14 +11304,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size1434;
-            ::apache::thrift::protocol::TType _etype1437;
-            xfer += iprot->readListBegin(_etype1437, _size1434);
-            this->success.resize(_size1434);
-            uint32_t _i1438;
-            for (_i1438 = 0; _i1438 < _size1434; ++_i1438)
+            uint32_t _size1458;
+            ::apache::thrift::protocol::TType _etype1461;
+            xfer += iprot->readListBegin(_etype1461, _size1458);
+            this->success.resize(_size1458);
+            uint32_t _i1462;
+            for (_i1462 = 0; _i1462 < _size1458; ++_i1462)
             {
-              xfer += iprot->readString(this->success[_i1438]);
+              xfer += iprot->readString(this->success[_i1462]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11366,10 +11366,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter1439;
-      for (_iter1439 = this->success.begin(); _iter1439 != this->success.end(); ++_iter1439)
+      std::vector<std::string> ::const_iterator _iter1463;
+      for (_iter1463 = this->success.begin(); _iter1463 != this->success.end(); ++_iter1463)
       {
-        xfer += oprot->writeString((*_iter1439));
+        xfer += oprot->writeString((*_iter1463));
       }
       xfer += oprot->writeListEnd();
     }
@@ -11422,14 +11422,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size1440;
-            ::apache::thrift::protocol::TType _etype1443;
-            xfer += iprot->readListBegin(_etype1443, _size1440);
-            (*(this->success)).resize(_size1440);
-            uint32_t _i1444;
-            for (_i1444 = 0; _i1444 < _size1440; ++_i1444)
+            uint32_t _size1464;
+            ::apache::thrift::protocol::TType _etype1467;
+            xfer += iprot->readListBegin(_etype1467, _size1464);
+            (*(this->success)).resize(_size1464);
+            uint32_t _i1468;
+            for (_i1468 = 0; _i1468 < _size1464; ++_i1468)
             {
-              xfer += iprot->readString((*(this->success))[_i1444]);
+              xfer += iprot->readString((*(this->success))[_i1468]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12763,14 +12763,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1445;
-            ::apache::thrift::protocol::TType _etype1448;
-            xfer += iprot->readListBegin(_etype1448, _size1445);
-            this->new_parts.resize(_size1445);
-            uint32_t _i1449;
-            for (_i1449 = 0; _i1449 < _size1445; ++_i1449)
+            uint32_t _size1469;
+            ::apache::thrift::protocol::TType _etype1472;
+            xfer += iprot->readListBegin(_etype1472, _size1469);
+            this->new_parts.resize(_size1469);
+            uint32_t _i1473;
+            for (_i1473 = 0; _i1473 < _size1469; ++_i1473)
             {
-              xfer += this->new_parts[_i1449].read(iprot);
+              xfer += this->new_parts[_i1473].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12799,10 +12799,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter1450;
-    for (_iter1450 = this->new_parts.begin(); _iter1450 != this->new_parts.end(); ++_iter1450)
+    std::vector<Partition> ::const_iterator _iter1474;
+    for (_iter1474 = this->new_parts.begin(); _iter1474 != this->new_parts.end(); ++_iter1474)
     {
-      xfer += (*_iter1450).write(oprot);
+      xfer += (*_iter1474).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12826,10 +12826,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter1451;
-    for (_iter1451 = (*(this->new_parts)).begin(); _iter1451 != (*(this->new_parts)).end(); ++_iter1451)
+    std::vector<Partition> ::const_iterator _iter1475;
+    for (_iter1475 = (*(this->new_parts)).begin(); _iter1475 != (*(this->new_parts)).end(); ++_iter1475)
     {
-      xfer += (*_iter1451).write(oprot);
+      xfer += (*_iter1475).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13038,14 +13038,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size1452;
-            ::apache::thrift::protocol::TType _etype1455;
-            xfer += iprot->readListBegin(_etype1455, _size1452);
-            this->new_parts.resize(_size1452);
-            uint32_t _i1456;
-            for (_i1456 = 0; _i1456 < _size1452; ++_i1456)
+            uint32_t _size1476;
+            ::apache::thrift::protocol::TType _etype1479;
+            xfer += iprot->readListBegin(_etype1479, _size1476);
+            this->new_parts.resize(_size1476);
+            uint32_t _i1480;
+            for (_i1480 = 0; _i1480 < _size1476; ++_i1480)
             {
-              xfer += this->new_parts[_i1456].read(iprot);
+              xfer += this->new_parts[_i1480].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13074,10 +13074,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter1457;
-    for (_iter1457 = this->new_parts.begin(); _iter1457 != this->new_parts.end(); ++_iter1457)
+    std::vector<PartitionSpec> ::const_iterator _iter1481;
+    for (_iter1481 = this->new_parts.begin(); _iter1481 != this->new_parts.end(); ++_iter1481)
     {
-      xfer += (*_iter1457).write(oprot);
+      xfer += (*_iter1481).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13101,10 +13101,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter1458;
-    for (_iter1458 = (*(this->new_parts)).begin(); _iter1458 != (*(this->new_parts)).end(); ++_iter1458)
+    std::vector<PartitionSpec> ::const_iterator _iter1482;
+    for (_iter1482 = (*(this->new_parts)).begin(); _iter1482 != (*(this->new_parts)).end(); ++_iter1482)
     {
-      xfer += (*_iter1458).write(oprot);
+      xfer += (*_iter1482).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13329,14 +13329,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1459;
-            ::apache::thrift::protocol::TType _etype1462;
-            xfer += iprot->readListBegin(_etype1462, _size1459);
-            this->part_vals.resize(_size1459);
-            uint32_t _i1463;
-            for (_i1463 = 0; _i1463 < _size1459; ++_i1463)
+            uint32_t _size1483;
+            ::apache::thrift::protocol::TType _etype1486;
+            xfer += iprot->readListBegin(_etype1486, _size1483);
+            this->part_vals.resize(_size1483);
+            uint32_t _i1487;
+            for (_i1487 = 0; _i1487 < _size1483; ++_i1487)
             {
-              xfer += iprot->readString(this->part_vals[_i1463]);
+              xfer += iprot->readString(this->part_vals[_i1487]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13373,10 +13373,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1464;
-    for (_iter1464 = this->part_vals.begin(); _iter1464 != this->part_vals.end(); ++_iter1464)
+    std::vector<std::string> ::const_iterator _iter1488;
+    for (_iter1488 = this->part_vals.begin(); _iter1488 != this->part_vals.end(); ++_iter1488)
     {
-      xfer += oprot->writeString((*_iter1464));
+      xfer += oprot->writeString((*_iter1488));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13408,10 +13408,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1465;
-    for (_iter1465 = (*(this->part_vals)).begin(); _iter1465 != (*(this->part_vals)).end(); ++_iter1465)
+    std::vector<std::string> ::const_iterator _iter1489;
+    for (_iter1489 = (*(this->part_vals)).begin(); _iter1489 != (*(this->part_vals)).end(); ++_iter1489)
     {
-      xfer += oprot->writeString((*_iter1465));
+      xfer += oprot->writeString((*_iter1489));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13883,14 +13883,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1466;
-            ::apache::thrift::protocol::TType _etype1469;
-            xfer += iprot->readListBegin(_etype1469, _size1466);
-            this->part_vals.resize(_size1466);
-            uint32_t _i1470;
-            for (_i1470 = 0; _i1470 < _size1466; ++_i1470)
+            uint32_t _size1490;
+            ::apache::thrift::protocol::TType _etype1493;
+            xfer += iprot->readListBegin(_etype1493, _size1490);
+            this->part_vals.resize(_size1490);
+            uint32_t _i1494;
+            for (_i1494 = 0; _i1494 < _size1490; ++_i1494)
             {
-              xfer += iprot->readString(this->part_vals[_i1470]);
+              xfer += iprot->readString(this->part_vals[_i1494]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13935,10 +13935,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1471;
-    for (_iter1471 = this->part_vals.begin(); _iter1471 != this->part_vals.end(); ++_iter1471)
+    std::vector<std::string> ::const_iterator _iter1495;
+    for (_iter1495 = this->part_vals.begin(); _iter1495 != this->part_vals.end(); ++_iter1495)
     {
-      xfer += oprot->writeString((*_iter1471));
+      xfer += oprot->writeString((*_iter1495));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13974,10 +13974,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1472;
-    for (_iter1472 = (*(this->part_vals)).begin(); _iter1472 != (*(this->part_vals)).end(); ++_iter1472)
+    std::vector<std::string> ::const_iterator _iter1496;
+    for (_iter1496 = (*(this->part_vals)).begin(); _iter1496 != (*(this->part_vals)).end(); ++_iter1496)
     {
-      xfer += oprot->writeString((*_iter1472));
+      xfer += oprot->writeString((*_iter1496));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14780,14 +14780,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1473;
-            ::apache::thrift::protocol::TType _etype1476;
-            xfer += iprot->readListBegin(_etype1476, _size1473);
-            this->part_vals.resize(_size1473);
-            uint32_t _i1477;
-            for (_i1477 = 0; _i1477 < _size1473; ++_i1477)
+            uint32_t _size1497;
+            ::apache::thrift::protocol::TType _etype1500;
+            xfer += iprot->readListBegin(_etype1500, _size1497);
+            this->part_vals.resize(_size1497);
+            uint32_t _i1501;
+            for (_i1501 = 0; _i1501 < _size1497; ++_i1501)
             {
-              xfer += iprot->readString(this->part_vals[_i1477]);
+              xfer += iprot->readString(this->part_vals[_i1501]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14832,10 +14832,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1478;
-    for (_iter1478 = this->part_vals.begin(); _iter1478 != this->part_vals.end(); ++_iter1478)
+    std::vector<std::string> ::const_iterator _iter1502;
+    for (_iter1502 = this->part_vals.begin(); _iter1502 != this->part_vals.end(); ++_iter1502)
     {
-      xfer += oprot->writeString((*_iter1478));
+      xfer += oprot->writeString((*_iter1502));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14871,10 +14871,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1479;
-    for (_iter1479 = (*(this->part_vals)).begin(); _iter1479 != (*(this->part_vals)).end(); ++_iter1479)
+    std::vector<std::string> ::const_iterator _iter1503;
+    for (_iter1503 = (*(this->part_vals)).begin(); _iter1503 != (*(this->part_vals)).end(); ++_iter1503)
     {
-      xfer += oprot->writeString((*_iter1479));
+      xfer += oprot->writeString((*_iter1503));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15083,14 +15083,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1480;
-            ::apache::thrift::protocol::TType _etype1483;
-            xfer += iprot->readListBegin(_etype1483, _size1480);
-            this->part_vals.resize(_size1480);
-            uint32_t _i1484;
-            for (_i1484 = 0; _i1484 < _size1480; ++_i1484)
+            uint32_t _size1504;
+            ::apache::thrift::protocol::TType _etype1507;
+            xfer += iprot->readListBegin(_etype1507, _size1504);
+            this->part_vals.resize(_size1504);
+            uint32_t _i1508;
+            for (_i1508 = 0; _i1508 < _size1504; ++_i1508)
             {
-              xfer += iprot->readString(this->part_vals[_i1484]);
+              xfer += iprot->readString(this->part_vals[_i1508]);
             }
             xfer += iprot->readListEnd();
           }
@@ -15143,10 +15143,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1485;
-    for (_iter1485 = this->part_vals.begin(); _iter1485 != this->part_vals.end(); ++_iter1485)
+    std::vector<std::string> ::const_iterator _iter1509;
+    for (_iter1509 = this->part_vals.begin(); _iter1509 != this->part_vals.end(); ++_iter1509)
     {
-      xfer += oprot->writeString((*_iter1485));
+      xfer += oprot->writeString((*_iter1509));
     }
     xfer += oprot->writeListEnd();
   }
@@ -15186,10 +15186,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1486;
-    for (_iter1486 = (*(this->part_vals)).begin(); _iter1486 != (*(this->part_vals)).end(); ++_iter1486)
+    std::vector<std::string> ::const_iterator _iter1510;
+    for (_iter1510 = (*(this->part_vals)).begin(); _iter1510 != (*(this->part_vals)).end(); ++_iter1510)
     {
-      xfer += oprot->writeString((*_iter1486));
+      xfer += oprot->writeString((*_iter1510));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16195,14 +16195,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size1487;
-            ::apache::thrift::protocol::TType _etype1490;
-            xfer += iprot->readListBegin(_etype1490, _size1487);
-            this->part_vals.resize(_size1487);
-            uint32_t _i1491;
-            for (_i1491 = 0; _i1491 < _size1487; ++_i1491)
+            uint32_t _size1511;
+            ::apache::thrift::protocol::TType _etype1514;
+            xfer += iprot->readListBegin(_etype1514, _size1511);
+            this->part_vals.resize(_size1511);
+            uint32_t _i1515;
+            for (_i1515 = 0; _i1515 < _size1511; ++_i1515)
             {
-              xfer += iprot->readString(this->part_vals[_i1491]);
+              xfer += iprot->readString(this->part_vals[_i1515]);
             }
             xfer += iprot->readListEnd();
           }
@@ -16239,10 +16239,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter1492;
-    for (_iter1492 = this->part_vals.begin(); _iter1492 != this->part_vals.end(); ++_iter1492)
+    std::vector<std::string> ::const_iterator _iter1516;
+    for (_iter1516 = this->part_vals.begin(); _iter1516 != this->part_vals.end(); ++_iter1516)
     {
-      xfer += oprot->writeString((*_iter1492));
+      xfer += oprot->writeString((*_iter1516));
     }
     xfer += oprot->writeListEnd();
   }
@@ -16274,10 +16274,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter1493;
-    for (_iter1493 = (*(this->part_vals)).begin(); _iter1493 != (*(this->part_vals)).end(); ++_iter1493)
+    std::vector<std::string> ::const_iterator _iter1517;
+    for (_iter1517 = (*(this->part_vals)).begin(); _iter1517 != (*(this->part_vals)).end(); ++_iter1517)
     {
-      xfer += oprot->writeString((*_iter1493));
+      xfer += oprot->writeString((*_iter1517));
     }
     xfer += oprot->writeListEnd();
 

<TRUNCATED>

[13/46] hive git commit: HIVE-19711 Refactor Hive Schema Tool (Miklos Gergely via Alan Gates).

Posted by se...@apache.org.
HIVE-19711 Refactor Hive Schema Tool (Miklos Gergely via Alan Gates).


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

Branch: refs/heads/master-txnstats
Commit: d83a0be9852467b3b8b3bef84721bb49e63f57b8
Parents: e7d1781
Author: Alan Gates <ga...@hortonworks.com>
Authored: Mon Jul 2 12:58:50 2018 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Mon Jul 2 12:58:50 2018 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/beeline/BeeLine.java   |    2 +-
 .../org/apache/hive/beeline/BeeLineOpts.java    |    2 +-
 .../org/apache/hive/beeline/HiveSchemaTool.java | 1563 ------------------
 .../hive/beeline/schematool/HiveSchemaTool.java |  415 +++++
 .../schematool/HiveSchemaToolCommandLine.java   |  286 ++++
 .../beeline/schematool/HiveSchemaToolTask.java  |   32 +
 .../HiveSchemaToolTaskAlterCatalog.java         |   90 +
 .../HiveSchemaToolTaskCreateCatalog.java        |  132 ++
 .../schematool/HiveSchemaToolTaskInfo.java      |   43 +
 .../schematool/HiveSchemaToolTaskInit.java      |   73 +
 .../HiveSchemaToolTaskMoveDatabase.java         |   96 ++
 .../schematool/HiveSchemaToolTaskMoveTable.java |  142 ++
 .../schematool/HiveSchemaToolTaskUpgrade.java   |  116 ++
 .../schematool/HiveSchemaToolTaskValidate.java  |  631 +++++++
 .../apache/hive/beeline/TestHiveSchemaTool.java |   92 --
 .../beeline/schematool/TestHiveSchemaTool.java  |   92 ++
 bin/ext/schemaTool.sh                           |    4 +-
 .../org/apache/hive/beeline/TestSchemaTool.java |  801 ---------
 .../hive/beeline/TestSchemaToolCatalogOps.java  |  417 -----
 .../hive/beeline/schematool/TestSchemaTool.java |  826 +++++++++
 .../schematool/TestSchemaToolCatalogOps.java    |  478 ++++++
 21 files changed, 3456 insertions(+), 2877 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/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 e1efa34..4eda8e3 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLine.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLine.java
@@ -172,7 +172,7 @@ public class BeeLine implements Closeable {
   private static final String HIVE_VAR_PREFIX = "--hivevar";
   private static final String HIVE_CONF_PREFIX = "--hiveconf";
   private static final String PROP_FILE_PREFIX = "--property-file";
-  static final String PASSWD_MASK = "[passwd stripped]";
+  public static final String PASSWD_MASK = "[passwd stripped]";
 
   private final Map<Object, Object> formats = map(new Object[] {
       "vertical", new VerticalOutputFormat(this),

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/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 85052d9..3877b5c 100644
--- a/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
+++ b/beeline/src/java/org/apache/hive/beeline/BeeLineOpts.java
@@ -48,7 +48,7 @@ import jline.console.completer.StringsCompleter;
 import jline.console.history.MemoryHistory;
 import org.apache.hadoop.hive.conf.HiveConf;
 
-class BeeLineOpts implements Completer {
+public class BeeLineOpts implements Completer {
   public static final int DEFAULT_MAX_WIDTH = 80;
   public static final int DEFAULT_MAX_HEIGHT = 80;
   public static final int DEFAULT_HEADER_INTERVAL = 100;

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
deleted file mode 100644
index 2cebe72..0000000
--- a/beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java
+++ /dev/null
@@ -1,1563 +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.hive.beeline;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.GnuParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.commons.cli.OptionGroup;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.io.output.NullOutputStream;
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
-import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
-import org.apache.hadoop.hive.metastore.TableType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.ImmutableMap;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import static org.apache.hadoop.hive.metastore.utils.StringUtils.normalizeIdentifier;
-
-public class HiveSchemaTool {
-  private String userName = null;
-  private String passWord = null;
-  private boolean dryRun = false;
-  private boolean verbose = false;
-  private String dbOpts = null;
-  private String url = null;
-  private String driver = null;
-  private URI[] validationServers = null; // The list of servers the database/partition/table can locate on
-  private final HiveConf hiveConf;
-  private final String dbType;
-  private final String metaDbType;
-  private final IMetaStoreSchemaInfo metaStoreSchemaInfo;
-  private boolean needsQuotedIdentifier;
-  private String quoteCharacter;
-
-  static final private Logger LOG = LoggerFactory.getLogger(HiveSchemaTool.class.getName());
-
-  public HiveSchemaTool(String dbType, String metaDbType) throws HiveMetaException {
-    this(System.getenv("HIVE_HOME"), new HiveConf(HiveSchemaTool.class), dbType, metaDbType);
-  }
-
-  public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType, String metaDbType)
-      throws HiveMetaException {
-    if (hiveHome == null || hiveHome.isEmpty()) {
-      throw new HiveMetaException("No Hive home directory provided");
-    }
-    this.hiveConf = hiveConf;
-    this.dbType = dbType;
-    this.metaDbType = metaDbType;
-    NestedScriptParser parser = getDbCommandParser(dbType, metaDbType);
-    this.needsQuotedIdentifier = parser.needsQuotedIdentifier();
-    this.quoteCharacter = parser.getQuoteCharacter();
-    this.metaStoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf, hiveHome, dbType);
-    // If the dbType is "hive", this is setting up the information schema in Hive. 
-    // We will set the default jdbc url and driver.
-    // It is overriden by command line options if passed (-url and -driver
-    if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
-      url = HiveSchemaHelper.EMBEDDED_HS2_URL;
-      driver = HiveSchemaHelper.HIVE_JDBC_DRIVER;
-    }
-  }
-
-  public HiveConf getHiveConf() {
-    return hiveConf;
-  }
-
-  public void setUrl(String url) {
-    this.url = url;
-  }
-
-  public void setDriver(String driver) {
-    this.driver = driver;
-  }
-
-  public void setUserName(String userName) {
-    this.userName = userName;
-  }
-
-  public void setPassWord(String passWord) {
-    this.passWord = passWord;
-  }
-
-  public void setDryRun(boolean dryRun) {
-    this.dryRun = dryRun;
-  }
-
-  public void setVerbose(boolean verbose) {
-    this.verbose = verbose;
-  }
-
-  public void setDbOpts(String dbOpts) {
-    this.dbOpts = dbOpts;
-  }
-
-  public void setValidationServers(String servers) {
-    if(StringUtils.isNotEmpty(servers)) {
-      String[] strServers = servers.split(",");
-      this.validationServers = new URI[strServers.length];
-      for (int i = 0; i < validationServers.length; i++) {
-        validationServers[i] = new Path(strServers[i]).toUri();
-      }
-    }
-  }
-
-  private static void printAndExit(Options cmdLineOptions) {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("schemaTool", cmdLineOptions);
-    System.exit(1);
-  }
-
-  Connection getConnectionToMetastore(boolean printInfo) throws HiveMetaException {
-    return HiveSchemaHelper.getConnectionToMetastore(userName, passWord, url, driver, printInfo, hiveConf,
-        null);
-  }
-
-  private NestedScriptParser getDbCommandParser(String dbType, String metaDbType) {
-    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName, passWord, hiveConf,
-        metaDbType, false);
-  }
-
-  /***
-   * Print Hive version and schema version
-   * @throws MetaException
-   */
-  public void showInfo() throws HiveMetaException {
-    String hiveVersion = metaStoreSchemaInfo.getHiveSchemaVersion();
-    String dbVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(true));
-    System.out.println("Hive distribution version:\t " + hiveVersion);
-    System.out.println("Metastore schema version:\t " + dbVersion);
-    assertCompatibleVersion(hiveVersion, dbVersion);
-  }
-
-  boolean validateLocations(Connection conn, URI[] defaultServers) throws HiveMetaException {
-    System.out.println("Validating DFS locations");
-    boolean rtn;
-    rtn = checkMetaStoreDBLocation(conn, defaultServers);
-    rtn = checkMetaStoreTableLocation(conn, defaultServers) && rtn;
-    rtn = checkMetaStorePartitionLocation(conn, defaultServers) && rtn;
-    rtn = checkMetaStoreSkewedColumnsLocation(conn, defaultServers) && rtn;
-    System.out.println((rtn ? "Succeeded" : "Failed") + " in DFS location validation.");
-    return rtn;
-  }
-
-  private String getNameOrID(ResultSet res, int nameInx, int idInx) throws SQLException {
-    String itemName = res.getString(nameInx);
-    return  (itemName == null || itemName.isEmpty()) ? "ID: " + res.getString(idInx) : "Name: " + itemName;
-  }
-
-  private boolean checkMetaStoreDBLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String dbLoc;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      dbLoc = "select dbt.\"DB_ID\", dbt.\"NAME\", dbt.\"DB_LOCATION_URI\" from \"DBS\" dbt order by dbt.\"DB_ID\" ";
-    } else {
-      dbLoc = "select dbt.DB_ID, dbt.NAME, dbt.DB_LOCATION_URI from DBS dbt order by dbt.DB_ID";
-    }
-
-    try(Statement stmt = conn.createStatement();
-        ResultSet res = stmt.executeQuery(dbLoc)) {
-      while (res.next()) {
-        String locValue = res.getString(3);
-        String dbName = getNameOrID(res,2,1);
-        if (!checkLocation("Database " + dbName, locValue, defaultServers)) {
-          numOfInvalid++;
-        }
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get DB Location Info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
-  }
-
-  private boolean checkMetaStoreTableLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String tabLoc, tabIDRange;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      tabIDRange = "select max(\"TBL_ID\"), min(\"TBL_ID\") from \"TBLS\" ";
-    } else {
-      tabIDRange = "select max(TBL_ID), min(TBL_ID) from TBLS";
-    }
-
-    if (needsQuotedIdentifier) {
-      tabLoc = "select tbl.\"TBL_ID\", tbl.\"TBL_NAME\", sd.\"LOCATION\", dbt.\"DB_ID\", dbt.\"NAME\" from \"TBLS\" tbl inner join " +
-    "\"SDS\" sd on tbl.\"SD_ID\" = sd.\"SD_ID\" and tbl.\"TBL_TYPE\" != '" + TableType.VIRTUAL_VIEW +
-    "' and tbl.\"TBL_ID\" >= ? and tbl.\"TBL_ID\"<= ? " + "inner join \"DBS\" dbt on tbl.\"DB_ID\" = dbt.\"DB_ID\" order by tbl.\"TBL_ID\" ";
-    } else {
-      tabLoc = "select tbl.TBL_ID, tbl.TBL_NAME, sd.LOCATION, dbt.DB_ID, dbt.NAME from TBLS tbl join SDS sd on tbl.SD_ID = sd.SD_ID and tbl.TBL_TYPE !='"
-      + TableType.VIRTUAL_VIEW + "' and tbl.TBL_ID >= ? and tbl.TBL_ID <= ?  inner join DBS dbt on tbl.DB_ID = dbt.DB_ID order by tbl.TBL_ID";
-    }
-
-    long maxID = 0, minID = 0;
-    long rtnSize = 2000;
-
-    try {
-      Statement stmt = conn.createStatement();
-      ResultSet res = stmt.executeQuery(tabIDRange);
-      if (res.next()) {
-        maxID = res.getLong(1);
-        minID = res.getLong(2);
-      }
-      res.close();
-      stmt.close();
-      PreparedStatement pStmt = conn.prepareStatement(tabLoc);
-      while (minID <= maxID) {
-        pStmt.setLong(1, minID);
-        pStmt.setLong(2, minID + rtnSize);
-        res = pStmt.executeQuery();
-        while (res.next()) {
-          String locValue = res.getString(3);
-          String entity = "Database " + getNameOrID(res, 5, 4) +
-              ", Table "  + getNameOrID(res,2,1);
-          if (!checkLocation(entity, locValue, defaultServers)) {
-            numOfInvalid++;
-          }
-        }
-        res.close();
-        minID += rtnSize + 1;
-
-      }
-      pStmt.close();
-
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get Table Location Info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
-  }
-
-  private boolean checkMetaStorePartitionLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String partLoc, partIDRange;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      partIDRange = "select max(\"PART_ID\"), min(\"PART_ID\") from \"PARTITIONS\" ";
-    } else {
-      partIDRange = "select max(PART_ID), min(PART_ID) from PARTITIONS";
-    }
-
-    if (needsQuotedIdentifier) {
-      partLoc = "select pt.\"PART_ID\", pt.\"PART_NAME\", sd.\"LOCATION\", tbl.\"TBL_ID\", tbl.\"TBL_NAME\",dbt.\"DB_ID\", dbt.\"NAME\" from \"PARTITIONS\" pt "
-           + "inner join \"SDS\" sd on pt.\"SD_ID\" = sd.\"SD_ID\" and pt.\"PART_ID\" >= ? and pt.\"PART_ID\"<= ? "
-           + " inner join \"TBLS\" tbl on pt.\"TBL_ID\" = tbl.\"TBL_ID\" inner join "
-           + "\"DBS\" dbt on tbl.\"DB_ID\" = dbt.\"DB_ID\" order by tbl.\"TBL_ID\" ";
-    } else {
-      partLoc = "select pt.PART_ID, pt.PART_NAME, sd.LOCATION, tbl.TBL_ID, tbl.TBL_NAME, dbt.DB_ID, dbt.NAME from PARTITIONS pt "
-          + "inner join SDS sd on pt.SD_ID = sd.SD_ID and pt.PART_ID >= ? and pt.PART_ID <= ?  "
-          + "inner join TBLS tbl on tbl.TBL_ID = pt.TBL_ID inner join DBS dbt on tbl.DB_ID = dbt.DB_ID order by tbl.TBL_ID ";
-    }
-
-    long maxID = 0, minID = 0;
-    long rtnSize = 2000;
-
-    try {
-      Statement stmt = conn.createStatement();
-      ResultSet res = stmt.executeQuery(partIDRange);
-      if (res.next()) {
-        maxID = res.getLong(1);
-        minID = res.getLong(2);
-      }
-      res.close();
-      stmt.close();
-      PreparedStatement pStmt = conn.prepareStatement(partLoc);
-      while (minID <= maxID) {
-        pStmt.setLong(1, minID);
-        pStmt.setLong(2, minID + rtnSize);
-        res = pStmt.executeQuery();
-        while (res.next()) {
-          String locValue = res.getString(3);
-          String entity = "Database " + getNameOrID(res,7,6) +
-              ", Table "  + getNameOrID(res,5,4) +
-              ", Partition " + getNameOrID(res,2,1);
-          if (!checkLocation(entity, locValue, defaultServers)) {
-            numOfInvalid++;
-          }
-        }
-        res.close();
-        minID += rtnSize + 1;
-      }
-      pStmt.close();
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get Partition Location Info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
-  }
-
-  private boolean checkMetaStoreSkewedColumnsLocation(Connection conn, URI[] defaultServers)
-      throws HiveMetaException {
-    String skewedColLoc, skewedColIDRange;
-    boolean isValid = true;
-    int numOfInvalid = 0;
-    if (needsQuotedIdentifier) {
-      skewedColIDRange = "select max(\"STRING_LIST_ID_KID\"), min(\"STRING_LIST_ID_KID\") from \"SKEWED_COL_VALUE_LOC_MAP\" ";
-    } else {
-      skewedColIDRange = "select max(STRING_LIST_ID_KID), min(STRING_LIST_ID_KID) from SKEWED_COL_VALUE_LOC_MAP";
-    }
-
-    if (needsQuotedIdentifier) {
-      skewedColLoc = "select t.\"TBL_NAME\", t.\"TBL_ID\", sk.\"STRING_LIST_ID_KID\", sk.\"LOCATION\", db.\"NAME\", db.\"DB_ID\" "
-           + " from \"TBLS\" t, \"SDS\" s, \"DBS\" db, \"SKEWED_COL_VALUE_LOC_MAP\" sk "
-           + "where sk.\"SD_ID\" = s.\"SD_ID\" and s.\"SD_ID\" = t.\"SD_ID\" and t.\"DB_ID\" = db.\"DB_ID\" and "
-           + "sk.\"STRING_LIST_ID_KID\" >= ? and sk.\"STRING_LIST_ID_KID\" <= ? order by t.\"TBL_ID\" ";
-    } else {
-      skewedColLoc = "select t.TBL_NAME, t.TBL_ID, sk.STRING_LIST_ID_KID, sk.LOCATION, db.NAME, db.DB_ID from TBLS t, SDS s, DBS db, SKEWED_COL_VALUE_LOC_MAP sk "
-           + "where sk.SD_ID = s.SD_ID and s.SD_ID = t.SD_ID and t.DB_ID = db.DB_ID and sk.STRING_LIST_ID_KID >= ? and sk.STRING_LIST_ID_KID <= ? order by t.TBL_ID ";
-    }
-
-    long maxID = 0, minID = 0;
-    long rtnSize = 2000;
-
-    try {
-      Statement stmt = conn.createStatement();
-      ResultSet res = stmt.executeQuery(skewedColIDRange);
-      if (res.next()) {
-        maxID = res.getLong(1);
-        minID = res.getLong(2);
-      }
-      res.close();
-      stmt.close();
-      PreparedStatement pStmt = conn.prepareStatement(skewedColLoc);
-      while (minID <= maxID) {
-        pStmt.setLong(1, minID);
-        pStmt.setLong(2, minID + rtnSize);
-        res = pStmt.executeQuery();
-        while (res.next()) {
-          String locValue = res.getString(4);
-          String entity = "Database " + getNameOrID(res,5,6) +
-              ", Table " + getNameOrID(res,1,2) +
-              ", String list " + res.getString(3);
-          if (!checkLocation(entity, locValue, defaultServers)) {
-            numOfInvalid++;
-          }
-        }
-        res.close();
-        minID += rtnSize + 1;
-      }
-      pStmt.close();
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to get skewed columns location info.", e);
-    }
-    if (numOfInvalid > 0) {
-      isValid = false;
-    }
-    return isValid;
-  }
-
-  /**
-   * Check if the location is valid for the given entity
-   * @param entity          the entity to represent a database, partition or table
-   * @param entityLocation  the location
-   * @param defaultServers  a list of the servers that the location needs to match.
-   *                        The location host needs to match one of the given servers.
-   *                        If empty, then no check against such list.
-   * @return true if the location is valid
-   */
-  private boolean checkLocation(
-      String entity,
-      String entityLocation,
-      URI[] defaultServers) {
-    boolean isValid = true;
-    if (entityLocation == null) {
-      System.err.println(entity + ", Error: empty location");
-      isValid = false;
-    } else {
-      try {
-        URI currentUri = new Path(entityLocation).toUri();
-        String scheme = currentUri.getScheme();
-        String path   = currentUri.getPath();
-        if (StringUtils.isEmpty(scheme)) {
-          System.err.println(entity + ", Location: "+ entityLocation + ", Error: missing location scheme.");
-          isValid = false;
-        } else if (StringUtils.isEmpty(path)) {
-          System.err.println(entity + ", Location: "+ entityLocation + ", Error: missing location path.");
-          isValid = false;
-        } else if (ArrayUtils.isNotEmpty(defaultServers) && currentUri.getAuthority() != null) {
-          String authority = currentUri.getAuthority();
-          boolean matchServer = false;
-          for(URI server : defaultServers) {
-            if (StringUtils.equalsIgnoreCase(server.getScheme(), scheme) &&
-                StringUtils.equalsIgnoreCase(server.getAuthority(), authority)) {
-              matchServer = true;
-              break;
-            }
-          }
-          if (!matchServer) {
-            System.err.println(entity + ", Location: " + entityLocation + ", Error: mismatched server.");
-            isValid = false;
-          }
-        }
-
-        // if there is no path element other than "/", report it but not fail
-        if (isValid && StringUtils.containsOnly(path, "/")) {
-          System.err.println(entity + ", Location: "+ entityLocation + ", Warn: location set to root, not a recommended config.");
-        }
-      } catch (Exception pe) {
-        System.err.println(entity + ", Error: invalid location - " + pe.getMessage());
-        isValid =false;
-      }
-    }
-
-    return isValid;
-  }
-
-  // test the connection metastore using the config property
-  private void testConnectionToMetastore() throws HiveMetaException {
-    Connection conn = getConnectionToMetastore(true);
-    try {
-      conn.close();
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to close metastore connection", e);
-    }
-  }
-
-
-  /**
-   * check if the current schema version in metastore matches the Hive version
-   * @throws MetaException
-   */
-  public void verifySchemaVersion() throws HiveMetaException {
-    // don't check version if its a dry run
-    if (dryRun) {
-      return;
-    }
-    String newSchemaVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-    // verify that the new version is added to schema
-    assertCompatibleVersion(metaStoreSchemaInfo.getHiveSchemaVersion(), newSchemaVersion);
-  }
-
-  private void assertCompatibleVersion(String hiveSchemaVersion, String dbSchemaVersion)
-      throws HiveMetaException {
-    if (!metaStoreSchemaInfo.isVersionCompatible(hiveSchemaVersion, dbSchemaVersion)) {
-      throw new HiveMetaException("Metastore schema version is not compatible. Hive Version: "
-          + hiveSchemaVersion + ", Database Schema Version: " + dbSchemaVersion);
-    }
-  }
-
-  /**
-   * Perform metastore schema upgrade. extract the current schema version from metastore
-   * @throws MetaException
-   */
-  public void doUpgrade() throws HiveMetaException {
-    String fromVersion =
-      metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-    if (fromVersion == null || fromVersion.isEmpty()) {
-      throw new HiveMetaException("Schema version not stored in the metastore. " +
-          "Metastore schema is too old or corrupt. Try specifying the version manually");
-    }
-    doUpgrade(fromVersion);
-  }
-
-  private MetaStoreConnectionInfo getConnectionInfo(boolean printInfo) {
-    return new MetaStoreConnectionInfo(userName, passWord, url, driver, printInfo, hiveConf,
-        dbType, metaDbType);
-  }
-  /**
-   * Perform metastore schema upgrade
-   *
-   * @param fromSchemaVer
-   *          Existing version of the metastore. If null, then read from the metastore
-   * @throws MetaException
-   */
-  public void doUpgrade(String fromSchemaVer) throws HiveMetaException {
-    if (metaStoreSchemaInfo.getHiveSchemaVersion().equals(fromSchemaVer)) {
-      System.out.println("No schema upgrade required from version " + fromSchemaVer);
-      return;
-    }
-    // Find the list of scripts to execute for this upgrade
-    List<String> upgradeScripts =
-        metaStoreSchemaInfo.getUpgradeScripts(fromSchemaVer);
-    testConnectionToMetastore();
-    System.out.println("Starting upgrade metastore schema from version " +
-        fromSchemaVer + " to " + metaStoreSchemaInfo.getHiveSchemaVersion());
-    String scriptDir = metaStoreSchemaInfo.getMetaStoreScriptDir();
-    try {
-      for (String scriptFile : upgradeScripts) {
-        System.out.println("Upgrade script " + scriptFile);
-        if (!dryRun) {
-          runPreUpgrade(scriptDir, scriptFile);
-          runBeeLine(scriptDir, scriptFile);
-          System.out.println("Completed " + scriptFile);
-        }
-      }
-    } catch (IOException eIO) {
-      throw new HiveMetaException(
-          "Upgrade FAILED! Metastore state would be inconsistent !!", eIO);
-    }
-
-    // Revalidated the new version after upgrade
-    verifySchemaVersion();
-  }
-
-  /**
-   * Initialize the metastore schema to current version
-   *
-   * @throws MetaException
-   */
-  public void doInit() throws HiveMetaException {
-    doInit(metaStoreSchemaInfo.getHiveSchemaVersion());
-
-    // Revalidated the new version after upgrade
-    verifySchemaVersion();
-  }
-
-  /**
-   * Initialize the metastore schema
-   *
-   * @param toVersion
-   *          If null then current hive version is used
-   * @throws MetaException
-   */
-  public void doInit(String toVersion) throws HiveMetaException {
-    testConnectionToMetastore();
-    System.out.println("Starting metastore schema initialization to " + toVersion);
-
-    String initScriptDir = metaStoreSchemaInfo.getMetaStoreScriptDir();
-    String initScriptFile = metaStoreSchemaInfo.generateInitFileName(toVersion);
-
-    try {
-      System.out.println("Initialization script " + initScriptFile);
-      if (!dryRun) {
-        runBeeLine(initScriptDir, initScriptFile);
-        System.out.println("Initialization script completed");
-      }
-    } catch (IOException e) {
-      throw new HiveMetaException("Schema initialization FAILED!" +
-          " Metastore state would be inconsistent !!", e);
-    }
-  }
-
-  public void doValidate() throws HiveMetaException {
-    System.out.println("Starting metastore validation\n");
-    Connection conn = getConnectionToMetastore(false);
-    boolean success = true;
-    try {
-      if (validateSchemaVersions()) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        success = false;
-        System.out.println("[FAIL]\n");
-      }
-      if (validateSequences(conn)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        success = false;
-        System.out.println("[FAIL]\n");
-      }
-      if (validateSchemaTables(conn)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        success = false;
-        System.out.println("[FAIL]\n");
-      }
-      if (validateLocations(conn, this.validationServers)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        System.out.println("[WARN]\n");
-      }
-      if (validateColumnNullValues(conn)) {
-        System.out.println("[SUCCESS]\n");
-      } else {
-        System.out.println("[WARN]\n");
-      }
-    } finally {
-      if (conn != null) {
-        try {
-          conn.close();
-        } catch (SQLException e) {
-          throw new HiveMetaException("Failed to close metastore connection", e);
-        }
-      }
-    }
-
-    System.out.print("Done with metastore validation: ");
-    if (!success) {
-      System.out.println("[FAIL]");
-      System.exit(1);
-    } else {
-      System.out.println("[SUCCESS]");
-    }
-  }
-
-  boolean validateSequences(Connection conn) throws HiveMetaException {
-    Map<String, Pair<String, String>> seqNameToTable =
-        new ImmutableMap.Builder<String, Pair<String, String>>()
-        .put("MDatabase", Pair.of("DBS", "DB_ID"))
-        .put("MRole", Pair.of("ROLES", "ROLE_ID"))
-        .put("MGlobalPrivilege", Pair.of("GLOBAL_PRIVS", "USER_GRANT_ID"))
-        .put("MTable", Pair.of("TBLS","TBL_ID"))
-        .put("MStorageDescriptor", Pair.of("SDS", "SD_ID"))
-        .put("MSerDeInfo", Pair.of("SERDES", "SERDE_ID"))
-        .put("MColumnDescriptor", Pair.of("CDS", "CD_ID"))
-        .put("MTablePrivilege", Pair.of("TBL_PRIVS", "TBL_GRANT_ID"))
-        .put("MTableColumnStatistics", Pair.of("TAB_COL_STATS", "CS_ID"))
-        .put("MPartition", Pair.of("PARTITIONS", "PART_ID"))
-        .put("MPartitionColumnStatistics", Pair.of("PART_COL_STATS", "CS_ID"))
-        .put("MFunction", Pair.of("FUNCS", "FUNC_ID"))
-        .put("MIndex", Pair.of("IDXS", "INDEX_ID"))
-        .put("MStringList", Pair.of("SKEWED_STRING_LIST", "STRING_LIST_ID"))
-        .build();
-
-    System.out.println("Validating sequence number for SEQUENCE_TABLE");
-
-    boolean isValid = true;
-    try {
-      Statement stmt = conn.createStatement();
-      for (String seqName : seqNameToTable.keySet()) {
-        String tableName = seqNameToTable.get(seqName).getLeft();
-        String tableKey = seqNameToTable.get(seqName).getRight();
-        String fullSequenceName = "org.apache.hadoop.hive.metastore.model." + seqName;
-        String seqQuery = needsQuotedIdentifier ?
-            ("select t.\"NEXT_VAL\" from \"SEQUENCE_TABLE\" t WHERE t.\"SEQUENCE_NAME\"=? order by t.\"SEQUENCE_NAME\" ")
-            : ("select t.NEXT_VAL from SEQUENCE_TABLE t WHERE t.SEQUENCE_NAME=? order by t.SEQUENCE_NAME ");
-        String maxIdQuery = needsQuotedIdentifier ?
-            ("select max(\"" + tableKey + "\") from \"" + tableName + "\"")
-            : ("select max(" + tableKey + ") from " + tableName);
-
-        ResultSet res = stmt.executeQuery(maxIdQuery);
-        if (res.next()) {
-          long maxId = res.getLong(1);
-          if (maxId > 0) {
-            PreparedStatement pStmt = conn.prepareStatement(seqQuery);
-            pStmt.setString(1, fullSequenceName);
-            ResultSet resSeq = pStmt.executeQuery();
-            if (!resSeq.next()) {
-              isValid = false;
-              System.err.println("Missing SEQUENCE_NAME " + seqName + " from SEQUENCE_TABLE");
-            } else if (resSeq.getLong(1) < maxId) {
-              isValid = false;
-              System.err.println("NEXT_VAL for " + seqName + " in SEQUENCE_TABLE < max(" +
-                  tableKey + ") in " + tableName);
-            }
-          }
-        }
-      }
-
-      System.out.println((isValid ? "Succeeded" :"Failed") + " in sequence number validation for SEQUENCE_TABLE.");
-      return isValid;
-    } catch(SQLException e) {
-        throw new HiveMetaException("Failed to validate sequence number for SEQUENCE_TABLE", e);
-    }
-  }
-
-  boolean validateSchemaVersions() throws HiveMetaException {
-    System.out.println("Validating schema version");
-    try {
-      String newSchemaVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-      assertCompatibleVersion(metaStoreSchemaInfo.getHiveSchemaVersion(), newSchemaVersion);
-    } catch (HiveMetaException hme) {
-      if (hme.getMessage().contains("Metastore schema version is not compatible")
-        || hme.getMessage().contains("Multiple versions were found in metastore")
-        || hme.getMessage().contains("Could not find version info in metastore VERSION table")) {
-        System.err.println(hme.getMessage());
-        System.out.println("Failed in schema version validation.");
-        return false;
-      } else {
-        throw hme;
-      }
-    }
-    System.out.println("Succeeded in schema version validation.");
-    return true;
-  }
-
-  boolean validateSchemaTables(Connection conn) throws HiveMetaException {
-    String version            = null;
-    ResultSet rs              = null;
-    DatabaseMetaData metadata = null;
-    List<String> dbTables     = new ArrayList<String>();
-    List<String> schemaTables = new ArrayList<String>();
-    List<String> subScripts   = new ArrayList<String>();
-    Connection hmsConn        = getConnectionToMetastore(false);
-
-    System.out.println("Validating metastore schema tables");
-    try {
-      version = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
-    } catch (HiveMetaException he) {
-      System.err.println("Failed to determine schema version from Hive Metastore DB. " + he.getMessage());
-      System.out.println("Failed in schema table validation.");
-      LOG.debug("Failed to determine schema version from Hive Metastore DB," + he.getMessage());
-      return false;
-    }
-
-    // re-open the hms connection
-    hmsConn = getConnectionToMetastore(false);
-
-    LOG.debug("Validating tables in the schema for version " + version);
-    try {
-      String schema = null;
-      try {
-        schema = hmsConn.getSchema();
-      } catch (SQLFeatureNotSupportedException e) {
-        LOG.debug("schema is not supported");
-      }
-      
-      metadata       = conn.getMetaData();
-      String[] types = {"TABLE"};
-      rs             = metadata.getTables(null, schema, "%", types);
-      String table   = null;
-
-      while (rs.next()) {
-        table = rs.getString("TABLE_NAME");
-        dbTables.add(table.toLowerCase());
-        LOG.debug("Found table " + table + " in HMS dbstore");
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to retrieve schema tables from Hive Metastore DB", e);
-    } finally {
-      if (rs != null) {
-        try {
-          rs.close();
-        } catch (SQLException e) {
-          throw new HiveMetaException("Failed to close resultset", e);
-        }
-      }
-    }
-
-    // parse the schema file to determine the tables that are expected to exist
-    // we are using oracle schema because it is simpler to parse, no quotes or backticks etc
-    String baseDir    = new File(metaStoreSchemaInfo.getMetaStoreScriptDir()).getParent();
-    String schemaFile = new File(metaStoreSchemaInfo.getMetaStoreScriptDir(),
-        metaStoreSchemaInfo.generateInitFileName(version)).getPath();
-    try {
-      LOG.debug("Parsing schema script " + schemaFile);
-      subScripts.addAll(findCreateTable(schemaFile, schemaTables));
-      while (subScripts.size() > 0) {
-        schemaFile = baseDir + "/" + dbType + "/" + subScripts.remove(0);
-        LOG.debug("Parsing subscript " + schemaFile);
-        subScripts.addAll(findCreateTable(schemaFile, schemaTables));
-      }
-    } catch (Exception e) {
-      System.err.println("Exception in parsing schema file. Cause:" + e.getMessage());
-      System.out.println("Failed in schema table validation.");
-      return false;
-    }
-
-    LOG.debug("Schema tables:[ " + Arrays.toString(schemaTables.toArray()) + " ]");
-    LOG.debug("DB tables:[ " + Arrays.toString(dbTables.toArray()) + " ]");
-    // now diff the lists
-    schemaTables.removeAll(dbTables);
-    if (schemaTables.size() > 0) {
-      Collections.sort(schemaTables);
-      System.err.println("Table(s) [ " + Arrays.toString(schemaTables.toArray())
-          + " ] are missing from the metastore database schema.");
-      System.out.println("Failed in schema table validation.");
-      return false;
-    } else {
-      System.out.println("Succeeded in schema table validation.");
-      return true;
-    }
-  }
-
-  private List<String> findCreateTable(String path, List<String> tableList)
-      throws Exception {
-    NestedScriptParser sp           = HiveSchemaHelper.getDbCommandParser(dbType, false);
-    Matcher matcher                 = null;
-    Pattern regexp                  = null;
-    List<String> subs               = new ArrayList<String>();
-    int groupNo                     = 2;
-
-    regexp = Pattern.compile("CREATE TABLE(\\s+IF NOT EXISTS)?\\s+(\\S+).*");
-
-    if (!(new File(path)).exists()) {
-      throw new Exception(path + " does not exist. Potentially incorrect version in the metastore VERSION table");
-    }
-
-    try (
-      BufferedReader reader = new BufferedReader(new FileReader(path));
-    ){
-      String line = null;
-      while ((line = reader.readLine()) != null) {
-        if (sp.isNestedScript(line)) {
-          String subScript = null;
-          subScript = sp.getScriptName(line);
-          LOG.debug("Schema subscript " + subScript + " found");
-          subs.add(subScript);
-          continue;
-        }
-        line    = line.replaceAll("( )+", " "); //suppress multi-spaces
-        line    = line.replaceAll("\\(", " ");
-        line    = line.replaceAll("IF NOT EXISTS ", "");
-        line    = line.replaceAll("`","");
-        line    = line.replaceAll("'","");
-        line    = line.replaceAll("\"","");
-        matcher = regexp.matcher(line);
-
-        if (matcher.find()) {
-          String table = matcher.group(groupNo);
-          if (dbType.equals("derby"))
-            table  = table.replaceAll("APP\\.","");
-          tableList.add(table.toLowerCase());
-          LOG.debug("Found table " + table + " in the schema");
-        }
-      }
-    } catch (IOException ex){
-      throw new Exception(ex.getMessage());
-    }
-
-    return subs;
-  }
-
-  boolean validateColumnNullValues(Connection conn) throws HiveMetaException {
-    System.out.println("Validating columns for incorrect NULL values.");
-    boolean isValid = true;
-    try {
-      Statement stmt = conn.createStatement();
-      String tblQuery = needsQuotedIdentifier ?
-          ("select t.* from \"TBLS\" t WHERE t.\"SD_ID\" IS NULL and (t.\"TBL_TYPE\"='" + TableType.EXTERNAL_TABLE + "' or t.\"TBL_TYPE\"='" + TableType.MANAGED_TABLE + "') order by t.\"TBL_ID\" ")
-          : ("select t.* from TBLS t WHERE t.SD_ID IS NULL and (t.TBL_TYPE='" + TableType.EXTERNAL_TABLE + "' or t.TBL_TYPE='" + TableType.MANAGED_TABLE + "') order by t.TBL_ID ");
-
-      ResultSet res = stmt.executeQuery(tblQuery);
-      while (res.next()) {
-         long tableId = res.getLong("TBL_ID");
-         String tableName = res.getString("TBL_NAME");
-         String tableType = res.getString("TBL_TYPE");
-         isValid = false;
-         System.err.println("SD_ID in TBLS should not be NULL for Table Name=" + tableName + ", Table ID=" + tableId + ", Table Type=" + tableType);
-      }
-
-      System.out.println((isValid ? "Succeeded" : "Failed") + " in column validation for incorrect NULL values.");
-      return isValid;
-    } catch(SQLException e) {
-        throw new HiveMetaException("Failed to validate columns for incorrect NULL values", e);
-    }
-  }
-
-  @VisibleForTesting
-  void createCatalog(String catName, String location, String description, boolean ifNotExists)
-      throws HiveMetaException {
-    catName = normalizeIdentifier(catName);
-    System.out.println("Create catalog " + catName + " at location " + location);
-
-    Connection conn = getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        // If they set ifNotExists check for existence first, and bail if it exists.  This is
-        // more reliable then attempting to parse the error message from the SQLException.
-        if (ifNotExists) {
-          String query = "select " + quoteIf("NAME") + " from " + quoteIf("CTLGS") +
-              " where " + quoteIf("NAME") + " = '" + catName + "'";
-          LOG.debug("Going to run " + query);
-          ResultSet rs = stmt.executeQuery(query);
-          if (rs.next()) {
-            System.out.println("Catalog " + catName + " already exists");
-            return;
-          }
-        }
-        String query = "select max(" + quoteIf("CTLG_ID") + ") from " + quoteIf("CTLGS");
-        LOG.debug("Going to run " + query);
-        ResultSet rs = stmt.executeQuery(query);
-        if (!rs.next()) {
-          throw new HiveMetaException("No catalogs found, have you upgraded the database?");
-        }
-        int catNum = rs.getInt(1) + 1;
-        // We need to stay out of the way of any sequences used by the underlying database.
-        // Otherwise the next time the client tries to add a catalog we'll get an error.
-        // There should never be billions of catalogs, so we'll shift our sequence number up
-        // there to avoid clashes.
-        int floor = 1 << 30;
-        if (catNum < floor) catNum = floor;
-
-        String update = "insert into " + quoteIf("CTLGS") +
-            "(" + quoteIf("CTLG_ID") + ", " + quoteIf("NAME") + ", " + quoteAlways("DESC") + ", " + quoteIf( "LOCATION_URI") + ") " +
-            " values (" + catNum + ", '" + catName + "', '" + description + "', '" + location + "')";
-        LOG.debug("Going to run " + update);
-        stmt.execute(update);
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to add catalog", e);
-    } finally {
-      try {
-        if (!success) conn.rollback();
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.", e);
-      }
-    }
-  }
-
-  @VisibleForTesting
-  void alterCatalog(String catName, String location, String description) throws HiveMetaException {
-    if (location == null && description == null) {
-      throw new HiveMetaException("Asked to update catalog " + catName +
-          " but not given any changes to update");
-    }
-    catName = normalizeIdentifier(catName);
-    System.out.println("Updating catalog " + catName);
-
-    Connection conn = getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        StringBuilder update = new StringBuilder("update ")
-            .append(quoteIf("CTLGS"))
-            .append(" set ");
-        if (location != null) {
-          update.append(quoteIf("LOCATION_URI"))
-              .append(" = '")
-              .append(location)
-              .append("' ");
-        }
-        if (description != null) {
-          if (location != null) update.append(", ");
-          update.append(quoteAlways("DESC"))
-              .append(" = '")
-              .append(description)
-              .append("'");
-        }
-        update.append(" where ")
-            .append(quoteIf("NAME"))
-            .append(" = '")
-            .append(catName)
-            .append("'");
-        LOG.debug("Going to run " + update.toString());
-        int count = stmt.executeUpdate(update.toString());
-        if (count != 1) {
-          throw new HiveMetaException("Failed to find catalog " + catName + " to update");
-        }
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to update catalog", e);
-    } finally {
-      try {
-        if (!success) conn.rollback();
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.", e);
-      }
-    }
-  }
-
-  @VisibleForTesting
-  void moveDatabase(String fromCatName, String toCatName, String dbName) throws HiveMetaException {
-    fromCatName = normalizeIdentifier(fromCatName);
-    toCatName = normalizeIdentifier(toCatName);
-    dbName = normalizeIdentifier(dbName);
-    System.out.println("Moving database " + dbName + " from catalog " + fromCatName +
-        " to catalog " + toCatName);
-    Connection conn = getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        updateCatalogNameInTable(stmt, "DBS", "CTLG_NAME", "NAME", fromCatName, toCatName, dbName, false);
-        updateCatalogNameInTable(stmt, "TAB_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        updateCatalogNameInTable(stmt, "PART_COL_STATS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        updateCatalogNameInTable(stmt, "PARTITION_EVENTS", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        updateCatalogNameInTable(stmt, "NOTIFICATION_LOG", "CAT_NAME", "DB_NAME", fromCatName, toCatName, dbName, true);
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException e) {
-      throw new HiveMetaException("Failed to move database", e);
-    } finally {
-      try {
-        if (!success) conn.rollback();
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.");
-      }
-    }
-  }
-
-  private void updateCatalogNameInTable(Statement stmt, String tableName, String catColName,
-                                        String dbColName, String fromCatName,
-                                        String toCatName, String dbName, boolean zeroUpdatesOk)
-      throws HiveMetaException, SQLException {
-    String update = "update " + quoteIf(tableName) + " " +
-        "set " + quoteIf(catColName) + " = '" + toCatName + "' " +
-        "where " + quoteIf(catColName) + " = '" + fromCatName + "' and " + quoteIf(dbColName) + " = '" + dbName + "'";
-    LOG.debug("Going to run " + update);
-    int numUpdated = stmt.executeUpdate(update);
-    if (numUpdated != 1 && !(zeroUpdatesOk && numUpdated == 0)) {
-      throw new HiveMetaException("Failed to properly update the " + tableName +
-          " table.  Expected to update 1 row but instead updated " + numUpdated);
-    }
-  }
-
-  @VisibleForTesting
-  void moveTable(String fromCat, String toCat, String fromDb, String toDb, String tableName)
-      throws HiveMetaException {
-    fromCat = normalizeIdentifier(fromCat);
-    toCat = normalizeIdentifier(toCat);
-    fromDb = normalizeIdentifier(fromDb);
-    toDb = normalizeIdentifier(toDb);
-    tableName = normalizeIdentifier(tableName);
-    Connection conn = getConnectionToMetastore(true);
-    boolean success = false;
-    try {
-      conn.setAutoCommit(false);
-      try (Statement stmt = conn.createStatement()) {
-        // Find the old database id
-        String query = "select " + quoteIf("DB_ID") +
-            " from " + quoteIf("DBS") +
-            " where " + quoteIf("NAME") + " = '" + fromDb + "' "
-                + "and " + quoteIf("CTLG_NAME") + " = '" + fromCat + "'";
-        LOG.debug("Going to run " + query);
-        ResultSet rs = stmt.executeQuery(query);
-        if (!rs.next()) {
-          throw new HiveMetaException("Unable to find database " + fromDb);
-        }
-        long oldDbId = rs.getLong(1);
-
-        // Find the new database id
-        query = "select " + quoteIf("DB_ID") +
-            " from " + quoteIf("DBS") +
-            " where " + quoteIf("NAME") + " = '" + toDb + "' "
-                + "and " + quoteIf("CTLG_NAME") + " = '" + toCat + "'";
-        LOG.debug("Going to run " + query);
-        rs = stmt.executeQuery(query);
-        if (!rs.next()) {
-          throw new HiveMetaException("Unable to find database " + toDb);
-        }
-        long newDbId = rs.getLong(1);
-
-        String update = "update " + quoteIf("TBLS") + " " +
-            "set " + quoteIf("DB_ID") + " = " + newDbId + " " +
-            "where " + quoteIf("DB_ID") + " = " + oldDbId +
-                " and " + quoteIf("TBL_NAME") + " = '" + tableName + "'";
-        LOG.debug("Going to run " + update);
-        int numUpdated = stmt.executeUpdate(update);
-        if (numUpdated != 1) {
-          throw new HiveMetaException(
-              "Failed to properly update TBLS table.  Expected to update " +
-                  "1 row but instead updated " + numUpdated);
-        }
-        updateDbNameForTable(stmt, "TAB_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        updateDbNameForTable(stmt, "PART_COL_STATS", "TABLE_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        updateDbNameForTable(stmt, "PARTITION_EVENTS", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        updateDbNameForTable(stmt, "NOTIFICATION_LOG", "TBL_NAME", fromCat, toCat, fromDb, toDb, tableName);
-        conn.commit();
-        success = true;
-      }
-    } catch (SQLException se) {
-      throw new HiveMetaException("Failed to move table", se);
-    } finally {
-      try {
-        if (!success) conn.rollback();
-      } catch (SQLException e) {
-        // Not really much we can do here.
-        LOG.error("Failed to rollback, everything will probably go bad from here.");
-      }
-
-    }
-  }
-
-  private void updateDbNameForTable(Statement stmt, String tableName,
-                                    String tableColumnName, String fromCat, String toCat,
-                                    String fromDb, String toDb, String hiveTblName)
-      throws HiveMetaException, SQLException {
-    String update = "update " + quoteIf(tableName) + " " +
-            "set " + quoteIf("CAT_NAME") + " = '" + toCat + "', " + quoteIf("DB_NAME") + " = '" + toDb + "' " +
-            "where " + quoteIf("CAT_NAME") + " = '" + fromCat + "' " +
-                "and " + quoteIf("DB_NAME") + " = '" + fromDb + "' " +
-                "and " + quoteIf(tableColumnName) + " = '" + hiveTblName + "'";
-    LOG.debug("Going to run " + update);
-    int numUpdated = stmt.executeUpdate(update);
-    if (numUpdated > 1 || numUpdated < 0) {
-      throw new HiveMetaException("Failed to properly update the " + tableName +
-          " table.  Expected to update 1 row but instead updated " + numUpdated);
-    }
-  }
-
-  // Quote if the database requires it
-  private String quoteIf(String identifier) {
-    return needsQuotedIdentifier ? quoteCharacter + identifier + quoteCharacter : identifier;
-  }
-
-  // Quote always, for fields that mimic SQL keywords, like DESC
-  private String quoteAlways(String identifier) {
-    return quoteCharacter + identifier + quoteCharacter;
-  }
-
-  /**
-   *  Run pre-upgrade scripts corresponding to a given upgrade script,
-   *  if any exist. The errors from pre-upgrade are ignored.
-   *  Pre-upgrade scripts typically contain setup statements which
-   *  may fail on some database versions and failure is ignorable.
-   *
-   *  @param scriptDir upgrade script directory name
-   *  @param scriptFile upgrade script file name
-   */
-  private void runPreUpgrade(String scriptDir, String scriptFile) {
-    for (int i = 0;; i++) {
-      String preUpgradeScript =
-          metaStoreSchemaInfo.getPreUpgradeScriptName(i, scriptFile);
-      File preUpgradeScriptFile = new File(scriptDir, preUpgradeScript);
-      if (!preUpgradeScriptFile.isFile()) {
-        break;
-      }
-
-      try {
-        runBeeLine(scriptDir, preUpgradeScript);
-        System.out.println("Completed " + preUpgradeScript);
-      } catch (Exception e) {
-        // Ignore the pre-upgrade script errors
-        System.err.println("Warning in pre-upgrade script " + preUpgradeScript + ": "
-            + e.getMessage());
-        if (verbose) {
-          e.printStackTrace();
-        }
-      }
-    }
-  }
-
-  /***
-   * Run beeline with the given metastore script. Flatten the nested scripts
-   * into single file.
-   */
-  private void runBeeLine(String scriptDir, String scriptFile)
-      throws IOException, HiveMetaException {
-    NestedScriptParser dbCommandParser = getDbCommandParser(dbType, metaDbType);
-
-    // expand the nested script
-    // If the metaDbType is set, this is setting up the information
-    // schema in Hive. That specifically means that the sql commands need
-    // to be adjusted for the underlying RDBMS (correct quotation
-    // strings, etc).
-    String sqlCommands = dbCommandParser.buildCommand(scriptDir, scriptFile, metaDbType != null);
-    File tmpFile = File.createTempFile("schematool", ".sql");
-    tmpFile.deleteOnExit();
-
-    // write out the buffer into a file. Add beeline commands for autocommit and close
-    FileWriter fstream = new FileWriter(tmpFile.getPath());
-    BufferedWriter out = new BufferedWriter(fstream);
-    out.write("!autocommit on" + System.getProperty("line.separator"));
-    out.write(sqlCommands);
-    out.write("!closeall" + System.getProperty("line.separator"));
-    out.close();
-    runBeeLine(tmpFile.getPath());
-  }
-
-  // Generate the beeline args per hive conf and execute the given script
-  public void runBeeLine(String sqlScriptFile) throws IOException {
-    CommandBuilder builder = new CommandBuilder(hiveConf, url, driver,
-        userName, passWord, sqlScriptFile);
-
-    // run the script using Beeline
-    try (BeeLine beeLine = new BeeLine()) {
-      if (!verbose) {
-        beeLine.setOutputStream(new PrintStream(new NullOutputStream()));
-        beeLine.getOpts().setSilent(true);
-      }
-      beeLine.getOpts().setAllowMultiLineCommand(false);
-      beeLine.getOpts().setIsolation("TRANSACTION_READ_COMMITTED");
-      // We can be pretty sure that an entire line can be processed as a single command since
-      // we always add a line separator at the end while calling dbCommandParser.buildCommand.
-      beeLine.getOpts().setEntireLineAsCommand(true);
-      LOG.debug("Going to run command <" + builder.buildToLog() + ">");
-      int status = beeLine.begin(builder.buildToRun(), null);
-      if (status != 0) {
-        throw new IOException("Schema script failed, errorcode " + status);
-      }
-    }
-  }
-
-  static class CommandBuilder {
-    private final HiveConf hiveConf;
-    private final String userName;
-    private final String password;
-    private final String sqlScriptFile;
-    private final String driver;
-    private final String url;
-
-    CommandBuilder(HiveConf hiveConf, String url, String driver,
-        String userName, String password, String sqlScriptFile) {
-      this.hiveConf = hiveConf;
-      this.userName = userName;
-      this.password = password;
-      this.url = url;
-      this.driver = driver;
-      this.sqlScriptFile = sqlScriptFile;
-    }
-
-    String[] buildToRun() throws IOException {
-      return argsWith(password);
-    }
-
-    String buildToLog() throws IOException {
-      logScript();
-      return StringUtils.join(argsWith(BeeLine.PASSWD_MASK), " ");
-    }
-
-    private String[] argsWith(String password) throws IOException {
-      return new String[]
-        {
-          "-u", url == null ? HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECT_URL_KEY, hiveConf) : url,
-          "-d", driver == null ? HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECTION_DRIVER, hiveConf) : driver,
-          "-n", userName,
-          "-p", password,
-          "-f", sqlScriptFile
-        };
-    }
-
-    private void logScript() throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Going to invoke file that contains:");
-        try (BufferedReader reader = new BufferedReader(new FileReader(sqlScriptFile))) {
-          String line;
-          while ((line = reader.readLine()) != null) {
-            LOG.debug("script: " + line);
-          }
-        }
-      }
-    }
-  }
-
-  // Create the required command line options
-  @SuppressWarnings("static-access")
-  private static void initOptions(Options cmdLineOptions) {
-    Option help = new Option("help", "print this message");
-    Option upgradeOpt = new Option("upgradeSchema", "Schema upgrade");
-    Option upgradeFromOpt = OptionBuilder.withArgName("upgradeFrom").hasArg().
-                withDescription("Schema upgrade from a version").
-                create("upgradeSchemaFrom");
-    Option initOpt = new Option("initSchema", "Schema initialization");
-    Option initToOpt = OptionBuilder.withArgName("initTo").hasArg().
-                withDescription("Schema initialization to a version").
-                create("initSchemaTo");
-    Option infoOpt = new Option("info", "Show config and schema details");
-    Option validateOpt = new Option("validate", "Validate the database");
-    Option createCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Create a catalog, requires --catalogLocation parameter as well")
-        .create("createCatalog");
-    Option alterCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Alter a catalog, requires --catalogLocation and/or --catalogDescription parameter as well")
-        .create("alterCatalog");
-    Option moveDatabase = OptionBuilder
-        .hasArg()
-        .withDescription("Move a database between catalogs.  Argument is the database name. " +
-            "Requires --fromCatalog and --toCatalog parameters as well")
-        .create("moveDatabase");
-    Option moveTable = OptionBuilder
-        .hasArg()
-        .withDescription("Move a table to a different database.  Argument is the table name. " +
-            "Requires --fromCatalog, --toCatalog, --fromDatabase, and --toDatabase " +
-            " parameters as well.")
-        .create("moveTable");
-
-    OptionGroup optGroup = new OptionGroup();
-    optGroup.addOption(upgradeOpt)
-        .addOption(initOpt)
-        .addOption(help)
-        .addOption(upgradeFromOpt)
-        .addOption(initToOpt)
-        .addOption(infoOpt)
-        .addOption(validateOpt)
-        .addOption(createCatalog)
-        .addOption(alterCatalog)
-        .addOption(moveDatabase)
-        .addOption(moveTable);
-    optGroup.setRequired(true);
-
-    Option userNameOpt = OptionBuilder.withArgName("user")
-                .hasArgs()
-                .withDescription("Override config file user name")
-                .create("userName");
-    Option passwdOpt = OptionBuilder.withArgName("password")
-                .hasArgs()
-                 .withDescription("Override config file password")
-                 .create("passWord");
-    Option dbTypeOpt = OptionBuilder.withArgName("databaseType")
-                .hasArgs().withDescription("Metastore database type")
-                .create("dbType");
-    Option metaDbTypeOpt = OptionBuilder.withArgName("metaDatabaseType")
-                .hasArgs().withDescription("Used only if upgrading the system catalog for hive")
-                .create("metaDbType");
-    Option urlOpt = OptionBuilder.withArgName("url")
-                .hasArgs().withDescription("connection url to the database")
-                .create("url");
-    Option driverOpt = OptionBuilder.withArgName("driver")
-                .hasArgs().withDescription("driver name for connection")
-                .create("driver");
-    Option dbOpts = OptionBuilder.withArgName("databaseOpts")
-                .hasArgs().withDescription("Backend DB specific options")
-                .create("dbOpts");
-    Option dryRunOpt = new Option("dryRun", "list SQL scripts (no execute)");
-    Option verboseOpt = new Option("verbose", "only print SQL statements");
-    Option serversOpt = OptionBuilder.withArgName("serverList")
-        .hasArgs().withDescription("a comma-separated list of servers used in location validation in the format of scheme://authority (e.g. hdfs://localhost:8000)")
-        .create("servers");
-    Option catalogLocation = OptionBuilder
-        .hasArg()
-        .withDescription("Location of new catalog, required when adding a catalog")
-        .create("catalogLocation");
-    Option catalogDescription = OptionBuilder
-        .hasArg()
-        .withDescription("Description of new catalog")
-        .create("catalogDescription");
-    Option ifNotExists = OptionBuilder
-        .withDescription("If passed then it is not an error to create an existing catalog")
-        .create("ifNotExists");
-    Option toCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Catalog a moving database or table is going to.  This is " +
-            "required if you are moving a database or table.")
-        .create("toCatalog");
-    Option fromCatalog = OptionBuilder
-        .hasArg()
-        .withDescription("Catalog a moving database or table is coming from.  This is " +
-            "required if you are moving a database or table.")
-        .create("fromCatalog");
-    Option toDatabase = OptionBuilder
-        .hasArg()
-        .withDescription("Database a moving table is going to.  This is " +
-            "required if you are moving a table.")
-        .create("toDatabase");
-    Option fromDatabase = OptionBuilder
-        .hasArg()
-        .withDescription("Database a moving table is coming from.  This is " +
-            "required if you are moving a table.")
-        .create("fromDatabase");
-    cmdLineOptions.addOption(help);
-    cmdLineOptions.addOption(dryRunOpt);
-    cmdLineOptions.addOption(userNameOpt);
-    cmdLineOptions.addOption(passwdOpt);
-    cmdLineOptions.addOption(dbTypeOpt);
-    cmdLineOptions.addOption(verboseOpt);
-    cmdLineOptions.addOption(metaDbTypeOpt);
-    cmdLineOptions.addOption(urlOpt);
-    cmdLineOptions.addOption(driverOpt);
-    cmdLineOptions.addOption(dbOpts);
-    cmdLineOptions.addOption(serversOpt);
-    cmdLineOptions.addOption(catalogLocation);
-    cmdLineOptions.addOption(catalogDescription);
-    cmdLineOptions.addOption(ifNotExists);
-    cmdLineOptions.addOption(toCatalog);
-    cmdLineOptions.addOption(fromCatalog);
-    cmdLineOptions.addOption(toDatabase);
-    cmdLineOptions.addOption(fromDatabase);
-    cmdLineOptions.addOptionGroup(optGroup);
-  }
-
-  public static void main(String[] args) {
-    CommandLineParser parser = new GnuParser();
-    CommandLine line = null;
-    String dbType = null;
-    String metaDbType = null;
-    String schemaVer = null;
-    Options cmdLineOptions = new Options();
-
-    // Argument handling
-    initOptions(cmdLineOptions);
-    try {
-      line = parser.parse(cmdLineOptions, args);
-    } catch (ParseException e) {
-      System.err.println("HiveSchemaTool:Parsing failed.  Reason: " + e.getLocalizedMessage());
-      printAndExit(cmdLineOptions);
-    }
-
-    if (line.hasOption("help")) {
-      HelpFormatter formatter = new HelpFormatter();
-      formatter.printHelp("schemaTool", cmdLineOptions);
-      return;
-    }
-
-    if (line.hasOption("dbType")) {
-      dbType = line.getOptionValue("dbType");
-      if ((!dbType.equalsIgnoreCase(HiveSchemaHelper.DB_DERBY) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MSSQL) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MYSQL) &&
-          !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_POSTGRACE) && !dbType
-          .equalsIgnoreCase(HiveSchemaHelper.DB_ORACLE))) {
-        System.err.println("Unsupported dbType " + dbType);
-        printAndExit(cmdLineOptions);
-      }
-    } else {
-      System.err.println("no dbType supplied");
-      printAndExit(cmdLineOptions);
-    }
-
-    if (line.hasOption("metaDbType")) {
-      metaDbType = line.getOptionValue("metaDbType");
-
-      if (!dbType.equals(HiveSchemaHelper.DB_HIVE)) {
-        System.err.println("metaDbType only supported for dbType = hive");
-        printAndExit(cmdLineOptions);
-      }
-
-      if (!metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_DERBY) &&
-          !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_MSSQL) &&
-          !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_MYSQL) &&
-          !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_POSTGRACE) &&
-          !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_ORACLE)) {
-        System.err.println("Unsupported metaDbType " + metaDbType);
-        printAndExit(cmdLineOptions);
-      }
-    } else if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
-      System.err.println("no metaDbType supplied");
-      printAndExit(cmdLineOptions);
-    }
-
-
-    System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.varname, "true");
-    try {
-      HiveSchemaTool schemaTool = new HiveSchemaTool(dbType, metaDbType);
-
-      if (line.hasOption("userName")) {
-        schemaTool.setUserName(line.getOptionValue("userName"));
-      } else {
-        schemaTool.setUserName(
-            schemaTool.getHiveConf().get(ConfVars.METASTORE_CONNECTION_USER_NAME.varname));
-      }
-      if (line.hasOption("passWord")) {
-        schemaTool.setPassWord(line.getOptionValue("passWord"));
-      } else {
-        try {
-          schemaTool.setPassWord(ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(),
-              HiveConf.ConfVars.METASTOREPWD.varname));
-        } catch (IOException err) {
-          throw new HiveMetaException("Error getting metastore password", err);
-        }
-      }
-      if (line.hasOption("url")) {
-        schemaTool.setUrl(line.getOptionValue("url"));
-      }
-      if (line.hasOption("driver")) {
-        schemaTool.setDriver(line.getOptionValue("driver"));
-      }
-      if (line.hasOption("dryRun")) {
-        schemaTool.setDryRun(true);
-      }
-      if (line.hasOption("verbose")) {
-        schemaTool.setVerbose(true);
-      }
-      if (line.hasOption("dbOpts")) {
-        schemaTool.setDbOpts(line.getOptionValue("dbOpts"));
-      }
-      if (line.hasOption("validate") && line.hasOption("servers")) {
-        schemaTool.setValidationServers(line.getOptionValue("servers"));
-      }
-      if (line.hasOption("info")) {
-        schemaTool.showInfo();
-      } else if (line.hasOption("upgradeSchema")) {
-        schemaTool.doUpgrade();
-      } else if (line.hasOption("upgradeSchemaFrom")) {
-        schemaVer = line.getOptionValue("upgradeSchemaFrom");
-        schemaTool.doUpgrade(schemaVer);
-      } else if (line.hasOption("initSchema")) {
-        schemaTool.doInit();
-      } else if (line.hasOption("initSchemaTo")) {
-        schemaVer = line.getOptionValue("initSchemaTo");
-        schemaTool.doInit(schemaVer);
-      } else if (line.hasOption("validate")) {
-        schemaTool.doValidate();
-      } else if (line.hasOption("createCatalog")) {
-        schemaTool.createCatalog(line.getOptionValue("createCatalog"),
-            line.getOptionValue("catalogLocation"), line.getOptionValue("catalogDescription"),
-            line.hasOption("ifNotExists"));
-      } else if (line.hasOption("alterCatalog")) {
-        schemaTool.alterCatalog(line.getOptionValue("alterCatalog"),
-            line.getOptionValue("catalogLocation"), line.getOptionValue("catalogDescription"));
-      } else if (line.hasOption("moveDatabase")) {
-        schemaTool.moveDatabase(line.getOptionValue("fromCatalog"),
-            line.getOptionValue("toCatalog"), line.getOptionValue("moveDatabase"));
-      } else if (line.hasOption("moveTable")) {
-        schemaTool.moveTable(line.getOptionValue("fromCatalog"), line.getOptionValue("toCatalog"),
-            line.getOptionValue("fromDatabase"), line.getOptionValue("toDatabase"),
-            line.getOptionValue("moveTable"));
-      } else {
-        System.err.println("no valid option supplied");
-        printAndExit(cmdLineOptions);
-      }
-    } catch (HiveMetaException e) {
-      System.err.println(e);
-      if (e.getCause() != null) {
-        Throwable t = e.getCause();
-        System.err.println("Underlying cause: "
-            + t.getClass().getName() + " : "
-            + t.getMessage());
-        if (e.getCause() instanceof SQLException) {
-          System.err.println("SQL Error code: " + ((SQLException)t).getErrorCode());
-        }
-      }
-      if (line.hasOption("verbose")) {
-        e.printStackTrace();
-      } else {
-        System.err.println("Use --verbose for detailed stacktrace.");
-      }
-      System.err.println("*** schemaTool failed ***");
-      System.exit(1);
-    }
-    System.out.println("schemaTool completed");
-    System.exit(0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
new file mode 100644
index 0000000..9c3f30b
--- /dev/null
+++ b/beeline/src/java/org/apache/hive/beeline/schematool/HiveSchemaTool.java
@@ -0,0 +1,415 @@
+/*
+ * 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.beeline.schematool;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.output.NullOutputStream;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaException;
+import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
+import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.MetaStoreConnectionInfo;
+import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hive.beeline.BeeLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URI;
+import java.sql.Connection;
+import java.sql.SQLException;
+
+public class HiveSchemaTool {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaTool.class.getName());
+
+  private final HiveConf hiveConf;
+  private final String dbType;
+  private final String metaDbType;
+  private final IMetaStoreSchemaInfo metaStoreSchemaInfo;
+  private final boolean needsQuotedIdentifier;
+  private String quoteCharacter;
+
+  private String url = null;
+  private String driver = null;
+  private String userName = null;
+  private String passWord = null;
+  private boolean dryRun = false;
+  private boolean verbose = false;
+  private String dbOpts = null;
+  private URI[] validationServers = null; // The list of servers the database/partition/table can locate on
+
+  private HiveSchemaTool(String dbType, String metaDbType) throws HiveMetaException {
+    this(System.getenv("HIVE_HOME"), new HiveConf(HiveSchemaTool.class), dbType, metaDbType);
+  }
+
+  @VisibleForTesting
+  public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType, String metaDbType)
+      throws HiveMetaException {
+    if (hiveHome == null || hiveHome.isEmpty()) {
+      throw new HiveMetaException("No Hive home directory provided");
+    }
+    this.hiveConf = hiveConf;
+    this.dbType = dbType;
+    this.metaDbType = metaDbType;
+    NestedScriptParser parser = getDbCommandParser(dbType, metaDbType);
+    this.needsQuotedIdentifier = parser.needsQuotedIdentifier();
+    this.quoteCharacter = parser.getQuoteCharacter();
+    this.metaStoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf, hiveHome, dbType);
+    // If the dbType is "hive", this is setting up the information schema in Hive.
+    // We will set the default jdbc url and driver.
+    // It is overriden by command line options if passed (-url and -driver
+    if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) {
+      url = HiveSchemaHelper.EMBEDDED_HS2_URL;
+      driver = HiveSchemaHelper.HIVE_JDBC_DRIVER;
+    }
+  }
+
+  HiveConf getHiveConf() {
+    return hiveConf;
+  }
+
+  String getDbType() {
+    return dbType;
+  }
+
+  IMetaStoreSchemaInfo getMetaStoreSchemaInfo() {
+    return metaStoreSchemaInfo;
+  }
+
+  private void setUrl(String url) {
+    this.url = url;
+  }
+
+  private void setDriver(String driver) {
+    this.driver = driver;
+  }
+
+  @VisibleForTesting
+  public void setUserName(String userName) {
+    this.userName = userName;
+  }
+
+  @VisibleForTesting
+  public void setPassWord(String passWord) {
+    this.passWord = passWord;
+  }
+
+  @VisibleForTesting
+  public void setDryRun(boolean dryRun) {
+    this.dryRun = dryRun;
+  }
+
+  boolean isDryRun() {
+    return dryRun;
+  }
+
+  @VisibleForTesting
+  public void setVerbose(boolean verbose) {
+    this.verbose = verbose;
+  }
+
+  boolean isVerbose() {
+    return verbose;
+  }
+
+  private void setDbOpts(String dbOpts) {
+    this.dbOpts = dbOpts;
+  }
+
+  private void setValidationServers(String servers) {
+    if(StringUtils.isNotEmpty(servers)) {
+      String[] strServers = servers.split(",");
+      this.validationServers = new URI[strServers.length];
+      for (int i = 0; i < validationServers.length; i++) {
+        validationServers[i] = new Path(strServers[i]).toUri();
+      }
+    }
+  }
+
+  URI[] getValidationServers() {
+    return validationServers;
+  }
+
+  Connection getConnectionToMetastore(boolean printInfo) throws HiveMetaException {
+    return HiveSchemaHelper.getConnectionToMetastore(userName, passWord, url, driver, printInfo, hiveConf,
+        null);
+  }
+
+  private NestedScriptParser getDbCommandParser(String dbType, String metaDbType) {
+    return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName, passWord, hiveConf,
+        metaDbType, false);
+  }
+
+  // test the connection metastore using the config property
+  void testConnectionToMetastore() throws HiveMetaException {
+    Connection conn = getConnectionToMetastore(true);
+    try {
+      conn.close();
+    } catch (SQLException e) {
+      throw new HiveMetaException("Failed to close metastore connection", e);
+    }
+  }
+
+  /**
+   * check if the current schema version in metastore matches the Hive version
+   * @throws MetaException
+   */
+  void verifySchemaVersion() throws HiveMetaException {
+    // don't check version if its a dry run
+    if (dryRun) {
+      return;
+    }
+    String newSchemaVersion = metaStoreSchemaInfo.getMetaStoreSchemaVersion(getConnectionInfo(false));
+    // verify that the new version is added to schema
+    assertCompatibleVersion(metaStoreSchemaInfo.getHiveSchemaVersion(), newSchemaVersion);
+  }
+
+  void assertCompatibleVersion(String hiveSchemaVersion, String dbSchemaVersion)
+      throws HiveMetaException {
+    if (!metaStoreSchemaInfo.isVersionCompatible(hiveSchemaVersion, dbSchemaVersion)) {
+      throw new HiveMetaException("Metastore schema version is not compatible. Hive Version: "
+          + hiveSchemaVersion + ", Database Schema Version: " + dbSchemaVersion);
+    }
+  }
+
+  MetaStoreConnectionInfo getConnectionInfo(boolean printInfo) {
+    return new MetaStoreConnectionInfo(userName, passWord, url, driver, printInfo, hiveConf,
+        dbType, metaDbType);
+  }
+
+  // Quote if the database requires it
+  String quote(String stmt) {
+    stmt = stmt.replace("<q>", needsQuotedIdentifier ? quoteCharacter : "");
+    stmt = stmt.replace("<qa>", quoteCharacter);
+    return stmt;
+  }
+
+  /***
+   * Run beeline with the given metastore script. Flatten the nested scripts
+   * into single file.
+   */
+  void runBeeLine(String scriptDir, String scriptFile)
+      throws IOException, HiveMetaException {
+    NestedScriptParser dbCommandParser = getDbCommandParser(dbType, metaDbType);
+
+    // expand the nested script
+    // If the metaDbType is set, this is setting up the information
+    // schema in Hive. That specifically means that the sql commands need
+    // to be adjusted for the underlying RDBMS (correct quotation
+    // strings, etc).
+    String sqlCommands = dbCommandParser.buildCommand(scriptDir, scriptFile, metaDbType != null);
+    File tmpFile = File.createTempFile("schematool", ".sql");
+    tmpFile.deleteOnExit();
+
+    // write out the buffer into a file. Add beeline commands for autocommit and close
+    FileWriter fstream = new FileWriter(tmpFile.getPath());
+    BufferedWriter out = new BufferedWriter(fstream);
+    out.write("!autocommit on" + System.getProperty("line.separator"));
+    out.write(sqlCommands);
+    out.write("!closeall" + System.getProperty("line.separator"));
+    out.close();
+    runBeeLine(tmpFile.getPath());
+  }
+
+  // Generate the beeline args per hive conf and execute the given script
+  void runBeeLine(String sqlScriptFile) throws IOException {
+    CommandBuilder builder = new CommandBuilder(hiveConf, url, driver,
+        userName, passWord, sqlScriptFile);
+
+    // run the script using Beeline
+    try (BeeLine beeLine = new BeeLine()) {
+      if (!verbose) {
+        beeLine.setOutputStream(new PrintStream(new NullOutputStream()));
+        beeLine.getOpts().setSilent(true);
+      }
+      beeLine.getOpts().setAllowMultiLineCommand(false);
+      beeLine.getOpts().setIsolation("TRANSACTION_READ_COMMITTED");
+      // We can be pretty sure that an entire line can be processed as a single command since
+      // we always add a line separator at the end while calling dbCommandParser.buildCommand.
+      beeLine.getOpts().setEntireLineAsCommand(true);
+      LOG.debug("Going to run command <" + builder.buildToLog() + ">");
+      int status = beeLine.begin(builder.buildToRun(), null);
+      if (status != 0) {
+        throw new IOException("Schema script failed, errorcode " + status);
+      }
+    }
+  }
+
+  static class CommandBuilder {
+    private final String userName;
+    private final String password;
+    private final String sqlScriptFile;
+    private final String driver;
+    private final String url;
+
+    CommandBuilder(HiveConf hiveConf, String url, String driver, String userName, String password,
+        String sqlScriptFile) throws IOException {
+      this.userName = userName;
+      this.password = password;
+      this.url = url == null ?
+          HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECT_URL_KEY, hiveConf) : url;
+      this.driver = driver == null ?
+          HiveSchemaHelper.getValidConfVar(MetastoreConf.ConfVars.CONNECTION_DRIVER, hiveConf) : driver;
+      this.sqlScriptFile = sqlScriptFile;
+    }
+
+    String[] buildToRun() {
+      return argsWith(password);
+    }
+
+    String buildToLog() throws IOException {
+      logScript();
+      return StringUtils.join(argsWith(BeeLine.PASSWD_MASK), " ");
+    }
+
+    private String[] argsWith(String password) {
+      return new String[]
+          {
+            "-u", url,
+            "-d", driver,
+            "-n", userName,
+            "-p", password,
+            "-f", sqlScriptFile
+          };
+    }
+
+    private void logScript() throws IOException {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Going to invoke file that contains:");
+        try (BufferedReader reader = new BufferedReader(new FileReader(sqlScriptFile))) {
+          String line;
+          while ((line = reader.readLine()) != null) {
+            LOG.debug("script: " + line);
+          }
+        }
+      }
+    }
+  }
+
+  public static void main(String[] args) {
+    HiveSchemaToolCommandLine line = null;
+    try {
+      line = new HiveSchemaToolCommandLine(args);
+    } catch (ParseException e) {
+      System.exit(1);
+    }
+
+    System.setProperty(MetastoreConf.ConfVars.SCHEMA_VERIFICATION.getVarname(), "true");
+    try {
+      HiveSchemaTool schemaTool = createSchemaTool(line);
+
+      HiveSchemaToolTask task = null;
+      if (line.hasOption("info")) {
+        task = new HiveSchemaToolTaskInfo();
+      } else if (line.hasOption("upgradeSchema") || line.hasOption("upgradeSchemaFrom")) {
+        task = new HiveSchemaToolTaskUpgrade();
+      } else if (line.hasOption("initSchema") || line.hasOption("initSchemaTo")) {
+        task = new HiveSchemaToolTaskInit();
+      } else if (line.hasOption("validate")) {
+        task = new HiveSchemaToolTaskValidate();
+      } else if (line.hasOption("createCatalog")) {
+        task = new HiveSchemaToolTaskCreateCatalog();
+      } else if (line.hasOption("alterCatalog")) {
+        task = new HiveSchemaToolTaskAlterCatalog();
+      } else if (line.hasOption("moveDatabase")) {
+        task = new HiveSchemaToolTaskMoveDatabase();
+      } else if (line.hasOption("moveTable")) {
+        task = new HiveSchemaToolTaskMoveTable();
+      } else {
+        throw new HiveMetaException("No task defined!");
+      }
+
+      task.setHiveSchemaTool(schemaTool);
+      task.setCommandLineArguments(line);
+      task.execute();
+
+    } catch (HiveMetaException e) {
+      System.err.println(e);
+      if (e.getCause() != null) {
+        Throwable t = e.getCause();
+        System.err.println("Underlying cause: " + t.getClass().getName() + " : " + t.getMessage());
+        if (e.getCause() instanceof SQLException) {
+          System.err.println("SQL Error code: " + ((SQLException)t).getErrorCode());
+        }
+      }
+      if (line.hasOption("verbose")) {
+        e.printStackTrace();
+      } else {
+        System.err.println("Use --verbose for detailed stacktrace.");
+      }
+      System.err.println("*** schemaTool failed ***");
+      System.exit(1);
+    }
+    System.out.println("schemaTool completed");
+    System.exit(0);
+  }
+
+  private static HiveSchemaTool createSchemaTool(HiveSchemaToolCommandLine line) throws HiveMetaException {
+    HiveSchemaTool schemaTool = new HiveSchemaTool(line.getDbType(), line.getMetaDbType());
+
+    if (line.hasOption("userName")) {
+      schemaTool.setUserName(line.getOptionValue("userName"));
+    } else {
+      schemaTool.setUserName(
+          schemaTool.getHiveConf().get(MetastoreConf.ConfVars.CONNECTION_USER_NAME.getVarname()));
+    }
+    if (line.hasOption("passWord")) {
+      schemaTool.setPassWord(line.getOptionValue("passWord"));
+    } else {
+      try {
+        schemaTool.setPassWord(ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(),
+            MetastoreConf.ConfVars.PWD.getVarname()));
+      } catch (IOException err) {
+        throw new HiveMetaException("Error getting metastore password", err);
+      }
+    }
+    if (line.hasOption("url")) {
+      schemaTool.setUrl(line.getOptionValue("url"));
+    }
+    if (line.hasOption("driver")) {
+      schemaTool.setDriver(line.getOptionValue("driver"));
+    }
+    if (line.hasOption("dryRun")) {
+      schemaTool.setDryRun(true);
+    }
+    if (line.hasOption("verbose")) {
+      schemaTool.setVerbose(true);
+    }
+    if (line.hasOption("dbOpts")) {
+      schemaTool.setDbOpts(line.getOptionValue("dbOpts"));
+    }
+    if (line.hasOption("validate") && line.hasOption("servers")) {
+      schemaTool.setValidationServers(line.getOptionValue("servers"));
+    }
+    return schemaTool;
+  }
+}


[27/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
index 27b6cf8..af62ca1 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsRequest.java
@@ -436,13 +436,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FULL_TABLE_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
-                struct.fullTableNames = new ArrayList<String>(_list602.size);
-                String _elem603;
-                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
+                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
+                struct.fullTableNames = new ArrayList<String>(_list610.size);
+                String _elem611;
+                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
                 {
-                  _elem603 = iprot.readString();
-                  struct.fullTableNames.add(_elem603);
+                  _elem611 = iprot.readString();
+                  struct.fullTableNames.add(_elem611);
                 }
                 iprot.readListEnd();
               }
@@ -476,9 +476,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FULL_TABLE_NAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.fullTableNames.size()));
-          for (String _iter605 : struct.fullTableNames)
+          for (String _iter613 : struct.fullTableNames)
           {
-            oprot.writeString(_iter605);
+            oprot.writeString(_iter613);
           }
           oprot.writeListEnd();
         }
@@ -508,9 +508,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fullTableNames.size());
-        for (String _iter606 : struct.fullTableNames)
+        for (String _iter614 : struct.fullTableNames)
         {
-          oprot.writeString(_iter606);
+          oprot.writeString(_iter614);
         }
       }
       oprot.writeString(struct.validTxnList);
@@ -520,13 +520,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.fullTableNames = new ArrayList<String>(_list607.size);
-        String _elem608;
-        for (int _i609 = 0; _i609 < _list607.size; ++_i609)
+        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.fullTableNames = new ArrayList<String>(_list615.size);
+        String _elem616;
+        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
         {
-          _elem608 = iprot.readString();
-          struct.fullTableNames.add(_elem608);
+          _elem616 = iprot.readString();
+          struct.fullTableNames.add(_elem616);
         }
       }
       struct.setFullTableNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
index 7a1bbc7..615a422 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetValidWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TBL_VALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
-                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list618.size);
-                TableValidWriteIds _elem619;
-                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
+                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
+                struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list626.size);
+                TableValidWriteIds _elem627;
+                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
                 {
-                  _elem619 = new TableValidWriteIds();
-                  _elem619.read(iprot);
-                  struct.tblValidWriteIds.add(_elem619);
+                  _elem627 = new TableValidWriteIds();
+                  _elem627.read(iprot);
+                  struct.tblValidWriteIds.add(_elem627);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TBL_VALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tblValidWriteIds.size()));
-          for (TableValidWriteIds _iter621 : struct.tblValidWriteIds)
+          for (TableValidWriteIds _iter629 : struct.tblValidWriteIds)
           {
-            _iter621.write(oprot);
+            _iter629.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tblValidWriteIds.size());
-        for (TableValidWriteIds _iter622 : struct.tblValidWriteIds)
+        for (TableValidWriteIds _iter630 : struct.tblValidWriteIds)
         {
-          _iter622.write(oprot);
+          _iter630.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetValidWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list623.size);
-        TableValidWriteIds _elem624;
-        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
+        org.apache.thrift.protocol.TList _list631 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tblValidWriteIds = new ArrayList<TableValidWriteIds>(_list631.size);
+        TableValidWriteIds _elem632;
+        for (int _i633 = 0; _i633 < _list631.size; ++_i633)
         {
-          _elem624 = new TableValidWriteIds();
-          _elem624.read(iprot);
-          struct.tblValidWriteIds.add(_elem624);
+          _elem632 = new TableValidWriteIds();
+          _elem632.read(iprot);
+          struct.tblValidWriteIds.add(_elem632);
         }
       }
       struct.setTblValidWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
index 4999215..a3dceab 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/HeartbeatTxnRangeResponse.java
@@ -453,13 +453,13 @@ import org.slf4j.LoggerFactory;
           case 1: // ABORTED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set666 = iprot.readSetBegin();
-                struct.aborted = new HashSet<Long>(2*_set666.size);
-                long _elem667;
-                for (int _i668 = 0; _i668 < _set666.size; ++_i668)
+                org.apache.thrift.protocol.TSet _set674 = iprot.readSetBegin();
+                struct.aborted = new HashSet<Long>(2*_set674.size);
+                long _elem675;
+                for (int _i676 = 0; _i676 < _set674.size; ++_i676)
                 {
-                  _elem667 = iprot.readI64();
-                  struct.aborted.add(_elem667);
+                  _elem675 = iprot.readI64();
+                  struct.aborted.add(_elem675);
                 }
                 iprot.readSetEnd();
               }
@@ -471,13 +471,13 @@ import org.slf4j.LoggerFactory;
           case 2: // NOSUCH
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set669 = iprot.readSetBegin();
-                struct.nosuch = new HashSet<Long>(2*_set669.size);
-                long _elem670;
-                for (int _i671 = 0; _i671 < _set669.size; ++_i671)
+                org.apache.thrift.protocol.TSet _set677 = iprot.readSetBegin();
+                struct.nosuch = new HashSet<Long>(2*_set677.size);
+                long _elem678;
+                for (int _i679 = 0; _i679 < _set677.size; ++_i679)
                 {
-                  _elem670 = iprot.readI64();
-                  struct.nosuch.add(_elem670);
+                  _elem678 = iprot.readI64();
+                  struct.nosuch.add(_elem678);
                 }
                 iprot.readSetEnd();
               }
@@ -503,9 +503,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(ABORTED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.aborted.size()));
-          for (long _iter672 : struct.aborted)
+          for (long _iter680 : struct.aborted)
           {
-            oprot.writeI64(_iter672);
+            oprot.writeI64(_iter680);
           }
           oprot.writeSetEnd();
         }
@@ -515,9 +515,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(NOSUCH_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, struct.nosuch.size()));
-          for (long _iter673 : struct.nosuch)
+          for (long _iter681 : struct.nosuch)
           {
-            oprot.writeI64(_iter673);
+            oprot.writeI64(_iter681);
           }
           oprot.writeSetEnd();
         }
@@ -542,16 +542,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.aborted.size());
-        for (long _iter674 : struct.aborted)
+        for (long _iter682 : struct.aborted)
         {
-          oprot.writeI64(_iter674);
+          oprot.writeI64(_iter682);
         }
       }
       {
         oprot.writeI32(struct.nosuch.size());
-        for (long _iter675 : struct.nosuch)
+        for (long _iter683 : struct.nosuch)
         {
-          oprot.writeI64(_iter675);
+          oprot.writeI64(_iter683);
         }
       }
     }
@@ -560,24 +560,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, HeartbeatTxnRangeResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set676 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.aborted = new HashSet<Long>(2*_set676.size);
-        long _elem677;
-        for (int _i678 = 0; _i678 < _set676.size; ++_i678)
+        org.apache.thrift.protocol.TSet _set684 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.aborted = new HashSet<Long>(2*_set684.size);
+        long _elem685;
+        for (int _i686 = 0; _i686 < _set684.size; ++_i686)
         {
-          _elem677 = iprot.readI64();
-          struct.aborted.add(_elem677);
+          _elem685 = iprot.readI64();
+          struct.aborted.add(_elem685);
         }
       }
       struct.setAbortedIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set679 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.nosuch = new HashSet<Long>(2*_set679.size);
-        long _elem680;
-        for (int _i681 = 0; _i681 < _set679.size; ++_i681)
+        org.apache.thrift.protocol.TSet _set687 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.nosuch = new HashSet<Long>(2*_set687.size);
+        long _elem688;
+        for (int _i689 = 0; _i689 < _set687.size; ++_i689)
         {
-          _elem680 = iprot.readI64();
-          struct.nosuch.add(_elem680);
+          _elem688 = iprot.readI64();
+          struct.nosuch.add(_elem688);
         }
       }
       struct.setNosuchIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
index 0a240e0..4a9824b 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/InsertEventRequestData.java
@@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory;
   private static final org.apache.thrift.protocol.TField REPLACE_FIELD_DESC = new org.apache.thrift.protocol.TField("replace", org.apache.thrift.protocol.TType.BOOL, (short)1);
   private static final org.apache.thrift.protocol.TField FILES_ADDED_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAdded", org.apache.thrift.protocol.TType.LIST, (short)2);
   private static final org.apache.thrift.protocol.TField FILES_ADDED_CHECKSUM_FIELD_DESC = new org.apache.thrift.protocol.TField("filesAddedChecksum", org.apache.thrift.protocol.TType.LIST, (short)3);
+  private static final org.apache.thrift.protocol.TField SUB_DIRECTORY_LIST_FIELD_DESC = new org.apache.thrift.protocol.TField("subDirectoryList", org.apache.thrift.protocol.TType.LIST, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -51,12 +52,14 @@ import org.slf4j.LoggerFactory;
   private boolean replace; // optional
   private List<String> filesAdded; // required
   private List<String> filesAddedChecksum; // optional
+  private List<String> subDirectoryList; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     REPLACE((short)1, "replace"),
     FILES_ADDED((short)2, "filesAdded"),
-    FILES_ADDED_CHECKSUM((short)3, "filesAddedChecksum");
+    FILES_ADDED_CHECKSUM((short)3, "filesAddedChecksum"),
+    SUB_DIRECTORY_LIST((short)4, "subDirectoryList");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -77,6 +80,8 @@ import org.slf4j.LoggerFactory;
           return FILES_ADDED;
         case 3: // FILES_ADDED_CHECKSUM
           return FILES_ADDED_CHECKSUM;
+        case 4: // SUB_DIRECTORY_LIST
+          return SUB_DIRECTORY_LIST;
         default:
           return null;
       }
@@ -119,7 +124,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __REPLACE_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.REPLACE,_Fields.FILES_ADDED_CHECKSUM};
+  private static final _Fields optionals[] = {_Fields.REPLACE,_Fields.FILES_ADDED_CHECKSUM,_Fields.SUB_DIRECTORY_LIST};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -131,6 +136,9 @@ import org.slf4j.LoggerFactory;
     tmpMap.put(_Fields.FILES_ADDED_CHECKSUM, new org.apache.thrift.meta_data.FieldMetaData("filesAddedChecksum", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.SUB_DIRECTORY_LIST, new org.apache.thrift.meta_data.FieldMetaData("subDirectoryList", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(InsertEventRequestData.class, metaDataMap);
   }
@@ -159,6 +167,10 @@ import org.slf4j.LoggerFactory;
       List<String> __this__filesAddedChecksum = new ArrayList<String>(other.filesAddedChecksum);
       this.filesAddedChecksum = __this__filesAddedChecksum;
     }
+    if (other.isSetSubDirectoryList()) {
+      List<String> __this__subDirectoryList = new ArrayList<String>(other.subDirectoryList);
+      this.subDirectoryList = __this__subDirectoryList;
+    }
   }
 
   public InsertEventRequestData deepCopy() {
@@ -171,6 +183,7 @@ import org.slf4j.LoggerFactory;
     this.replace = false;
     this.filesAdded = null;
     this.filesAddedChecksum = null;
+    this.subDirectoryList = null;
   }
 
   public boolean isReplace() {
@@ -271,6 +284,44 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public int getSubDirectoryListSize() {
+    return (this.subDirectoryList == null) ? 0 : this.subDirectoryList.size();
+  }
+
+  public java.util.Iterator<String> getSubDirectoryListIterator() {
+    return (this.subDirectoryList == null) ? null : this.subDirectoryList.iterator();
+  }
+
+  public void addToSubDirectoryList(String elem) {
+    if (this.subDirectoryList == null) {
+      this.subDirectoryList = new ArrayList<String>();
+    }
+    this.subDirectoryList.add(elem);
+  }
+
+  public List<String> getSubDirectoryList() {
+    return this.subDirectoryList;
+  }
+
+  public void setSubDirectoryList(List<String> subDirectoryList) {
+    this.subDirectoryList = subDirectoryList;
+  }
+
+  public void unsetSubDirectoryList() {
+    this.subDirectoryList = null;
+  }
+
+  /** Returns true if field subDirectoryList is set (has been assigned a value) and false otherwise */
+  public boolean isSetSubDirectoryList() {
+    return this.subDirectoryList != null;
+  }
+
+  public void setSubDirectoryListIsSet(boolean value) {
+    if (!value) {
+      this.subDirectoryList = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case REPLACE:
@@ -297,6 +348,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case SUB_DIRECTORY_LIST:
+      if (value == null) {
+        unsetSubDirectoryList();
+      } else {
+        setSubDirectoryList((List<String>)value);
+      }
+      break;
+
     }
   }
 
@@ -311,6 +370,9 @@ import org.slf4j.LoggerFactory;
     case FILES_ADDED_CHECKSUM:
       return getFilesAddedChecksum();
 
+    case SUB_DIRECTORY_LIST:
+      return getSubDirectoryList();
+
     }
     throw new IllegalStateException();
   }
@@ -328,6 +390,8 @@ import org.slf4j.LoggerFactory;
       return isSetFilesAdded();
     case FILES_ADDED_CHECKSUM:
       return isSetFilesAddedChecksum();
+    case SUB_DIRECTORY_LIST:
+      return isSetSubDirectoryList();
     }
     throw new IllegalStateException();
   }
@@ -372,6 +436,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_subDirectoryList = true && this.isSetSubDirectoryList();
+    boolean that_present_subDirectoryList = true && that.isSetSubDirectoryList();
+    if (this_present_subDirectoryList || that_present_subDirectoryList) {
+      if (!(this_present_subDirectoryList && that_present_subDirectoryList))
+        return false;
+      if (!this.subDirectoryList.equals(that.subDirectoryList))
+        return false;
+    }
+
     return true;
   }
 
@@ -394,6 +467,11 @@ import org.slf4j.LoggerFactory;
     if (present_filesAddedChecksum)
       list.add(filesAddedChecksum);
 
+    boolean present_subDirectoryList = true && (isSetSubDirectoryList());
+    list.add(present_subDirectoryList);
+    if (present_subDirectoryList)
+      list.add(subDirectoryList);
+
     return list.hashCode();
   }
 
@@ -435,6 +513,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetSubDirectoryList()).compareTo(other.isSetSubDirectoryList());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSubDirectoryList()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.subDirectoryList, other.subDirectoryList);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -478,6 +566,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetSubDirectoryList()) {
+      if (!first) sb.append(", ");
+      sb.append("subDirectoryList:");
+      if (this.subDirectoryList == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.subDirectoryList);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -538,13 +636,13 @@ import org.slf4j.LoggerFactory;
           case 2: // FILES_ADDED
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
-                struct.filesAdded = new ArrayList<String>(_list724.size);
-                String _elem725;
-                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
+                org.apache.thrift.protocol.TList _list732 = iprot.readListBegin();
+                struct.filesAdded = new ArrayList<String>(_list732.size);
+                String _elem733;
+                for (int _i734 = 0; _i734 < _list732.size; ++_i734)
                 {
-                  _elem725 = iprot.readString();
-                  struct.filesAdded.add(_elem725);
+                  _elem733 = iprot.readString();
+                  struct.filesAdded.add(_elem733);
                 }
                 iprot.readListEnd();
               }
@@ -556,13 +654,13 @@ import org.slf4j.LoggerFactory;
           case 3: // FILES_ADDED_CHECKSUM
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list727 = iprot.readListBegin();
-                struct.filesAddedChecksum = new ArrayList<String>(_list727.size);
-                String _elem728;
-                for (int _i729 = 0; _i729 < _list727.size; ++_i729)
+                org.apache.thrift.protocol.TList _list735 = iprot.readListBegin();
+                struct.filesAddedChecksum = new ArrayList<String>(_list735.size);
+                String _elem736;
+                for (int _i737 = 0; _i737 < _list735.size; ++_i737)
                 {
-                  _elem728 = iprot.readString();
-                  struct.filesAddedChecksum.add(_elem728);
+                  _elem736 = iprot.readString();
+                  struct.filesAddedChecksum.add(_elem736);
                 }
                 iprot.readListEnd();
               }
@@ -571,6 +669,24 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // SUB_DIRECTORY_LIST
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list738 = iprot.readListBegin();
+                struct.subDirectoryList = new ArrayList<String>(_list738.size);
+                String _elem739;
+                for (int _i740 = 0; _i740 < _list738.size; ++_i740)
+                {
+                  _elem739 = iprot.readString();
+                  struct.subDirectoryList.add(_elem739);
+                }
+                iprot.readListEnd();
+              }
+              struct.setSubDirectoryListIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -593,9 +709,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILES_ADDED_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAdded.size()));
-          for (String _iter730 : struct.filesAdded)
+          for (String _iter741 : struct.filesAdded)
           {
-            oprot.writeString(_iter730);
+            oprot.writeString(_iter741);
           }
           oprot.writeListEnd();
         }
@@ -606,9 +722,23 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FILES_ADDED_CHECKSUM_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filesAddedChecksum.size()));
-            for (String _iter731 : struct.filesAddedChecksum)
+            for (String _iter742 : struct.filesAddedChecksum)
             {
-              oprot.writeString(_iter731);
+              oprot.writeString(_iter742);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.subDirectoryList != null) {
+        if (struct.isSetSubDirectoryList()) {
+          oprot.writeFieldBegin(SUB_DIRECTORY_LIST_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.subDirectoryList.size()));
+            for (String _iter743 : struct.subDirectoryList)
+            {
+              oprot.writeString(_iter743);
             }
             oprot.writeListEnd();
           }
@@ -634,9 +764,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.filesAdded.size());
-        for (String _iter732 : struct.filesAdded)
+        for (String _iter744 : struct.filesAdded)
         {
-          oprot.writeString(_iter732);
+          oprot.writeString(_iter744);
         }
       }
       BitSet optionals = new BitSet();
@@ -646,16 +776,28 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFilesAddedChecksum()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetSubDirectoryList()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetReplace()) {
         oprot.writeBool(struct.replace);
       }
       if (struct.isSetFilesAddedChecksum()) {
         {
           oprot.writeI32(struct.filesAddedChecksum.size());
-          for (String _iter733 : struct.filesAddedChecksum)
+          for (String _iter745 : struct.filesAddedChecksum)
+          {
+            oprot.writeString(_iter745);
+          }
+        }
+      }
+      if (struct.isSetSubDirectoryList()) {
+        {
+          oprot.writeI32(struct.subDirectoryList.size());
+          for (String _iter746 : struct.subDirectoryList)
           {
-            oprot.writeString(_iter733);
+            oprot.writeString(_iter746);
           }
         }
       }
@@ -665,34 +807,47 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, InsertEventRequestData struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list734 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.filesAdded = new ArrayList<String>(_list734.size);
-        String _elem735;
-        for (int _i736 = 0; _i736 < _list734.size; ++_i736)
+        org.apache.thrift.protocol.TList _list747 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.filesAdded = new ArrayList<String>(_list747.size);
+        String _elem748;
+        for (int _i749 = 0; _i749 < _list747.size; ++_i749)
         {
-          _elem735 = iprot.readString();
-          struct.filesAdded.add(_elem735);
+          _elem748 = iprot.readString();
+          struct.filesAdded.add(_elem748);
         }
       }
       struct.setFilesAddedIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.replace = iprot.readBool();
         struct.setReplaceIsSet(true);
       }
       if (incoming.get(1)) {
         {
-          org.apache.thrift.protocol.TList _list737 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.filesAddedChecksum = new ArrayList<String>(_list737.size);
-          String _elem738;
-          for (int _i739 = 0; _i739 < _list737.size; ++_i739)
+          org.apache.thrift.protocol.TList _list750 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.filesAddedChecksum = new ArrayList<String>(_list750.size);
+          String _elem751;
+          for (int _i752 = 0; _i752 < _list750.size; ++_i752)
           {
-            _elem738 = iprot.readString();
-            struct.filesAddedChecksum.add(_elem738);
+            _elem751 = iprot.readString();
+            struct.filesAddedChecksum.add(_elem751);
           }
         }
         struct.setFilesAddedChecksumIsSet(true);
       }
+      if (incoming.get(2)) {
+        {
+          org.apache.thrift.protocol.TList _list753 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.subDirectoryList = new ArrayList<String>(_list753.size);
+          String _elem754;
+          for (int _i755 = 0; _i755 < _list753.size; ++_i755)
+          {
+            _elem754 = iprot.readString();
+            struct.subDirectoryList.add(_elem754);
+          }
+        }
+        struct.setSubDirectoryListIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
index d0dc21c..d4eed32 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockRequest.java
@@ -689,14 +689,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPONENT
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
-                struct.component = new ArrayList<LockComponent>(_list650.size);
-                LockComponent _elem651;
-                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
+                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
+                struct.component = new ArrayList<LockComponent>(_list658.size);
+                LockComponent _elem659;
+                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
                 {
-                  _elem651 = new LockComponent();
-                  _elem651.read(iprot);
-                  struct.component.add(_elem651);
+                  _elem659 = new LockComponent();
+                  _elem659.read(iprot);
+                  struct.component.add(_elem659);
                 }
                 iprot.readListEnd();
               }
@@ -754,9 +754,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPONENT_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.component.size()));
-          for (LockComponent _iter653 : struct.component)
+          for (LockComponent _iter661 : struct.component)
           {
-            _iter653.write(oprot);
+            _iter661.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.component.size());
-        for (LockComponent _iter654 : struct.component)
+        for (LockComponent _iter662 : struct.component)
         {
-          _iter654.write(oprot);
+          _iter662.write(oprot);
         }
       }
       oprot.writeString(struct.user);
@@ -830,14 +830,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, LockRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.component = new ArrayList<LockComponent>(_list655.size);
-        LockComponent _elem656;
-        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
+        org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.component = new ArrayList<LockComponent>(_list663.size);
+        LockComponent _elem664;
+        for (int _i665 = 0; _i665 < _list663.size; ++_i665)
         {
-          _elem656 = new LockComponent();
-          _elem656.read(iprot);
-          struct.component.add(_elem656);
+          _elem664 = new LockComponent();
+          _elem664.read(iprot);
+          struct.component.add(_elem664);
         }
       }
       struct.setComponentIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
index 4e792bc..3510995 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Materialization.java
@@ -589,13 +589,13 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set840 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set840.size);
-                String _elem841;
-                for (int _i842 = 0; _i842 < _set840.size; ++_i842)
+                org.apache.thrift.protocol.TSet _set864 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set864.size);
+                String _elem865;
+                for (int _i866 = 0; _i866 < _set864.size; ++_i866)
                 {
-                  _elem841 = iprot.readString();
-                  struct.tablesUsed.add(_elem841);
+                  _elem865 = iprot.readString();
+                  struct.tablesUsed.add(_elem865);
                 }
                 iprot.readSetEnd();
               }
@@ -645,9 +645,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter843 : struct.tablesUsed)
+          for (String _iter867 : struct.tablesUsed)
           {
-            oprot.writeString(_iter843);
+            oprot.writeString(_iter867);
           }
           oprot.writeSetEnd();
         }
@@ -689,9 +689,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter844 : struct.tablesUsed)
+        for (String _iter868 : struct.tablesUsed)
         {
-          oprot.writeString(_iter844);
+          oprot.writeString(_iter868);
         }
       }
       BitSet optionals = new BitSet();
@@ -720,13 +720,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, Materialization struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TSet _set845 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set845.size);
-        String _elem846;
-        for (int _i847 = 0; _i847 < _set845.size; ++_i847)
+        org.apache.thrift.protocol.TSet _set869 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set869.size);
+        String _elem870;
+        for (int _i871 = 0; _i871 < _set869.size; ++_i871)
         {
-          _elem846 = iprot.readString();
-          struct.tablesUsed.add(_elem846);
+          _elem870 = iprot.readString();
+          struct.tablesUsed.add(_elem870);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
index 0c850fa..9228c39 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/NotificationEventResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // EVENTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list716 = iprot.readListBegin();
-                struct.events = new ArrayList<NotificationEvent>(_list716.size);
-                NotificationEvent _elem717;
-                for (int _i718 = 0; _i718 < _list716.size; ++_i718)
+                org.apache.thrift.protocol.TList _list724 = iprot.readListBegin();
+                struct.events = new ArrayList<NotificationEvent>(_list724.size);
+                NotificationEvent _elem725;
+                for (int _i726 = 0; _i726 < _list724.size; ++_i726)
                 {
-                  _elem717 = new NotificationEvent();
-                  _elem717.read(iprot);
-                  struct.events.add(_elem717);
+                  _elem725 = new NotificationEvent();
+                  _elem725.read(iprot);
+                  struct.events.add(_elem725);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(EVENTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.events.size()));
-          for (NotificationEvent _iter719 : struct.events)
+          for (NotificationEvent _iter727 : struct.events)
           {
-            _iter719.write(oprot);
+            _iter727.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.events.size());
-        for (NotificationEvent _iter720 : struct.events)
+        for (NotificationEvent _iter728 : struct.events)
         {
-          _iter720.write(oprot);
+          _iter728.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, NotificationEventResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list721 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.events = new ArrayList<NotificationEvent>(_list721.size);
-        NotificationEvent _elem722;
-        for (int _i723 = 0; _i723 < _list721.size; ++_i723)
+        org.apache.thrift.protocol.TList _list729 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.events = new ArrayList<NotificationEvent>(_list729.size);
+        NotificationEvent _elem730;
+        for (int _i731 = 0; _i731 < _list729.size; ++_i731)
         {
-          _elem722 = new NotificationEvent();
-          _elem722.read(iprot);
-          struct.events.add(_elem722);
+          _elem730 = new NotificationEvent();
+          _elem730.read(iprot);
+          struct.events.add(_elem730);
         }
       }
       struct.setEventsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
index 77c260d..7d9ebba 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/PutFileMetadataRequest.java
@@ -547,13 +547,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list784 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list784.size);
-                long _elem785;
-                for (int _i786 = 0; _i786 < _list784.size; ++_i786)
+                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list808.size);
+                long _elem809;
+                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
                 {
-                  _elem785 = iprot.readI64();
-                  struct.fileIds.add(_elem785);
+                  _elem809 = iprot.readI64();
+                  struct.fileIds.add(_elem809);
                 }
                 iprot.readListEnd();
               }
@@ -565,13 +565,13 @@ import org.slf4j.LoggerFactory;
           case 2: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list787 = iprot.readListBegin();
-                struct.metadata = new ArrayList<ByteBuffer>(_list787.size);
-                ByteBuffer _elem788;
-                for (int _i789 = 0; _i789 < _list787.size; ++_i789)
+                org.apache.thrift.protocol.TList _list811 = iprot.readListBegin();
+                struct.metadata = new ArrayList<ByteBuffer>(_list811.size);
+                ByteBuffer _elem812;
+                for (int _i813 = 0; _i813 < _list811.size; ++_i813)
                 {
-                  _elem788 = iprot.readBinary();
-                  struct.metadata.add(_elem788);
+                  _elem812 = iprot.readBinary();
+                  struct.metadata.add(_elem812);
                 }
                 iprot.readListEnd();
               }
@@ -605,9 +605,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter790 : struct.fileIds)
+          for (long _iter814 : struct.fileIds)
           {
-            oprot.writeI64(_iter790);
+            oprot.writeI64(_iter814);
           }
           oprot.writeListEnd();
         }
@@ -617,9 +617,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (ByteBuffer _iter791 : struct.metadata)
+          for (ByteBuffer _iter815 : struct.metadata)
           {
-            oprot.writeBinary(_iter791);
+            oprot.writeBinary(_iter815);
           }
           oprot.writeListEnd();
         }
@@ -651,16 +651,16 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter792 : struct.fileIds)
+        for (long _iter816 : struct.fileIds)
         {
-          oprot.writeI64(_iter792);
+          oprot.writeI64(_iter816);
         }
       }
       {
         oprot.writeI32(struct.metadata.size());
-        for (ByteBuffer _iter793 : struct.metadata)
+        for (ByteBuffer _iter817 : struct.metadata)
         {
-          oprot.writeBinary(_iter793);
+          oprot.writeBinary(_iter817);
         }
       }
       BitSet optionals = new BitSet();
@@ -677,24 +677,24 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, PutFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list794 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list794.size);
-        long _elem795;
-        for (int _i796 = 0; _i796 < _list794.size; ++_i796)
+        org.apache.thrift.protocol.TList _list818 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list818.size);
+        long _elem819;
+        for (int _i820 = 0; _i820 < _list818.size; ++_i820)
         {
-          _elem795 = iprot.readI64();
-          struct.fileIds.add(_elem795);
+          _elem819 = iprot.readI64();
+          struct.fileIds.add(_elem819);
         }
       }
       struct.setFileIdsIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list797 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new ArrayList<ByteBuffer>(_list797.size);
-        ByteBuffer _elem798;
-        for (int _i799 = 0; _i799 < _list797.size; ++_i799)
+        org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new ArrayList<ByteBuffer>(_list821.size);
+        ByteBuffer _elem822;
+        for (int _i823 = 0; _i823 < _list821.size; ++_i823)
         {
-          _elem798 = iprot.readBinary();
-          struct.metadata.add(_elem798);
+          _elem822 = iprot.readBinary();
+          struct.metadata.add(_elem822);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
index 97bb8a4..0aeca14 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ReplTblWriteIdStateRequest.java
@@ -813,13 +813,13 @@ import org.slf4j.LoggerFactory;
           case 6: // PART_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
-                struct.partNames = new ArrayList<String>(_list594.size);
-                String _elem595;
-                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
+                org.apache.thrift.protocol.TList _list602 = iprot.readListBegin();
+                struct.partNames = new ArrayList<String>(_list602.size);
+                String _elem603;
+                for (int _i604 = 0; _i604 < _list602.size; ++_i604)
                 {
-                  _elem595 = iprot.readString();
-                  struct.partNames.add(_elem595);
+                  _elem603 = iprot.readString();
+                  struct.partNames.add(_elem603);
                 }
                 iprot.readListEnd();
               }
@@ -871,9 +871,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PART_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partNames.size()));
-            for (String _iter597 : struct.partNames)
+            for (String _iter605 : struct.partNames)
             {
-              oprot.writeString(_iter597);
+              oprot.writeString(_iter605);
             }
             oprot.writeListEnd();
           }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartNames()) {
         {
           oprot.writeI32(struct.partNames.size());
-          for (String _iter598 : struct.partNames)
+          for (String _iter606 : struct.partNames)
           {
-            oprot.writeString(_iter598);
+            oprot.writeString(_iter606);
           }
         }
       }
@@ -934,13 +934,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partNames = new ArrayList<String>(_list599.size);
-          String _elem600;
-          for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+          org.apache.thrift.protocol.TList _list607 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partNames = new ArrayList<String>(_list607.size);
+          String _elem608;
+          for (int _i609 = 0; _i609 < _list607.size; ++_i609)
           {
-            _elem600 = iprot.readString();
-            struct.partNames.add(_elem600);
+            _elem608 = iprot.readString();
+            struct.partNames.add(_elem608);
           }
         }
         struct.setPartNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
index 76dfe17..88d7e3f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SchemaVersion.java
@@ -1119,14 +1119,14 @@ import org.slf4j.LoggerFactory;
           case 4: // COLS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list912 = iprot.readListBegin();
-                struct.cols = new ArrayList<FieldSchema>(_list912.size);
-                FieldSchema _elem913;
-                for (int _i914 = 0; _i914 < _list912.size; ++_i914)
+                org.apache.thrift.protocol.TList _list936 = iprot.readListBegin();
+                struct.cols = new ArrayList<FieldSchema>(_list936.size);
+                FieldSchema _elem937;
+                for (int _i938 = 0; _i938 < _list936.size; ++_i938)
                 {
-                  _elem913 = new FieldSchema();
-                  _elem913.read(iprot);
-                  struct.cols.add(_elem913);
+                  _elem937 = new FieldSchema();
+                  _elem937.read(iprot);
+                  struct.cols.add(_elem937);
                 }
                 iprot.readListEnd();
               }
@@ -1212,9 +1212,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COLS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.cols.size()));
-          for (FieldSchema _iter915 : struct.cols)
+          for (FieldSchema _iter939 : struct.cols)
           {
-            _iter915.write(oprot);
+            _iter939.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -1323,9 +1323,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetCols()) {
         {
           oprot.writeI32(struct.cols.size());
-          for (FieldSchema _iter916 : struct.cols)
+          for (FieldSchema _iter940 : struct.cols)
           {
-            _iter916.write(oprot);
+            _iter940.write(oprot);
           }
         }
       }
@@ -1368,14 +1368,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(3)) {
         {
-          org.apache.thrift.protocol.TList _list917 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.cols = new ArrayList<FieldSchema>(_list917.size);
-          FieldSchema _elem918;
-          for (int _i919 = 0; _i919 < _list917.size; ++_i919)
+          org.apache.thrift.protocol.TList _list941 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.cols = new ArrayList<FieldSchema>(_list941.size);
+          FieldSchema _elem942;
+          for (int _i943 = 0; _i943 < _list941.size; ++_i943)
           {
-            _elem918 = new FieldSchema();
-            _elem918.read(iprot);
-            struct.cols.add(_elem918);
+            _elem942 = new FieldSchema();
+            _elem942.read(iprot);
+            struct.cols.add(_elem942);
           }
         }
         struct.setColsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
index d7e5132..9fb037f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowCompactResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // COMPACTS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list692 = iprot.readListBegin();
-                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list692.size);
-                ShowCompactResponseElement _elem693;
-                for (int _i694 = 0; _i694 < _list692.size; ++_i694)
+                org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
+                struct.compacts = new ArrayList<ShowCompactResponseElement>(_list700.size);
+                ShowCompactResponseElement _elem701;
+                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
                 {
-                  _elem693 = new ShowCompactResponseElement();
-                  _elem693.read(iprot);
-                  struct.compacts.add(_elem693);
+                  _elem701 = new ShowCompactResponseElement();
+                  _elem701.read(iprot);
+                  struct.compacts.add(_elem701);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(COMPACTS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.compacts.size()));
-          for (ShowCompactResponseElement _iter695 : struct.compacts)
+          for (ShowCompactResponseElement _iter703 : struct.compacts)
           {
-            _iter695.write(oprot);
+            _iter703.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.compacts.size());
-        for (ShowCompactResponseElement _iter696 : struct.compacts)
+        for (ShowCompactResponseElement _iter704 : struct.compacts)
         {
-          _iter696.write(oprot);
+          _iter704.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ShowCompactResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list697 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list697.size);
-        ShowCompactResponseElement _elem698;
-        for (int _i699 = 0; _i699 < _list697.size; ++_i699)
+        org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.compacts = new ArrayList<ShowCompactResponseElement>(_list705.size);
+        ShowCompactResponseElement _elem706;
+        for (int _i707 = 0; _i707 < _list705.size; ++_i707)
         {
-          _elem698 = new ShowCompactResponseElement();
-          _elem698.read(iprot);
-          struct.compacts.add(_elem698);
+          _elem706 = new ShowCompactResponseElement();
+          _elem706.read(iprot);
+          struct.compacts.add(_elem706);
         }
       }
       struct.setCompactsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
index 0e1009c..e0db2f7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ShowLocksResponse.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // LOCKS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list658 = iprot.readListBegin();
-                struct.locks = new ArrayList<ShowLocksResponseElement>(_list658.size);
-                ShowLocksResponseElement _elem659;
-                for (int _i660 = 0; _i660 < _list658.size; ++_i660)
+                org.apache.thrift.protocol.TList _list666 = iprot.readListBegin();
+                struct.locks = new ArrayList<ShowLocksResponseElement>(_list666.size);
+                ShowLocksResponseElement _elem667;
+                for (int _i668 = 0; _i668 < _list666.size; ++_i668)
                 {
-                  _elem659 = new ShowLocksResponseElement();
-                  _elem659.read(iprot);
-                  struct.locks.add(_elem659);
+                  _elem667 = new ShowLocksResponseElement();
+                  _elem667.read(iprot);
+                  struct.locks.add(_elem667);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(LOCKS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.locks.size()));
-          for (ShowLocksResponseElement _iter661 : struct.locks)
+          for (ShowLocksResponseElement _iter669 : struct.locks)
           {
-            _iter661.write(oprot);
+            _iter669.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetLocks()) {
         {
           oprot.writeI32(struct.locks.size());
-          for (ShowLocksResponseElement _iter662 : struct.locks)
+          for (ShowLocksResponseElement _iter670 : struct.locks)
           {
-            _iter662.write(oprot);
+            _iter670.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list663 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.locks = new ArrayList<ShowLocksResponseElement>(_list663.size);
-          ShowLocksResponseElement _elem664;
-          for (int _i665 = 0; _i665 < _list663.size; ++_i665)
+          org.apache.thrift.protocol.TList _list671 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.locks = new ArrayList<ShowLocksResponseElement>(_list671.size);
+          ShowLocksResponseElement _elem672;
+          for (int _i673 = 0; _i673 < _list671.size; ++_i673)
           {
-            _elem664 = new ShowLocksResponseElement();
-            _elem664.read(iprot);
-            struct.locks.add(_elem664);
+            _elem672 = new ShowLocksResponseElement();
+            _elem672.read(iprot);
+            struct.locks.add(_elem672);
           }
         }
         struct.setLocksIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
index 20f225d..de15fc6 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/TableValidWriteIds.java
@@ -708,13 +708,13 @@ import org.slf4j.LoggerFactory;
           case 3: // INVALID_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list610 = iprot.readListBegin();
-                struct.invalidWriteIds = new ArrayList<Long>(_list610.size);
-                long _elem611;
-                for (int _i612 = 0; _i612 < _list610.size; ++_i612)
+                org.apache.thrift.protocol.TList _list618 = iprot.readListBegin();
+                struct.invalidWriteIds = new ArrayList<Long>(_list618.size);
+                long _elem619;
+                for (int _i620 = 0; _i620 < _list618.size; ++_i620)
                 {
-                  _elem611 = iprot.readI64();
-                  struct.invalidWriteIds.add(_elem611);
+                  _elem619 = iprot.readI64();
+                  struct.invalidWriteIds.add(_elem619);
                 }
                 iprot.readListEnd();
               }
@@ -764,9 +764,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(INVALID_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.invalidWriteIds.size()));
-          for (long _iter613 : struct.invalidWriteIds)
+          for (long _iter621 : struct.invalidWriteIds)
           {
-            oprot.writeI64(_iter613);
+            oprot.writeI64(_iter621);
           }
           oprot.writeListEnd();
         }
@@ -803,9 +803,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeI64(struct.writeIdHighWaterMark);
       {
         oprot.writeI32(struct.invalidWriteIds.size());
-        for (long _iter614 : struct.invalidWriteIds)
+        for (long _iter622 : struct.invalidWriteIds)
         {
-          oprot.writeI64(_iter614);
+          oprot.writeI64(_iter622);
         }
       }
       oprot.writeBinary(struct.abortedBits);
@@ -827,13 +827,13 @@ import org.slf4j.LoggerFactory;
       struct.writeIdHighWaterMark = iprot.readI64();
       struct.setWriteIdHighWaterMarkIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list615 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.invalidWriteIds = new ArrayList<Long>(_list615.size);
-        long _elem616;
-        for (int _i617 = 0; _i617 < _list615.size; ++_i617)
+        org.apache.thrift.protocol.TList _list623 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.invalidWriteIds = new ArrayList<Long>(_list623.size);
+        long _elem624;
+        for (int _i625 = 0; _i625 < _list623.size; ++_i625)
         {
-          _elem616 = iprot.readI64();
-          struct.invalidWriteIds.add(_elem616);
+          _elem624 = iprot.readI64();
+          struct.invalidWriteIds.add(_elem624);
         }
       }
       struct.setInvalidWriteIdsIsSet(true);


[40/46] hive git commit: HIVE-19416 : merge master into branch (Sergey Shelukhin) 0703

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 7ced369,93b5780..de676d9
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@@ -31030,302 -30890,6 +31614,302 @@@ class GetRuntimeStatsRequest 
  
  }
  
 +class AlterPartitionsRequest {
 +  static $_TSPEC;
 +
 +  /**
 +   * @var string
 +   */
 +  public $dbName = null;
 +  /**
 +   * @var string
 +   */
 +  public $tableName = null;
 +  /**
 +   * @var \metastore\Partition[]
 +   */
 +  public $partitions = null;
 +  /**
 +   * @var \metastore\EnvironmentContext
 +   */
 +  public $environmentContext = null;
 +  /**
 +   * @var int
 +   */
 +  public $txnId = -1;
 +  /**
 +   * @var int
 +   */
 +  public $writeId = -1;
 +  /**
 +   * @var string
 +   */
 +  public $validWriteIdList = null;
 +
 +  public function __construct($vals=null) {
 +    if (!isset(self::$_TSPEC)) {
 +      self::$_TSPEC = array(
 +        1 => array(
 +          'var' => 'dbName',
 +          'type' => TType::STRING,
 +          ),
 +        2 => array(
 +          'var' => 'tableName',
 +          'type' => TType::STRING,
 +          ),
 +        3 => array(
 +          'var' => 'partitions',
 +          'type' => TType::LST,
 +          'etype' => TType::STRUCT,
 +          'elem' => array(
 +            'type' => TType::STRUCT,
 +            'class' => '\metastore\Partition',
 +            ),
 +          ),
 +        4 => array(
 +          'var' => 'environmentContext',
 +          'type' => TType::STRUCT,
 +          'class' => '\metastore\EnvironmentContext',
 +          ),
 +        5 => array(
 +          'var' => 'txnId',
 +          'type' => TType::I64,
 +          ),
 +        6 => array(
 +          'var' => 'writeId',
 +          'type' => TType::I64,
 +          ),
 +        7 => array(
 +          'var' => 'validWriteIdList',
 +          'type' => TType::STRING,
 +          ),
 +        );
 +    }
 +    if (is_array($vals)) {
 +      if (isset($vals['dbName'])) {
 +        $this->dbName = $vals['dbName'];
 +      }
 +      if (isset($vals['tableName'])) {
 +        $this->tableName = $vals['tableName'];
 +      }
 +      if (isset($vals['partitions'])) {
 +        $this->partitions = $vals['partitions'];
 +      }
 +      if (isset($vals['environmentContext'])) {
 +        $this->environmentContext = $vals['environmentContext'];
 +      }
 +      if (isset($vals['txnId'])) {
 +        $this->txnId = $vals['txnId'];
 +      }
 +      if (isset($vals['writeId'])) {
 +        $this->writeId = $vals['writeId'];
 +      }
 +      if (isset($vals['validWriteIdList'])) {
 +        $this->validWriteIdList = $vals['validWriteIdList'];
 +      }
 +    }
 +  }
 +
 +  public function getName() {
 +    return 'AlterPartitionsRequest';
 +  }
 +
 +  public function read($input)
 +  {
 +    $xfer = 0;
 +    $fname = null;
 +    $ftype = 0;
 +    $fid = 0;
 +    $xfer += $input->readStructBegin($fname);
 +    while (true)
 +    {
 +      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
 +      if ($ftype == TType::STOP) {
 +        break;
 +      }
 +      switch ($fid)
 +      {
 +        case 1:
 +          if ($ftype == TType::STRING) {
 +            $xfer += $input->readString($this->dbName);
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        case 2:
 +          if ($ftype == TType::STRING) {
 +            $xfer += $input->readString($this->tableName);
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        case 3:
 +          if ($ftype == TType::LST) {
 +            $this->partitions = array();
-             $_size820 = 0;
-             $_etype823 = 0;
-             $xfer += $input->readListBegin($_etype823, $_size820);
-             for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
++            $_size841 = 0;
++            $_etype844 = 0;
++            $xfer += $input->readListBegin($_etype844, $_size841);
++            for ($_i845 = 0; $_i845 < $_size841; ++$_i845)
 +            {
-               $elem825 = null;
-               $elem825 = new \metastore\Partition();
-               $xfer += $elem825->read($input);
-               $this->partitions []= $elem825;
++              $elem846 = null;
++              $elem846 = new \metastore\Partition();
++              $xfer += $elem846->read($input);
++              $this->partitions []= $elem846;
 +            }
 +            $xfer += $input->readListEnd();
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        case 4:
 +          if ($ftype == TType::STRUCT) {
 +            $this->environmentContext = new \metastore\EnvironmentContext();
 +            $xfer += $this->environmentContext->read($input);
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        case 5:
 +          if ($ftype == TType::I64) {
 +            $xfer += $input->readI64($this->txnId);
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        case 6:
 +          if ($ftype == TType::I64) {
 +            $xfer += $input->readI64($this->writeId);
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        case 7:
 +          if ($ftype == TType::STRING) {
 +            $xfer += $input->readString($this->validWriteIdList);
 +          } else {
 +            $xfer += $input->skip($ftype);
 +          }
 +          break;
 +        default:
 +          $xfer += $input->skip($ftype);
 +          break;
 +      }
 +      $xfer += $input->readFieldEnd();
 +    }
 +    $xfer += $input->readStructEnd();
 +    return $xfer;
 +  }
 +
 +  public function write($output) {
 +    $xfer = 0;
 +    $xfer += $output->writeStructBegin('AlterPartitionsRequest');
 +    if ($this->dbName !== null) {
 +      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 1);
 +      $xfer += $output->writeString($this->dbName);
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    if ($this->tableName !== null) {
 +      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 2);
 +      $xfer += $output->writeString($this->tableName);
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    if ($this->partitions !== null) {
 +      if (!is_array($this->partitions)) {
 +        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
 +      }
 +      $xfer += $output->writeFieldBegin('partitions', TType::LST, 3);
 +      {
 +        $output->writeListBegin(TType::STRUCT, count($this->partitions));
 +        {
-           foreach ($this->partitions as $iter826)
++          foreach ($this->partitions as $iter847)
 +          {
-             $xfer += $iter826->write($output);
++            $xfer += $iter847->write($output);
 +          }
 +        }
 +        $output->writeListEnd();
 +      }
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    if ($this->environmentContext !== null) {
 +      if (!is_object($this->environmentContext)) {
 +        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
 +      }
 +      $xfer += $output->writeFieldBegin('environmentContext', TType::STRUCT, 4);
 +      $xfer += $this->environmentContext->write($output);
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    if ($this->txnId !== null) {
 +      $xfer += $output->writeFieldBegin('txnId', TType::I64, 5);
 +      $xfer += $output->writeI64($this->txnId);
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    if ($this->writeId !== null) {
 +      $xfer += $output->writeFieldBegin('writeId', TType::I64, 6);
 +      $xfer += $output->writeI64($this->writeId);
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    if ($this->validWriteIdList !== null) {
 +      $xfer += $output->writeFieldBegin('validWriteIdList', TType::STRING, 7);
 +      $xfer += $output->writeString($this->validWriteIdList);
 +      $xfer += $output->writeFieldEnd();
 +    }
 +    $xfer += $output->writeFieldStop();
 +    $xfer += $output->writeStructEnd();
 +    return $xfer;
 +  }
 +
 +}
 +
 +class AlterPartitionsResponse {
 +  static $_TSPEC;
 +
 +
 +  public function __construct() {
 +    if (!isset(self::$_TSPEC)) {
 +      self::$_TSPEC = array(
 +        );
 +    }
 +  }
 +
 +  public function getName() {
 +    return 'AlterPartitionsResponse';
 +  }
 +
 +  public function read($input)
 +  {
 +    $xfer = 0;
 +    $fname = null;
 +    $ftype = 0;
 +    $fid = 0;
 +    $xfer += $input->readStructBegin($fname);
 +    while (true)
 +    {
 +      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
 +      if ($ftype == TType::STOP) {
 +        break;
 +      }
 +      switch ($fid)
 +      {
 +        default:
 +          $xfer += $input->skip($ftype);
 +          break;
 +      }
 +      $xfer += $input->readFieldEnd();
 +    }
 +    $xfer += $input->readStructEnd();
 +    return $xfer;
 +  }
 +
 +  public function write($output) {
 +    $xfer = 0;
 +    $xfer += $output->writeStructBegin('AlterPartitionsResponse');
 +    $xfer += $output->writeFieldStop();
 +    $xfer += $output->writeStructEnd();
 +    return $xfer;
 +  }
 +
 +}
 +
  class MetaException extends TException {
    static $_TSPEC;
  

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore-remote
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/4db8b1c6/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
----------------------------------------------------------------------
diff --cc standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
index cb5e158,a5bcc10..952a158
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ThriftHiveMetastore.py
@@@ -15980,10 -16045,10 +16038,10 @@@ class get_databases_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype826, _size823) = iprot.readListBegin()
-           for _i827 in xrange(_size823):
-             _elem828 = iprot.readString()
-             self.success.append(_elem828)
 -          (_etype840, _size837) = iprot.readListBegin()
 -          for _i841 in xrange(_size837):
 -            _elem842 = iprot.readString()
 -            self.success.append(_elem842)
++          (_etype847, _size844) = iprot.readListBegin()
++          for _i848 in xrange(_size844):
++            _elem849 = iprot.readString()
++            self.success.append(_elem849)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -16006,8 -16071,8 +16064,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter829 in self.success:
-         oprot.writeString(iter829)
 -      for iter843 in self.success:
 -        oprot.writeString(iter843)
++      for iter850 in self.success:
++        oprot.writeString(iter850)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -16112,10 -16177,10 +16170,10 @@@ class get_all_databases_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype833, _size830) = iprot.readListBegin()
-           for _i834 in xrange(_size830):
-             _elem835 = iprot.readString()
-             self.success.append(_elem835)
 -          (_etype847, _size844) = iprot.readListBegin()
 -          for _i848 in xrange(_size844):
 -            _elem849 = iprot.readString()
 -            self.success.append(_elem849)
++          (_etype854, _size851) = iprot.readListBegin()
++          for _i855 in xrange(_size851):
++            _elem856 = iprot.readString()
++            self.success.append(_elem856)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -16138,8 -16203,8 +16196,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter836 in self.success:
-         oprot.writeString(iter836)
 -      for iter850 in self.success:
 -        oprot.writeString(iter850)
++      for iter857 in self.success:
++        oprot.writeString(iter857)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -16909,12 -16974,12 +16967,12 @@@ class get_type_all_result
        if fid == 0:
          if ftype == TType.MAP:
            self.success = {}
-           (_ktype838, _vtype839, _size837 ) = iprot.readMapBegin()
-           for _i841 in xrange(_size837):
-             _key842 = iprot.readString()
-             _val843 = Type()
-             _val843.read(iprot)
-             self.success[_key842] = _val843
 -          (_ktype852, _vtype853, _size851 ) = iprot.readMapBegin()
 -          for _i855 in xrange(_size851):
 -            _key856 = iprot.readString()
 -            _val857 = Type()
 -            _val857.read(iprot)
 -            self.success[_key856] = _val857
++          (_ktype859, _vtype860, _size858 ) = iprot.readMapBegin()
++          for _i862 in xrange(_size858):
++            _key863 = iprot.readString()
++            _val864 = Type()
++            _val864.read(iprot)
++            self.success[_key863] = _val864
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -16937,9 -17002,9 +16995,9 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.MAP, 0)
        oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-       for kiter844,viter845 in self.success.items():
-         oprot.writeString(kiter844)
-         viter845.write(oprot)
 -      for kiter858,viter859 in self.success.items():
 -        oprot.writeString(kiter858)
 -        viter859.write(oprot)
++      for kiter865,viter866 in self.success.items():
++        oprot.writeString(kiter865)
++        viter866.write(oprot)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.o2 is not None:
@@@ -17082,11 -17147,11 +17140,11 @@@ class get_fields_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype849, _size846) = iprot.readListBegin()
-           for _i850 in xrange(_size846):
-             _elem851 = FieldSchema()
-             _elem851.read(iprot)
-             self.success.append(_elem851)
 -          (_etype863, _size860) = iprot.readListBegin()
 -          for _i864 in xrange(_size860):
 -            _elem865 = FieldSchema()
 -            _elem865.read(iprot)
 -            self.success.append(_elem865)
++          (_etype870, _size867) = iprot.readListBegin()
++          for _i871 in xrange(_size867):
++            _elem872 = FieldSchema()
++            _elem872.read(iprot)
++            self.success.append(_elem872)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -17121,8 -17186,8 +17179,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter852 in self.success:
-         iter852.write(oprot)
 -      for iter866 in self.success:
 -        iter866.write(oprot)
++      for iter873 in self.success:
++        iter873.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -17289,11 -17354,11 +17347,11 @@@ class get_fields_with_environment_conte
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype856, _size853) = iprot.readListBegin()
-           for _i857 in xrange(_size853):
-             _elem858 = FieldSchema()
-             _elem858.read(iprot)
-             self.success.append(_elem858)
 -          (_etype870, _size867) = iprot.readListBegin()
 -          for _i871 in xrange(_size867):
 -            _elem872 = FieldSchema()
 -            _elem872.read(iprot)
 -            self.success.append(_elem872)
++          (_etype877, _size874) = iprot.readListBegin()
++          for _i878 in xrange(_size874):
++            _elem879 = FieldSchema()
++            _elem879.read(iprot)
++            self.success.append(_elem879)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -17328,8 -17393,8 +17386,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter859 in self.success:
-         iter859.write(oprot)
 -      for iter873 in self.success:
 -        iter873.write(oprot)
++      for iter880 in self.success:
++        iter880.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -17482,11 -17547,11 +17540,11 @@@ class get_schema_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype863, _size860) = iprot.readListBegin()
-           for _i864 in xrange(_size860):
-             _elem865 = FieldSchema()
-             _elem865.read(iprot)
-             self.success.append(_elem865)
 -          (_etype877, _size874) = iprot.readListBegin()
 -          for _i878 in xrange(_size874):
 -            _elem879 = FieldSchema()
 -            _elem879.read(iprot)
 -            self.success.append(_elem879)
++          (_etype884, _size881) = iprot.readListBegin()
++          for _i885 in xrange(_size881):
++            _elem886 = FieldSchema()
++            _elem886.read(iprot)
++            self.success.append(_elem886)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -17521,8 -17586,8 +17579,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter866 in self.success:
-         iter866.write(oprot)
 -      for iter880 in self.success:
 -        iter880.write(oprot)
++      for iter887 in self.success:
++        iter887.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -17689,11 -17754,11 +17747,11 @@@ class get_schema_with_environment_conte
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype870, _size867) = iprot.readListBegin()
-           for _i871 in xrange(_size867):
-             _elem872 = FieldSchema()
-             _elem872.read(iprot)
-             self.success.append(_elem872)
 -          (_etype884, _size881) = iprot.readListBegin()
 -          for _i885 in xrange(_size881):
 -            _elem886 = FieldSchema()
 -            _elem886.read(iprot)
 -            self.success.append(_elem886)
++          (_etype891, _size888) = iprot.readListBegin()
++          for _i892 in xrange(_size888):
++            _elem893 = FieldSchema()
++            _elem893.read(iprot)
++            self.success.append(_elem893)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -17728,8 -17793,8 +17786,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter873 in self.success:
-         iter873.write(oprot)
 -      for iter887 in self.success:
 -        iter887.write(oprot)
++      for iter894 in self.success:
++        iter894.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -18182,66 -18247,66 +18240,66 @@@ class create_table_with_constraints_arg
        elif fid == 2:
          if ftype == TType.LIST:
            self.primaryKeys = []
-           (_etype877, _size874) = iprot.readListBegin()
-           for _i878 in xrange(_size874):
-             _elem879 = SQLPrimaryKey()
-             _elem879.read(iprot)
-             self.primaryKeys.append(_elem879)
 -          (_etype891, _size888) = iprot.readListBegin()
 -          for _i892 in xrange(_size888):
 -            _elem893 = SQLPrimaryKey()
 -            _elem893.read(iprot)
 -            self.primaryKeys.append(_elem893)
++          (_etype898, _size895) = iprot.readListBegin()
++          for _i899 in xrange(_size895):
++            _elem900 = SQLPrimaryKey()
++            _elem900.read(iprot)
++            self.primaryKeys.append(_elem900)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
        elif fid == 3:
          if ftype == TType.LIST:
            self.foreignKeys = []
-           (_etype883, _size880) = iprot.readListBegin()
-           for _i884 in xrange(_size880):
-             _elem885 = SQLForeignKey()
-             _elem885.read(iprot)
-             self.foreignKeys.append(_elem885)
 -          (_etype897, _size894) = iprot.readListBegin()
 -          for _i898 in xrange(_size894):
 -            _elem899 = SQLForeignKey()
 -            _elem899.read(iprot)
 -            self.foreignKeys.append(_elem899)
++          (_etype904, _size901) = iprot.readListBegin()
++          for _i905 in xrange(_size901):
++            _elem906 = SQLForeignKey()
++            _elem906.read(iprot)
++            self.foreignKeys.append(_elem906)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
        elif fid == 4:
          if ftype == TType.LIST:
            self.uniqueConstraints = []
-           (_etype889, _size886) = iprot.readListBegin()
-           for _i890 in xrange(_size886):
-             _elem891 = SQLUniqueConstraint()
-             _elem891.read(iprot)
-             self.uniqueConstraints.append(_elem891)
 -          (_etype903, _size900) = iprot.readListBegin()
 -          for _i904 in xrange(_size900):
 -            _elem905 = SQLUniqueConstraint()
 -            _elem905.read(iprot)
 -            self.uniqueConstraints.append(_elem905)
++          (_etype910, _size907) = iprot.readListBegin()
++          for _i911 in xrange(_size907):
++            _elem912 = SQLUniqueConstraint()
++            _elem912.read(iprot)
++            self.uniqueConstraints.append(_elem912)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
        elif fid == 5:
          if ftype == TType.LIST:
            self.notNullConstraints = []
-           (_etype895, _size892) = iprot.readListBegin()
-           for _i896 in xrange(_size892):
-             _elem897 = SQLNotNullConstraint()
-             _elem897.read(iprot)
-             self.notNullConstraints.append(_elem897)
 -          (_etype909, _size906) = iprot.readListBegin()
 -          for _i910 in xrange(_size906):
 -            _elem911 = SQLNotNullConstraint()
 -            _elem911.read(iprot)
 -            self.notNullConstraints.append(_elem911)
++          (_etype916, _size913) = iprot.readListBegin()
++          for _i917 in xrange(_size913):
++            _elem918 = SQLNotNullConstraint()
++            _elem918.read(iprot)
++            self.notNullConstraints.append(_elem918)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
        elif fid == 6:
          if ftype == TType.LIST:
            self.defaultConstraints = []
-           (_etype901, _size898) = iprot.readListBegin()
-           for _i902 in xrange(_size898):
-             _elem903 = SQLDefaultConstraint()
-             _elem903.read(iprot)
-             self.defaultConstraints.append(_elem903)
 -          (_etype915, _size912) = iprot.readListBegin()
 -          for _i916 in xrange(_size912):
 -            _elem917 = SQLDefaultConstraint()
 -            _elem917.read(iprot)
 -            self.defaultConstraints.append(_elem917)
++          (_etype922, _size919) = iprot.readListBegin()
++          for _i923 in xrange(_size919):
++            _elem924 = SQLDefaultConstraint()
++            _elem924.read(iprot)
++            self.defaultConstraints.append(_elem924)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
        elif fid == 7:
          if ftype == TType.LIST:
            self.checkConstraints = []
-           (_etype907, _size904) = iprot.readListBegin()
-           for _i908 in xrange(_size904):
-             _elem909 = SQLCheckConstraint()
-             _elem909.read(iprot)
-             self.checkConstraints.append(_elem909)
 -          (_etype921, _size918) = iprot.readListBegin()
 -          for _i922 in xrange(_size918):
 -            _elem923 = SQLCheckConstraint()
 -            _elem923.read(iprot)
 -            self.checkConstraints.append(_elem923)
++          (_etype928, _size925) = iprot.readListBegin()
++          for _i929 in xrange(_size925):
++            _elem930 = SQLCheckConstraint()
++            _elem930.read(iprot)
++            self.checkConstraints.append(_elem930)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -18262,43 -18327,43 +18320,43 @@@
      if self.primaryKeys is not None:
        oprot.writeFieldBegin('primaryKeys', TType.LIST, 2)
        oprot.writeListBegin(TType.STRUCT, len(self.primaryKeys))
-       for iter910 in self.primaryKeys:
-         iter910.write(oprot)
 -      for iter924 in self.primaryKeys:
 -        iter924.write(oprot)
++      for iter931 in self.primaryKeys:
++        iter931.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.foreignKeys is not None:
        oprot.writeFieldBegin('foreignKeys', TType.LIST, 3)
        oprot.writeListBegin(TType.STRUCT, len(self.foreignKeys))
-       for iter911 in self.foreignKeys:
-         iter911.write(oprot)
 -      for iter925 in self.foreignKeys:
 -        iter925.write(oprot)
++      for iter932 in self.foreignKeys:
++        iter932.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.uniqueConstraints is not None:
        oprot.writeFieldBegin('uniqueConstraints', TType.LIST, 4)
        oprot.writeListBegin(TType.STRUCT, len(self.uniqueConstraints))
-       for iter912 in self.uniqueConstraints:
-         iter912.write(oprot)
 -      for iter926 in self.uniqueConstraints:
 -        iter926.write(oprot)
++      for iter933 in self.uniqueConstraints:
++        iter933.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.notNullConstraints is not None:
        oprot.writeFieldBegin('notNullConstraints', TType.LIST, 5)
        oprot.writeListBegin(TType.STRUCT, len(self.notNullConstraints))
-       for iter913 in self.notNullConstraints:
-         iter913.write(oprot)
 -      for iter927 in self.notNullConstraints:
 -        iter927.write(oprot)
++      for iter934 in self.notNullConstraints:
++        iter934.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.defaultConstraints is not None:
        oprot.writeFieldBegin('defaultConstraints', TType.LIST, 6)
        oprot.writeListBegin(TType.STRUCT, len(self.defaultConstraints))
-       for iter914 in self.defaultConstraints:
-         iter914.write(oprot)
 -      for iter928 in self.defaultConstraints:
 -        iter928.write(oprot)
++      for iter935 in self.defaultConstraints:
++        iter935.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.checkConstraints is not None:
        oprot.writeFieldBegin('checkConstraints', TType.LIST, 7)
        oprot.writeListBegin(TType.STRUCT, len(self.checkConstraints))
-       for iter915 in self.checkConstraints:
-         iter915.write(oprot)
 -      for iter929 in self.checkConstraints:
 -        iter929.write(oprot)
++      for iter936 in self.checkConstraints:
++        iter936.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -19858,10 -19923,10 +19916,10 @@@ class truncate_table_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.partNames = []
-           (_etype919, _size916) = iprot.readListBegin()
-           for _i920 in xrange(_size916):
-             _elem921 = iprot.readString()
-             self.partNames.append(_elem921)
 -          (_etype933, _size930) = iprot.readListBegin()
 -          for _i934 in xrange(_size930):
 -            _elem935 = iprot.readString()
 -            self.partNames.append(_elem935)
++          (_etype940, _size937) = iprot.readListBegin()
++          for _i941 in xrange(_size937):
++            _elem942 = iprot.readString()
++            self.partNames.append(_elem942)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -19886,8 -19951,8 +19944,8 @@@
      if self.partNames is not None:
        oprot.writeFieldBegin('partNames', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.partNames))
-       for iter922 in self.partNames:
-         oprot.writeString(iter922)
 -      for iter936 in self.partNames:
 -        oprot.writeString(iter936)
++      for iter943 in self.partNames:
++        oprot.writeString(iter943)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -20087,10 -20152,10 +20145,10 @@@ class get_tables_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype926, _size923) = iprot.readListBegin()
-           for _i927 in xrange(_size923):
-             _elem928 = iprot.readString()
-             self.success.append(_elem928)
 -          (_etype940, _size937) = iprot.readListBegin()
 -          for _i941 in xrange(_size937):
 -            _elem942 = iprot.readString()
 -            self.success.append(_elem942)
++          (_etype947, _size944) = iprot.readListBegin()
++          for _i948 in xrange(_size944):
++            _elem949 = iprot.readString()
++            self.success.append(_elem949)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20113,8 -20178,8 +20171,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter929 in self.success:
-         oprot.writeString(iter929)
 -      for iter943 in self.success:
 -        oprot.writeString(iter943)
++      for iter950 in self.success:
++        oprot.writeString(iter950)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20264,10 -20329,10 +20322,10 @@@ class get_tables_by_type_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype933, _size930) = iprot.readListBegin()
-           for _i934 in xrange(_size930):
-             _elem935 = iprot.readString()
-             self.success.append(_elem935)
 -          (_etype947, _size944) = iprot.readListBegin()
 -          for _i948 in xrange(_size944):
 -            _elem949 = iprot.readString()
 -            self.success.append(_elem949)
++          (_etype954, _size951) = iprot.readListBegin()
++          for _i955 in xrange(_size951):
++            _elem956 = iprot.readString()
++            self.success.append(_elem956)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20290,8 -20355,8 +20348,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter936 in self.success:
-         oprot.writeString(iter936)
 -      for iter950 in self.success:
 -        oprot.writeString(iter950)
++      for iter957 in self.success:
++        oprot.writeString(iter957)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20415,10 -20480,10 +20473,10 @@@ class get_materialized_views_for_rewrit
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype940, _size937) = iprot.readListBegin()
-           for _i941 in xrange(_size937):
-             _elem942 = iprot.readString()
-             self.success.append(_elem942)
 -          (_etype954, _size951) = iprot.readListBegin()
 -          for _i955 in xrange(_size951):
 -            _elem956 = iprot.readString()
 -            self.success.append(_elem956)
++          (_etype961, _size958) = iprot.readListBegin()
++          for _i962 in xrange(_size958):
++            _elem963 = iprot.readString()
++            self.success.append(_elem963)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20441,8 -20506,8 +20499,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter943 in self.success:
-         oprot.writeString(iter943)
 -      for iter957 in self.success:
 -        oprot.writeString(iter957)
++      for iter964 in self.success:
++        oprot.writeString(iter964)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20515,10 -20580,10 +20573,10 @@@ class get_table_meta_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.tbl_types = []
-           (_etype947, _size944) = iprot.readListBegin()
-           for _i948 in xrange(_size944):
-             _elem949 = iprot.readString()
-             self.tbl_types.append(_elem949)
 -          (_etype961, _size958) = iprot.readListBegin()
 -          for _i962 in xrange(_size958):
 -            _elem963 = iprot.readString()
 -            self.tbl_types.append(_elem963)
++          (_etype968, _size965) = iprot.readListBegin()
++          for _i969 in xrange(_size965):
++            _elem970 = iprot.readString()
++            self.tbl_types.append(_elem970)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20543,8 -20608,8 +20601,8 @@@
      if self.tbl_types is not None:
        oprot.writeFieldBegin('tbl_types', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.tbl_types))
-       for iter950 in self.tbl_types:
-         oprot.writeString(iter950)
 -      for iter964 in self.tbl_types:
 -        oprot.writeString(iter964)
++      for iter971 in self.tbl_types:
++        oprot.writeString(iter971)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -20600,11 -20665,11 +20658,11 @@@ class get_table_meta_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype954, _size951) = iprot.readListBegin()
-           for _i955 in xrange(_size951):
-             _elem956 = TableMeta()
-             _elem956.read(iprot)
-             self.success.append(_elem956)
 -          (_etype968, _size965) = iprot.readListBegin()
 -          for _i969 in xrange(_size965):
 -            _elem970 = TableMeta()
 -            _elem970.read(iprot)
 -            self.success.append(_elem970)
++          (_etype975, _size972) = iprot.readListBegin()
++          for _i976 in xrange(_size972):
++            _elem977 = TableMeta()
++            _elem977.read(iprot)
++            self.success.append(_elem977)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20627,8 -20692,8 +20685,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter957 in self.success:
-         iter957.write(oprot)
 -      for iter971 in self.success:
 -        iter971.write(oprot)
++      for iter978 in self.success:
++        iter978.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -20752,10 -20817,10 +20810,10 @@@ class get_all_tables_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype961, _size958) = iprot.readListBegin()
-           for _i962 in xrange(_size958):
-             _elem963 = iprot.readString()
-             self.success.append(_elem963)
 -          (_etype975, _size972) = iprot.readListBegin()
 -          for _i976 in xrange(_size972):
 -            _elem977 = iprot.readString()
 -            self.success.append(_elem977)
++          (_etype982, _size979) = iprot.readListBegin()
++          for _i983 in xrange(_size979):
++            _elem984 = iprot.readString()
++            self.success.append(_elem984)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -20778,8 -20843,8 +20836,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter964 in self.success:
-         oprot.writeString(iter964)
 -      for iter978 in self.success:
 -        oprot.writeString(iter978)
++      for iter985 in self.success:
++        oprot.writeString(iter985)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -21015,10 -21080,10 +21073,10 @@@ class get_table_objects_by_name_args
        elif fid == 2:
          if ftype == TType.LIST:
            self.tbl_names = []
-           (_etype968, _size965) = iprot.readListBegin()
-           for _i969 in xrange(_size965):
-             _elem970 = iprot.readString()
-             self.tbl_names.append(_elem970)
 -          (_etype982, _size979) = iprot.readListBegin()
 -          for _i983 in xrange(_size979):
 -            _elem984 = iprot.readString()
 -            self.tbl_names.append(_elem984)
++          (_etype989, _size986) = iprot.readListBegin()
++          for _i990 in xrange(_size986):
++            _elem991 = iprot.readString()
++            self.tbl_names.append(_elem991)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21039,8 -21104,8 +21097,8 @@@
      if self.tbl_names is not None:
        oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
        oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-       for iter971 in self.tbl_names:
-         oprot.writeString(iter971)
 -      for iter985 in self.tbl_names:
 -        oprot.writeString(iter985)
++      for iter992 in self.tbl_names:
++        oprot.writeString(iter992)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -21092,11 -21157,11 +21150,11 @@@ class get_table_objects_by_name_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype975, _size972) = iprot.readListBegin()
-           for _i976 in xrange(_size972):
-             _elem977 = Table()
-             _elem977.read(iprot)
-             self.success.append(_elem977)
 -          (_etype989, _size986) = iprot.readListBegin()
 -          for _i990 in xrange(_size986):
 -            _elem991 = Table()
 -            _elem991.read(iprot)
 -            self.success.append(_elem991)
++          (_etype996, _size993) = iprot.readListBegin()
++          for _i997 in xrange(_size993):
++            _elem998 = Table()
++            _elem998.read(iprot)
++            self.success.append(_elem998)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21113,8 -21178,8 +21171,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter978 in self.success:
-         iter978.write(oprot)
 -      for iter992 in self.success:
 -        iter992.write(oprot)
++      for iter999 in self.success:
++        iter999.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -21506,10 -21571,10 +21564,10 @@@ class get_materialization_invalidation_
        elif fid == 2:
          if ftype == TType.LIST:
            self.tbl_names = []
-           (_etype982, _size979) = iprot.readListBegin()
-           for _i983 in xrange(_size979):
-             _elem984 = iprot.readString()
-             self.tbl_names.append(_elem984)
 -          (_etype996, _size993) = iprot.readListBegin()
 -          for _i997 in xrange(_size993):
 -            _elem998 = iprot.readString()
 -            self.tbl_names.append(_elem998)
++          (_etype1003, _size1000) = iprot.readListBegin()
++          for _i1004 in xrange(_size1000):
++            _elem1005 = iprot.readString()
++            self.tbl_names.append(_elem1005)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -21530,8 -21595,8 +21588,8 @@@
      if self.tbl_names is not None:
        oprot.writeFieldBegin('tbl_names', TType.LIST, 2)
        oprot.writeListBegin(TType.STRING, len(self.tbl_names))
-       for iter985 in self.tbl_names:
-         oprot.writeString(iter985)
 -      for iter999 in self.tbl_names:
 -        oprot.writeString(iter999)
++      for iter1006 in self.tbl_names:
++        oprot.writeString(iter1006)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -21592,12 -21657,12 +21650,12 @@@ class get_materialization_invalidation_
        if fid == 0:
          if ftype == TType.MAP:
            self.success = {}
-           (_ktype987, _vtype988, _size986 ) = iprot.readMapBegin()
-           for _i990 in xrange(_size986):
-             _key991 = iprot.readString()
-             _val992 = Materialization()
-             _val992.read(iprot)
-             self.success[_key991] = _val992
 -          (_ktype1001, _vtype1002, _size1000 ) = iprot.readMapBegin()
 -          for _i1004 in xrange(_size1000):
 -            _key1005 = iprot.readString()
 -            _val1006 = Materialization()
 -            _val1006.read(iprot)
 -            self.success[_key1005] = _val1006
++          (_ktype1008, _vtype1009, _size1007 ) = iprot.readMapBegin()
++          for _i1011 in xrange(_size1007):
++            _key1012 = iprot.readString()
++            _val1013 = Materialization()
++            _val1013.read(iprot)
++            self.success[_key1012] = _val1013
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -21632,9 -21697,9 +21690,9 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.MAP, 0)
        oprot.writeMapBegin(TType.STRING, TType.STRUCT, len(self.success))
-       for kiter993,viter994 in self.success.items():
-         oprot.writeString(kiter993)
-         viter994.write(oprot)
 -      for kiter1007,viter1008 in self.success.items():
 -        oprot.writeString(kiter1007)
 -        viter1008.write(oprot)
++      for kiter1014,viter1015 in self.success.items():
++        oprot.writeString(kiter1014)
++        viter1015.write(oprot)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -21999,10 -22064,10 +22057,10 @@@ class get_table_names_by_filter_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype998, _size995) = iprot.readListBegin()
-           for _i999 in xrange(_size995):
-             _elem1000 = iprot.readString()
-             self.success.append(_elem1000)
 -          (_etype1012, _size1009) = iprot.readListBegin()
 -          for _i1013 in xrange(_size1009):
 -            _elem1014 = iprot.readString()
 -            self.success.append(_elem1014)
++          (_etype1019, _size1016) = iprot.readListBegin()
++          for _i1020 in xrange(_size1016):
++            _elem1021 = iprot.readString()
++            self.success.append(_elem1021)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -22037,8 -22102,8 +22095,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter1001 in self.success:
-         oprot.writeString(iter1001)
 -      for iter1015 in self.success:
 -        oprot.writeString(iter1015)
++      for iter1022 in self.success:
++        oprot.writeString(iter1022)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -23008,11 -23073,11 +23066,11 @@@ class add_partitions_args
        if fid == 1:
          if ftype == TType.LIST:
            self.new_parts = []
-           (_etype1005, _size1002) = iprot.readListBegin()
-           for _i1006 in xrange(_size1002):
-             _elem1007 = Partition()
-             _elem1007.read(iprot)
-             self.new_parts.append(_elem1007)
 -          (_etype1019, _size1016) = iprot.readListBegin()
 -          for _i1020 in xrange(_size1016):
 -            _elem1021 = Partition()
 -            _elem1021.read(iprot)
 -            self.new_parts.append(_elem1021)
++          (_etype1026, _size1023) = iprot.readListBegin()
++          for _i1027 in xrange(_size1023):
++            _elem1028 = Partition()
++            _elem1028.read(iprot)
++            self.new_parts.append(_elem1028)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23029,8 -23094,8 +23087,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 1)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-       for iter1008 in self.new_parts:
-         iter1008.write(oprot)
 -      for iter1022 in self.new_parts:
 -        iter1022.write(oprot)
++      for iter1029 in self.new_parts:
++        iter1029.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -23188,11 -23253,11 +23246,11 @@@ class add_partitions_pspec_args
        if fid == 1:
          if ftype == TType.LIST:
            self.new_parts = []
-           (_etype1012, _size1009) = iprot.readListBegin()
-           for _i1013 in xrange(_size1009):
-             _elem1014 = PartitionSpec()
-             _elem1014.read(iprot)
-             self.new_parts.append(_elem1014)
 -          (_etype1026, _size1023) = iprot.readListBegin()
 -          for _i1027 in xrange(_size1023):
 -            _elem1028 = PartitionSpec()
 -            _elem1028.read(iprot)
 -            self.new_parts.append(_elem1028)
++          (_etype1033, _size1030) = iprot.readListBegin()
++          for _i1034 in xrange(_size1030):
++            _elem1035 = PartitionSpec()
++            _elem1035.read(iprot)
++            self.new_parts.append(_elem1035)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23209,8 -23274,8 +23267,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 1)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-       for iter1015 in self.new_parts:
-         iter1015.write(oprot)
 -      for iter1029 in self.new_parts:
 -        iter1029.write(oprot)
++      for iter1036 in self.new_parts:
++        iter1036.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -23384,10 -23449,10 +23442,10 @@@ class append_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1019, _size1016) = iprot.readListBegin()
-           for _i1020 in xrange(_size1016):
-             _elem1021 = iprot.readString()
-             self.part_vals.append(_elem1021)
 -          (_etype1033, _size1030) = iprot.readListBegin()
 -          for _i1034 in xrange(_size1030):
 -            _elem1035 = iprot.readString()
 -            self.part_vals.append(_elem1035)
++          (_etype1040, _size1037) = iprot.readListBegin()
++          for _i1041 in xrange(_size1037):
++            _elem1042 = iprot.readString()
++            self.part_vals.append(_elem1042)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23412,8 -23477,8 +23470,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1022 in self.part_vals:
-         oprot.writeString(iter1022)
 -      for iter1036 in self.part_vals:
 -        oprot.writeString(iter1036)
++      for iter1043 in self.part_vals:
++        oprot.writeString(iter1043)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -23766,10 -23831,10 +23824,10 @@@ class append_partition_with_environment
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1026, _size1023) = iprot.readListBegin()
-           for _i1027 in xrange(_size1023):
-             _elem1028 = iprot.readString()
-             self.part_vals.append(_elem1028)
 -          (_etype1040, _size1037) = iprot.readListBegin()
 -          for _i1041 in xrange(_size1037):
 -            _elem1042 = iprot.readString()
 -            self.part_vals.append(_elem1042)
++          (_etype1047, _size1044) = iprot.readListBegin()
++          for _i1048 in xrange(_size1044):
++            _elem1049 = iprot.readString()
++            self.part_vals.append(_elem1049)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -23800,8 -23865,8 +23858,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1029 in self.part_vals:
-         oprot.writeString(iter1029)
 -      for iter1043 in self.part_vals:
 -        oprot.writeString(iter1043)
++      for iter1050 in self.part_vals:
++        oprot.writeString(iter1050)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.environment_context is not None:
@@@ -24396,10 -24461,10 +24454,10 @@@ class drop_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1033, _size1030) = iprot.readListBegin()
-           for _i1034 in xrange(_size1030):
-             _elem1035 = iprot.readString()
-             self.part_vals.append(_elem1035)
 -          (_etype1047, _size1044) = iprot.readListBegin()
 -          for _i1048 in xrange(_size1044):
 -            _elem1049 = iprot.readString()
 -            self.part_vals.append(_elem1049)
++          (_etype1054, _size1051) = iprot.readListBegin()
++          for _i1055 in xrange(_size1051):
++            _elem1056 = iprot.readString()
++            self.part_vals.append(_elem1056)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -24429,8 -24494,8 +24487,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1036 in self.part_vals:
-         oprot.writeString(iter1036)
 -      for iter1050 in self.part_vals:
 -        oprot.writeString(iter1050)
++      for iter1057 in self.part_vals:
++        oprot.writeString(iter1057)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.deleteData is not None:
@@@ -24603,10 -24668,10 +24661,10 @@@ class drop_partition_with_environment_c
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1040, _size1037) = iprot.readListBegin()
-           for _i1041 in xrange(_size1037):
-             _elem1042 = iprot.readString()
-             self.part_vals.append(_elem1042)
 -          (_etype1054, _size1051) = iprot.readListBegin()
 -          for _i1055 in xrange(_size1051):
 -            _elem1056 = iprot.readString()
 -            self.part_vals.append(_elem1056)
++          (_etype1061, _size1058) = iprot.readListBegin()
++          for _i1062 in xrange(_size1058):
++            _elem1063 = iprot.readString()
++            self.part_vals.append(_elem1063)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -24642,8 -24707,8 +24700,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1043 in self.part_vals:
-         oprot.writeString(iter1043)
 -      for iter1057 in self.part_vals:
 -        oprot.writeString(iter1057)
++      for iter1064 in self.part_vals:
++        oprot.writeString(iter1064)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.deleteData is not None:
@@@ -25380,10 -25445,10 +25438,10 @@@ class get_partition_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1047, _size1044) = iprot.readListBegin()
-           for _i1048 in xrange(_size1044):
-             _elem1049 = iprot.readString()
-             self.part_vals.append(_elem1049)
 -          (_etype1061, _size1058) = iprot.readListBegin()
 -          for _i1062 in xrange(_size1058):
 -            _elem1063 = iprot.readString()
 -            self.part_vals.append(_elem1063)
++          (_etype1068, _size1065) = iprot.readListBegin()
++          for _i1069 in xrange(_size1065):
++            _elem1070 = iprot.readString()
++            self.part_vals.append(_elem1070)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -25408,8 -25473,8 +25466,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1050 in self.part_vals:
-         oprot.writeString(iter1050)
 -      for iter1064 in self.part_vals:
 -        oprot.writeString(iter1064)
++      for iter1071 in self.part_vals:
++        oprot.writeString(iter1071)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -25568,11 -25633,11 +25626,11 @@@ class exchange_partition_args
        if fid == 1:
          if ftype == TType.MAP:
            self.partitionSpecs = {}
-           (_ktype1052, _vtype1053, _size1051 ) = iprot.readMapBegin()
-           for _i1055 in xrange(_size1051):
-             _key1056 = iprot.readString()
-             _val1057 = iprot.readString()
-             self.partitionSpecs[_key1056] = _val1057
 -          (_ktype1066, _vtype1067, _size1065 ) = iprot.readMapBegin()
 -          for _i1069 in xrange(_size1065):
 -            _key1070 = iprot.readString()
 -            _val1071 = iprot.readString()
 -            self.partitionSpecs[_key1070] = _val1071
++          (_ktype1073, _vtype1074, _size1072 ) = iprot.readMapBegin()
++          for _i1076 in xrange(_size1072):
++            _key1077 = iprot.readString()
++            _val1078 = iprot.readString()
++            self.partitionSpecs[_key1077] = _val1078
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -25609,9 -25674,9 +25667,9 @@@
      if self.partitionSpecs is not None:
        oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-       for kiter1058,viter1059 in self.partitionSpecs.items():
-         oprot.writeString(kiter1058)
-         oprot.writeString(viter1059)
 -      for kiter1072,viter1073 in self.partitionSpecs.items():
 -        oprot.writeString(kiter1072)
 -        oprot.writeString(viter1073)
++      for kiter1079,viter1080 in self.partitionSpecs.items():
++        oprot.writeString(kiter1079)
++        oprot.writeString(viter1080)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.source_db is not None:
@@@ -25816,11 -25881,11 +25874,11 @@@ class exchange_partitions_args
        if fid == 1:
          if ftype == TType.MAP:
            self.partitionSpecs = {}
-           (_ktype1061, _vtype1062, _size1060 ) = iprot.readMapBegin()
-           for _i1064 in xrange(_size1060):
-             _key1065 = iprot.readString()
-             _val1066 = iprot.readString()
-             self.partitionSpecs[_key1065] = _val1066
 -          (_ktype1075, _vtype1076, _size1074 ) = iprot.readMapBegin()
 -          for _i1078 in xrange(_size1074):
 -            _key1079 = iprot.readString()
 -            _val1080 = iprot.readString()
 -            self.partitionSpecs[_key1079] = _val1080
++          (_ktype1082, _vtype1083, _size1081 ) = iprot.readMapBegin()
++          for _i1085 in xrange(_size1081):
++            _key1086 = iprot.readString()
++            _val1087 = iprot.readString()
++            self.partitionSpecs[_key1086] = _val1087
            iprot.readMapEnd()
          else:
            iprot.skip(ftype)
@@@ -25857,9 -25922,9 +25915,9 @@@
      if self.partitionSpecs is not None:
        oprot.writeFieldBegin('partitionSpecs', TType.MAP, 1)
        oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.partitionSpecs))
-       for kiter1067,viter1068 in self.partitionSpecs.items():
-         oprot.writeString(kiter1067)
-         oprot.writeString(viter1068)
 -      for kiter1081,viter1082 in self.partitionSpecs.items():
 -        oprot.writeString(kiter1081)
 -        oprot.writeString(viter1082)
++      for kiter1088,viter1089 in self.partitionSpecs.items():
++        oprot.writeString(kiter1088)
++        oprot.writeString(viter1089)
        oprot.writeMapEnd()
        oprot.writeFieldEnd()
      if self.source_db is not None:
@@@ -25942,11 -26007,11 +26000,11 @@@ class exchange_partitions_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1072, _size1069) = iprot.readListBegin()
-           for _i1073 in xrange(_size1069):
-             _elem1074 = Partition()
-             _elem1074.read(iprot)
-             self.success.append(_elem1074)
 -          (_etype1086, _size1083) = iprot.readListBegin()
 -          for _i1087 in xrange(_size1083):
 -            _elem1088 = Partition()
 -            _elem1088.read(iprot)
 -            self.success.append(_elem1088)
++          (_etype1093, _size1090) = iprot.readListBegin()
++          for _i1094 in xrange(_size1090):
++            _elem1095 = Partition()
++            _elem1095.read(iprot)
++            self.success.append(_elem1095)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -25987,8 -26052,8 +26045,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1075 in self.success:
-         iter1075.write(oprot)
 -      for iter1089 in self.success:
 -        iter1089.write(oprot)
++      for iter1096 in self.success:
++        iter1096.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -26082,10 -26147,10 +26140,10 @@@ class get_partition_with_auth_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1079, _size1076) = iprot.readListBegin()
-           for _i1080 in xrange(_size1076):
-             _elem1081 = iprot.readString()
-             self.part_vals.append(_elem1081)
 -          (_etype1093, _size1090) = iprot.readListBegin()
 -          for _i1094 in xrange(_size1090):
 -            _elem1095 = iprot.readString()
 -            self.part_vals.append(_elem1095)
++          (_etype1100, _size1097) = iprot.readListBegin()
++          for _i1101 in xrange(_size1097):
++            _elem1102 = iprot.readString()
++            self.part_vals.append(_elem1102)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -26097,10 -26162,10 +26155,10 @@@
        elif fid == 5:
          if ftype == TType.LIST:
            self.group_names = []
-           (_etype1085, _size1082) = iprot.readListBegin()
-           for _i1086 in xrange(_size1082):
-             _elem1087 = iprot.readString()
-             self.group_names.append(_elem1087)
 -          (_etype1099, _size1096) = iprot.readListBegin()
 -          for _i1100 in xrange(_size1096):
 -            _elem1101 = iprot.readString()
 -            self.group_names.append(_elem1101)
++          (_etype1106, _size1103) = iprot.readListBegin()
++          for _i1107 in xrange(_size1103):
++            _elem1108 = iprot.readString()
++            self.group_names.append(_elem1108)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -26125,8 -26190,8 +26183,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1088 in self.part_vals:
-         oprot.writeString(iter1088)
 -      for iter1102 in self.part_vals:
 -        oprot.writeString(iter1102)
++      for iter1109 in self.part_vals:
++        oprot.writeString(iter1109)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.user_name is not None:
@@@ -26136,8 -26201,8 +26194,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 5)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
-       for iter1089 in self.group_names:
-         oprot.writeString(iter1089)
 -      for iter1103 in self.group_names:
 -        oprot.writeString(iter1103)
++      for iter1110 in self.group_names:
++        oprot.writeString(iter1110)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -26566,11 -26631,11 +26624,11 @@@ class get_partitions_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1093, _size1090) = iprot.readListBegin()
-           for _i1094 in xrange(_size1090):
-             _elem1095 = Partition()
-             _elem1095.read(iprot)
-             self.success.append(_elem1095)
 -          (_etype1107, _size1104) = iprot.readListBegin()
 -          for _i1108 in xrange(_size1104):
 -            _elem1109 = Partition()
 -            _elem1109.read(iprot)
 -            self.success.append(_elem1109)
++          (_etype1114, _size1111) = iprot.readListBegin()
++          for _i1115 in xrange(_size1111):
++            _elem1116 = Partition()
++            _elem1116.read(iprot)
++            self.success.append(_elem1116)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -26599,8 -26664,8 +26657,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1096 in self.success:
-         iter1096.write(oprot)
 -      for iter1110 in self.success:
 -        iter1110.write(oprot)
++      for iter1117 in self.success:
++        iter1117.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -26694,10 -26759,10 +26752,10 @@@ class get_partitions_with_auth_args
        elif fid == 5:
          if ftype == TType.LIST:
            self.group_names = []
-           (_etype1100, _size1097) = iprot.readListBegin()
-           for _i1101 in xrange(_size1097):
-             _elem1102 = iprot.readString()
-             self.group_names.append(_elem1102)
 -          (_etype1114, _size1111) = iprot.readListBegin()
 -          for _i1115 in xrange(_size1111):
 -            _elem1116 = iprot.readString()
 -            self.group_names.append(_elem1116)
++          (_etype1121, _size1118) = iprot.readListBegin()
++          for _i1122 in xrange(_size1118):
++            _elem1123 = iprot.readString()
++            self.group_names.append(_elem1123)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -26730,8 -26795,8 +26788,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 5)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
-       for iter1103 in self.group_names:
-         oprot.writeString(iter1103)
 -      for iter1117 in self.group_names:
 -        oprot.writeString(iter1117)
++      for iter1124 in self.group_names:
++        oprot.writeString(iter1124)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -26792,11 -26857,11 +26850,11 @@@ class get_partitions_with_auth_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1107, _size1104) = iprot.readListBegin()
-           for _i1108 in xrange(_size1104):
-             _elem1109 = Partition()
-             _elem1109.read(iprot)
-             self.success.append(_elem1109)
 -          (_etype1121, _size1118) = iprot.readListBegin()
 -          for _i1122 in xrange(_size1118):
 -            _elem1123 = Partition()
 -            _elem1123.read(iprot)
 -            self.success.append(_elem1123)
++          (_etype1128, _size1125) = iprot.readListBegin()
++          for _i1129 in xrange(_size1125):
++            _elem1130 = Partition()
++            _elem1130.read(iprot)
++            self.success.append(_elem1130)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -26825,8 -26890,8 +26883,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1110 in self.success:
-         iter1110.write(oprot)
 -      for iter1124 in self.success:
 -        iter1124.write(oprot)
++      for iter1131 in self.success:
++        iter1131.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -26984,11 -27049,11 +27042,11 @@@ class get_partitions_pspec_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1114, _size1111) = iprot.readListBegin()
-           for _i1115 in xrange(_size1111):
-             _elem1116 = PartitionSpec()
-             _elem1116.read(iprot)
-             self.success.append(_elem1116)
 -          (_etype1128, _size1125) = iprot.readListBegin()
 -          for _i1129 in xrange(_size1125):
 -            _elem1130 = PartitionSpec()
 -            _elem1130.read(iprot)
 -            self.success.append(_elem1130)
++          (_etype1135, _size1132) = iprot.readListBegin()
++          for _i1136 in xrange(_size1132):
++            _elem1137 = PartitionSpec()
++            _elem1137.read(iprot)
++            self.success.append(_elem1137)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27017,8 -27082,8 +27075,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1117 in self.success:
-         iter1117.write(oprot)
 -      for iter1131 in self.success:
 -        iter1131.write(oprot)
++      for iter1138 in self.success:
++        iter1138.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -27176,10 -27241,10 +27234,10 @@@ class get_partition_names_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1121, _size1118) = iprot.readListBegin()
-           for _i1122 in xrange(_size1118):
-             _elem1123 = iprot.readString()
-             self.success.append(_elem1123)
 -          (_etype1135, _size1132) = iprot.readListBegin()
 -          for _i1136 in xrange(_size1132):
 -            _elem1137 = iprot.readString()
 -            self.success.append(_elem1137)
++          (_etype1142, _size1139) = iprot.readListBegin()
++          for _i1143 in xrange(_size1139):
++            _elem1144 = iprot.readString()
++            self.success.append(_elem1144)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27208,8 -27273,8 +27266,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter1124 in self.success:
-         oprot.writeString(iter1124)
 -      for iter1138 in self.success:
 -        oprot.writeString(iter1138)
++      for iter1145 in self.success:
++        oprot.writeString(iter1145)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -27449,10 -27514,10 +27507,10 @@@ class get_partitions_ps_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1128, _size1125) = iprot.readListBegin()
-           for _i1129 in xrange(_size1125):
-             _elem1130 = iprot.readString()
-             self.part_vals.append(_elem1130)
 -          (_etype1142, _size1139) = iprot.readListBegin()
 -          for _i1143 in xrange(_size1139):
 -            _elem1144 = iprot.readString()
 -            self.part_vals.append(_elem1144)
++          (_etype1149, _size1146) = iprot.readListBegin()
++          for _i1150 in xrange(_size1146):
++            _elem1151 = iprot.readString()
++            self.part_vals.append(_elem1151)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27482,8 -27547,8 +27540,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1131 in self.part_vals:
-         oprot.writeString(iter1131)
 -      for iter1145 in self.part_vals:
 -        oprot.writeString(iter1145)
++      for iter1152 in self.part_vals:
++        oprot.writeString(iter1152)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.max_parts is not None:
@@@ -27547,11 -27612,11 +27605,11 @@@ class get_partitions_ps_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1135, _size1132) = iprot.readListBegin()
-           for _i1136 in xrange(_size1132):
-             _elem1137 = Partition()
-             _elem1137.read(iprot)
-             self.success.append(_elem1137)
 -          (_etype1149, _size1146) = iprot.readListBegin()
 -          for _i1150 in xrange(_size1146):
 -            _elem1151 = Partition()
 -            _elem1151.read(iprot)
 -            self.success.append(_elem1151)
++          (_etype1156, _size1153) = iprot.readListBegin()
++          for _i1157 in xrange(_size1153):
++            _elem1158 = Partition()
++            _elem1158.read(iprot)
++            self.success.append(_elem1158)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27580,8 -27645,8 +27638,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1138 in self.success:
-         iter1138.write(oprot)
 -      for iter1152 in self.success:
 -        iter1152.write(oprot)
++      for iter1159 in self.success:
++        iter1159.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -27668,10 -27733,10 +27726,10 @@@ class get_partitions_ps_with_auth_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1142, _size1139) = iprot.readListBegin()
-           for _i1143 in xrange(_size1139):
-             _elem1144 = iprot.readString()
-             self.part_vals.append(_elem1144)
 -          (_etype1156, _size1153) = iprot.readListBegin()
 -          for _i1157 in xrange(_size1153):
 -            _elem1158 = iprot.readString()
 -            self.part_vals.append(_elem1158)
++          (_etype1163, _size1160) = iprot.readListBegin()
++          for _i1164 in xrange(_size1160):
++            _elem1165 = iprot.readString()
++            self.part_vals.append(_elem1165)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27688,10 -27753,10 +27746,10 @@@
        elif fid == 6:
          if ftype == TType.LIST:
            self.group_names = []
-           (_etype1148, _size1145) = iprot.readListBegin()
-           for _i1149 in xrange(_size1145):
-             _elem1150 = iprot.readString()
-             self.group_names.append(_elem1150)
 -          (_etype1162, _size1159) = iprot.readListBegin()
 -          for _i1163 in xrange(_size1159):
 -            _elem1164 = iprot.readString()
 -            self.group_names.append(_elem1164)
++          (_etype1169, _size1166) = iprot.readListBegin()
++          for _i1170 in xrange(_size1166):
++            _elem1171 = iprot.readString()
++            self.group_names.append(_elem1171)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27716,8 -27781,8 +27774,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1151 in self.part_vals:
-         oprot.writeString(iter1151)
 -      for iter1165 in self.part_vals:
 -        oprot.writeString(iter1165)
++      for iter1172 in self.part_vals:
++        oprot.writeString(iter1172)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.max_parts is not None:
@@@ -27731,8 -27796,8 +27789,8 @@@
      if self.group_names is not None:
        oprot.writeFieldBegin('group_names', TType.LIST, 6)
        oprot.writeListBegin(TType.STRING, len(self.group_names))
-       for iter1152 in self.group_names:
-         oprot.writeString(iter1152)
 -      for iter1166 in self.group_names:
 -        oprot.writeString(iter1166)
++      for iter1173 in self.group_names:
++        oprot.writeString(iter1173)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -27794,11 -27859,11 +27852,11 @@@ class get_partitions_ps_with_auth_resul
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1156, _size1153) = iprot.readListBegin()
-           for _i1157 in xrange(_size1153):
-             _elem1158 = Partition()
-             _elem1158.read(iprot)
-             self.success.append(_elem1158)
 -          (_etype1170, _size1167) = iprot.readListBegin()
 -          for _i1171 in xrange(_size1167):
 -            _elem1172 = Partition()
 -            _elem1172.read(iprot)
 -            self.success.append(_elem1172)
++          (_etype1177, _size1174) = iprot.readListBegin()
++          for _i1178 in xrange(_size1174):
++            _elem1179 = Partition()
++            _elem1179.read(iprot)
++            self.success.append(_elem1179)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27827,8 -27892,8 +27885,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1159 in self.success:
-         iter1159.write(oprot)
 -      for iter1173 in self.success:
 -        iter1173.write(oprot)
++      for iter1180 in self.success:
++        iter1180.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -27909,10 -27974,10 +27967,10 @@@ class get_partition_names_ps_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.part_vals = []
-           (_etype1163, _size1160) = iprot.readListBegin()
-           for _i1164 in xrange(_size1160):
-             _elem1165 = iprot.readString()
-             self.part_vals.append(_elem1165)
 -          (_etype1177, _size1174) = iprot.readListBegin()
 -          for _i1178 in xrange(_size1174):
 -            _elem1179 = iprot.readString()
 -            self.part_vals.append(_elem1179)
++          (_etype1184, _size1181) = iprot.readListBegin()
++          for _i1185 in xrange(_size1181):
++            _elem1186 = iprot.readString()
++            self.part_vals.append(_elem1186)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -27942,8 -28007,8 +28000,8 @@@
      if self.part_vals is not None:
        oprot.writeFieldBegin('part_vals', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.part_vals))
-       for iter1166 in self.part_vals:
-         oprot.writeString(iter1166)
 -      for iter1180 in self.part_vals:
 -        oprot.writeString(iter1180)
++      for iter1187 in self.part_vals:
++        oprot.writeString(iter1187)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.max_parts is not None:
@@@ -28007,10 -28072,10 +28065,10 @@@ class get_partition_names_ps_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1170, _size1167) = iprot.readListBegin()
-           for _i1171 in xrange(_size1167):
-             _elem1172 = iprot.readString()
-             self.success.append(_elem1172)
 -          (_etype1184, _size1181) = iprot.readListBegin()
 -          for _i1185 in xrange(_size1181):
 -            _elem1186 = iprot.readString()
 -            self.success.append(_elem1186)
++          (_etype1191, _size1188) = iprot.readListBegin()
++          for _i1192 in xrange(_size1188):
++            _elem1193 = iprot.readString()
++            self.success.append(_elem1193)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -28039,8 -28104,8 +28097,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRING, len(self.success))
-       for iter1173 in self.success:
-         oprot.writeString(iter1173)
 -      for iter1187 in self.success:
 -        oprot.writeString(iter1187)
++      for iter1194 in self.success:
++        oprot.writeString(iter1194)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -28211,11 -28276,11 +28269,11 @@@ class get_partitions_by_filter_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1177, _size1174) = iprot.readListBegin()
-           for _i1178 in xrange(_size1174):
-             _elem1179 = Partition()
-             _elem1179.read(iprot)
-             self.success.append(_elem1179)
 -          (_etype1191, _size1188) = iprot.readListBegin()
 -          for _i1192 in xrange(_size1188):
 -            _elem1193 = Partition()
 -            _elem1193.read(iprot)
 -            self.success.append(_elem1193)
++          (_etype1198, _size1195) = iprot.readListBegin()
++          for _i1199 in xrange(_size1195):
++            _elem1200 = Partition()
++            _elem1200.read(iprot)
++            self.success.append(_elem1200)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -28244,8 -28309,8 +28302,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1180 in self.success:
-         iter1180.write(oprot)
 -      for iter1194 in self.success:
 -        iter1194.write(oprot)
++      for iter1201 in self.success:
++        iter1201.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -28416,11 -28481,11 +28474,11 @@@ class get_part_specs_by_filter_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1184, _size1181) = iprot.readListBegin()
-           for _i1185 in xrange(_size1181):
-             _elem1186 = PartitionSpec()
-             _elem1186.read(iprot)
-             self.success.append(_elem1186)
 -          (_etype1198, _size1195) = iprot.readListBegin()
 -          for _i1199 in xrange(_size1195):
 -            _elem1200 = PartitionSpec()
 -            _elem1200.read(iprot)
 -            self.success.append(_elem1200)
++          (_etype1205, _size1202) = iprot.readListBegin()
++          for _i1206 in xrange(_size1202):
++            _elem1207 = PartitionSpec()
++            _elem1207.read(iprot)
++            self.success.append(_elem1207)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -28449,8 -28514,8 +28507,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1187 in self.success:
-         iter1187.write(oprot)
 -      for iter1201 in self.success:
 -        iter1201.write(oprot)
++      for iter1208 in self.success:
++        iter1208.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -28870,10 -28935,10 +28928,10 @@@ class get_partitions_by_names_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.names = []
-           (_etype1191, _size1188) = iprot.readListBegin()
-           for _i1192 in xrange(_size1188):
-             _elem1193 = iprot.readString()
-             self.names.append(_elem1193)
 -          (_etype1205, _size1202) = iprot.readListBegin()
 -          for _i1206 in xrange(_size1202):
 -            _elem1207 = iprot.readString()
 -            self.names.append(_elem1207)
++          (_etype1212, _size1209) = iprot.readListBegin()
++          for _i1213 in xrange(_size1209):
++            _elem1214 = iprot.readString()
++            self.names.append(_elem1214)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -28898,8 -28963,8 +28956,8 @@@
      if self.names is not None:
        oprot.writeFieldBegin('names', TType.LIST, 3)
        oprot.writeListBegin(TType.STRING, len(self.names))
-       for iter1194 in self.names:
-         oprot.writeString(iter1194)
 -      for iter1208 in self.names:
 -        oprot.writeString(iter1208)
++      for iter1215 in self.names:
++        oprot.writeString(iter1215)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()
@@@ -28958,11 -29023,11 +29016,11 @@@ class get_partitions_by_names_result
        if fid == 0:
          if ftype == TType.LIST:
            self.success = []
-           (_etype1198, _size1195) = iprot.readListBegin()
-           for _i1199 in xrange(_size1195):
-             _elem1200 = Partition()
-             _elem1200.read(iprot)
-             self.success.append(_elem1200)
 -          (_etype1212, _size1209) = iprot.readListBegin()
 -          for _i1213 in xrange(_size1209):
 -            _elem1214 = Partition()
 -            _elem1214.read(iprot)
 -            self.success.append(_elem1214)
++          (_etype1219, _size1216) = iprot.readListBegin()
++          for _i1220 in xrange(_size1216):
++            _elem1221 = Partition()
++            _elem1221.read(iprot)
++            self.success.append(_elem1221)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -28991,8 -29056,8 +29049,8 @@@
      if self.success is not None:
        oprot.writeFieldBegin('success', TType.LIST, 0)
        oprot.writeListBegin(TType.STRUCT, len(self.success))
-       for iter1201 in self.success:
-         iter1201.write(oprot)
 -      for iter1215 in self.success:
 -        iter1215.write(oprot)
++      for iter1222 in self.success:
++        iter1222.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      if self.o1 is not None:
@@@ -29242,11 -29307,11 +29300,11 @@@ class alter_partitions_args
        elif fid == 3:
          if ftype == TType.LIST:
            self.new_parts = []
-           (_etype1205, _size1202) = iprot.readListBegin()
-           for _i1206 in xrange(_size1202):
-             _elem1207 = Partition()
-             _elem1207.read(iprot)
-             self.new_parts.append(_elem1207)
 -          (_etype1219, _size1216) = iprot.readListBegin()
 -          for _i1220 in xrange(_size1216):
 -            _elem1221 = Partition()
 -            _elem1221.read(iprot)
 -            self.new_parts.append(_elem1221)
++          (_etype1226, _size1223) = iprot.readListBegin()
++          for _i1227 in xrange(_size1223):
++            _elem1228 = Partition()
++            _elem1228.read(iprot)
++            self.new_parts.append(_elem1228)
            iprot.readListEnd()
          else:
            iprot.skip(ftype)
@@@ -29271,8 -29336,8 +29329,8 @@@
      if self.new_parts is not None:
        oprot.writeFieldBegin('new_parts', TType.LIST, 3)
        oprot.writeListBegin(TType.STRUCT, len(self.new_parts))
-       for iter1208 in self.new_parts:
-         iter1208.write(oprot)
 -      for iter1222 in self.new_parts:
 -        iter1222.write(oprot)
++      for iter1229 in self.new_parts:
++        iter1229.write(oprot)
        oprot.writeListEnd()
        oprot.writeFieldEnd()
      oprot.writeFieldStop()


[11/46] hive git commit: HIVE-19711 Refactor Hive Schema Tool (Miklos Gergely via Alan Gates).

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
deleted file mode 100644
index 3b22f15..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java
+++ /dev/null
@@ -1,801 +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.hive.beeline;
-
-import java.io.BufferedWriter;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.net.URI;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.SQLException;
-import java.util.Random;
-
-import junit.framework.TestCase;
-
-import org.apache.commons.dbcp.DelegatingConnection;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.IMetaStoreSchemaInfo;
-import org.apache.hadoop.hive.metastore.MetaStoreSchemaInfoFactory;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.NestedScriptParser;
-import org.apache.hadoop.hive.metastore.tools.HiveSchemaHelper.PostgresCommandParser;
-import org.apache.hadoop.hive.shims.ShimLoader;
-
-public class TestSchemaTool extends TestCase {
-  private HiveSchemaTool schemaTool;
-  private Connection conn;
-  private HiveConf hiveConf;
-  private String testMetastoreDB;
-  private PrintStream errStream;
-  private PrintStream outStream;
-
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    testMetastoreDB = System.getProperty("java.io.tmpdir") +
-        File.separator + "test_metastore-" + new Random().nextInt();
-    System.setProperty(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname,
-        "jdbc:derby:" + testMetastoreDB + ";create=true");
-    hiveConf = new HiveConf(this.getClass());
-    schemaTool = new HiveSchemaTool(
-        System.getProperty("test.tmp.dir", "target/tmp"), hiveConf, "derby", null);
-    schemaTool.setUserName(
-        schemaTool.getHiveConf().get(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME.varname));
-    schemaTool.setPassWord(ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(),
-          HiveConf.ConfVars.METASTOREPWD.varname));
-    System.setProperty("beeLine.system.exit", "true");
-    errStream = System.err;
-    outStream = System.out;
-    conn = schemaTool.getConnectionToMetastore(false);
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    File metaStoreDir = new File(testMetastoreDB);
-    if (metaStoreDir.exists()) {
-      FileUtils.forceDeleteOnExit(metaStoreDir);
-    }
-    System.setOut(outStream);
-    System.setErr(errStream);
-    if (conn != null) {
-      conn.close();
-    }
-  }
-
-  /**
-   * Test the sequence validation functionality
-   * @throws Exception
-   */
-  public void testValidateSequences() throws Exception {
-    schemaTool.doInit();
-
-    // Test empty database
-    boolean isValid = schemaTool.validateSequences(conn);
-    assertTrue(isValid);
-
-    // Test valid case
-    String[] scripts = new String[] {
-        "insert into CTLGS values(99, 'test_cat_1', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
-        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
-        "insert into DBS values(99, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1')"
-    };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateSequences(conn);
-    assertTrue(isValid);
-
-    // Test invalid case
-    scripts = new String[] {
-        "delete from SEQUENCE_TABLE",
-        "delete from DBS",
-        "insert into SEQUENCE_TABLE values('org.apache.hadoop.hive.metastore.model.MDatabase', 100)",
-        "insert into DBS values(102, 'test db1', 'hdfs:///tmp', 'db1', 'test', 'test', 'test_cat_1')"
-    };
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateSequences(conn);
-    assertFalse(isValid);
-  }
-
-  /**
-   * Test to validate that all tables exist in the HMS metastore.
-   * @throws Exception
-   */
-  public void testValidateSchemaTables() throws Exception {
-    schemaTool.doInit("2.0.0");
-
-    boolean isValid = (boolean)schemaTool.validateSchemaTables(conn);
-    assertTrue(isValid);
-
-    // upgrade from 2.0.0 schema and re-validate
-    schemaTool.doUpgrade("2.0.0");
-    isValid = (boolean)schemaTool.validateSchemaTables(conn);
-    assertTrue(isValid);
-
-    // Simulate a missing table scenario by renaming a couple of tables
-    String[] scripts = new String[] {
-        "RENAME TABLE SEQUENCE_TABLE to SEQUENCE_TABLE_RENAMED",
-        "RENAME TABLE NUCLEUS_TABLES to NUCLEUS_TABLES_RENAMED"
-    };
-
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateSchemaTables(conn);
-    assertFalse(isValid);
-
-    // Restored the renamed tables
-    scripts = new String[] {
-        "RENAME TABLE SEQUENCE_TABLE_RENAMED to SEQUENCE_TABLE",
-        "RENAME TABLE NUCLEUS_TABLES_RENAMED to NUCLEUS_TABLES"
-    };
-
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateSchemaTables(conn);
-    assertTrue(isValid);
-
-    // Check that an exception from getMetaData() is reported correctly
-    try {
-      // Make a Connection object that will throw an exception
-      BadMetaDataConnection bad = new BadMetaDataConnection(conn);
-      schemaTool.validateSchemaTables(bad);
-      fail("did not get expected exception");
-    } catch (HiveMetaException hme) {
-      String message = hme.getMessage();
-      assertTrue("Bad HiveMetaException message :" + message,
-          message.contains("Failed to retrieve schema tables from Hive Metastore DB"));
-      Throwable cause = hme.getCause();
-      assertNotNull("HiveMetaException did not contain a cause", cause);
-      String causeMessage = cause.getMessage();
-      assertTrue("Bad SQLException message: " + causeMessage, causeMessage.contains(
-          BadMetaDataConnection.FAILURE_TEXT));
-    }
-  }
-
-  /*
-   * Test the validation of incorrect NULL values in the tables
-   * @throws Exception
-   */
-  public void testValidateNullValues() throws Exception {
-    schemaTool.doInit();
-
-    // Test empty database
-    boolean isValid = schemaTool.validateColumnNullValues(conn);
-    assertTrue(isValid);
-
-    // Test valid case
-    createTestHiveTableSchemas();
-    isValid = schemaTool.validateColumnNullValues(conn);
-
-    // Test invalid case
-    String[] scripts = new String[] {
-        "update TBLS set SD_ID=null"
-    };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateColumnNullValues(conn);
-    assertFalse(isValid);
-  }
-
-  /**
-   * Test dryrun of schema initialization
-   * @throws Exception
-   */
-  public void testSchemaInitDryRun() throws Exception {
-    schemaTool.setDryRun(true);
-    schemaTool.doInit("0.7.0");
-    schemaTool.setDryRun(false);
-    try {
-      schemaTool.verifySchemaVersion();
-    } catch (HiveMetaException e) {
-      // The connection should fail since it the dry run
-      return;
-    }
-    fail("Dry run shouldn't create actual metastore");
-  }
-
-  /**
-   * Test dryrun of schema upgrade
-   * @throws Exception
-   */
-  public void testSchemaUpgradeDryRun() throws Exception {
-    schemaTool.doInit("0.7.0");
-
-    schemaTool.setDryRun(true);
-    schemaTool.doUpgrade("0.7.0");
-    schemaTool.setDryRun(false);
-    try {
-      schemaTool.verifySchemaVersion();
-    } catch (HiveMetaException e) {
-      // The connection should fail since it the dry run
-      return;
-    }
-    fail("Dry run shouldn't upgrade metastore schema");
-  }
-
-  /**
-   * Test schema initialization
-   * @throws Exception
-   */
-  public void testSchemaInit() throws Exception {
-    IMetaStoreSchemaInfo metastoreSchemaInfo = MetaStoreSchemaInfoFactory.get(hiveConf,
-        System.getProperty("test.tmp.dir", "target/tmp"), "derby");
-    schemaTool.doInit(metastoreSchemaInfo.getHiveSchemaVersion());
-    schemaTool.verifySchemaVersion();
-  }
-
-  /**
-  * Test validation for schema versions
-  * @throws Exception
-  */
- public void testValidateSchemaVersions() throws Exception {
-   schemaTool.doInit();
-   boolean isValid = schemaTool.validateSchemaVersions();
-   // Test an invalid case with multiple versions
-   String[] scripts = new String[] {
-       "insert into VERSION values(100, '2.2.0', 'Hive release version 2.2.0')"
-   };
-   File scriptFile = generateTestScript(scripts);
-   schemaTool.runBeeLine(scriptFile.getPath());
-   isValid = schemaTool.validateSchemaVersions();
-   assertFalse(isValid);
-
-   scripts = new String[] {
-       "delete from VERSION where VER_ID = 100"
-   };
-   scriptFile = generateTestScript(scripts);
-   schemaTool.runBeeLine(scriptFile.getPath());
-   isValid = schemaTool.validateSchemaVersions();
-   assertTrue(isValid);
-
-   // Test an invalid case without version
-   scripts = new String[] {
-       "delete from VERSION"
-   };
-   scriptFile = generateTestScript(scripts);
-   schemaTool.runBeeLine(scriptFile.getPath());
-   isValid = schemaTool.validateSchemaVersions();
-   assertFalse(isValid);
- }
-
-  /**
-   * Test schema upgrade
-   * @throws Exception
-   */
-  public void testSchemaUpgrade() throws Exception {
-    boolean foundException = false;
-    // Initialize 0.7.0 schema
-    schemaTool.doInit("0.7.0");
-    // verify that driver fails due to older version schema
-    try {
-      schemaTool.verifySchemaVersion();
-    } catch (HiveMetaException e) {
-      // Expected to fail due to old schema
-      foundException = true;
-    }
-    if (!foundException) {
-      throw new Exception(
-          "Hive operations shouldn't pass with older version schema");
-    }
-
-    // Generate dummy pre-upgrade script with errors
-    String invalidPreUpgradeScript = writeDummyPreUpgradeScript(
-        0, "upgrade-0.11.0-to-0.12.0.derby.sql", "foo bar;");
-    // Generate dummy pre-upgrade scripts with valid SQL
-    String validPreUpgradeScript0 = writeDummyPreUpgradeScript(
-        0, "upgrade-0.12.0-to-0.13.0.derby.sql",
-        "CREATE TABLE schema_test0 (id integer);");
-    String validPreUpgradeScript1 = writeDummyPreUpgradeScript(
-        1, "upgrade-0.12.0-to-0.13.0.derby.sql",
-        "CREATE TABLE schema_test1 (id integer);");
-
-    // Capture system out and err
-    schemaTool.setVerbose(true);
-    OutputStream stderr = new ByteArrayOutputStream();
-    PrintStream errPrintStream = new PrintStream(stderr);
-    System.setErr(errPrintStream);
-    OutputStream stdout = new ByteArrayOutputStream();
-    PrintStream outPrintStream = new PrintStream(stdout);
-    System.setOut(outPrintStream);
-
-    // Upgrade schema from 0.7.0 to latest
-    schemaTool.doUpgrade("0.7.0");
-
-    // Verify that the schemaTool ran pre-upgrade scripts and ignored errors
-    assertTrue(stderr.toString().contains(invalidPreUpgradeScript));
-    assertTrue(stderr.toString().contains("foo"));
-    assertFalse(stderr.toString().contains(validPreUpgradeScript0));
-    assertFalse(stderr.toString().contains(validPreUpgradeScript1));
-    assertTrue(stdout.toString().contains(validPreUpgradeScript0));
-    assertTrue(stdout.toString().contains(validPreUpgradeScript1));
-
-    // Verify that driver works fine with latest schema
-    schemaTool.verifySchemaVersion();
-  }
-
-  /**
-   * Test script formatting
-   * @throws Exception
-   */
-  public void testScripts() throws Exception {
-    String testScript[] = {
-        "-- this is a comment",
-      "DROP TABLE IF EXISTS fooTab;",
-      "/*!1234 this is comment code like mysql */;",
-      "CREATE TABLE fooTab(id INTEGER);",
-      "DROP TABLE footab;",
-      "-- ending comment"
-    };
-    String resultScript[] = {
-      "DROP TABLE IF EXISTS fooTab",
-      "/*!1234 this is comment code like mysql */",
-      "CREATE TABLE fooTab(id INTEGER)",
-      "DROP TABLE footab",
-    };
-    String expectedSQL = StringUtils.join(resultScript, System.getProperty("line.separator")) +
-        System.getProperty("line.separator");
-    File testScriptFile = generateTestScript(testScript);
-    String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby", false)
-        .buildCommand(testScriptFile.getParentFile().getPath(),
-            testScriptFile.getName());
-
-    assertEquals(expectedSQL, flattenedSql);
-  }
-
-  /**
-   * Test nested script formatting
-   * @throws Exception
-   */
-  public void testNestedScriptsForDerby() throws Exception {
-    String childTab1 = "childTab1";
-    String childTab2 = "childTab2";
-    String parentTab = "fooTab";
-
-    String childTestScript1[] = {
-      "-- this is a comment ",
-      "DROP TABLE IF EXISTS " + childTab1 + ";",
-      "CREATE TABLE " + childTab1 + "(id INTEGER);",
-      "DROP TABLE " + childTab1 + ";"
-    };
-    String childTestScript2[] = {
-        "-- this is a comment",
-        "DROP TABLE IF EXISTS " + childTab2 + ";",
-        "CREATE TABLE " + childTab2 + "(id INTEGER);",
-        "-- this is also a comment",
-        "DROP TABLE " + childTab2 + ";"
-    };
-
-    String parentTestScript[] = {
-        " -- this is a comment",
-        "DROP TABLE IF EXISTS " + parentTab + ";",
-        " -- this is another comment ",
-        "CREATE TABLE " + parentTab + "(id INTEGER);",
-        "RUN '" + generateTestScript(childTestScript1).getName() + "';",
-        "DROP TABLE " + parentTab + ";",
-        "RUN '" + generateTestScript(childTestScript2).getName() + "';",
-        "--ending comment ",
-      };
-
-    File testScriptFile = generateTestScript(parentTestScript);
-    String flattenedSql = HiveSchemaHelper.getDbCommandParser("derby", false)
-        .buildCommand(testScriptFile.getParentFile().getPath(),
-            testScriptFile.getName());
-    assertFalse(flattenedSql.contains("RUN"));
-    assertFalse(flattenedSql.contains("comment"));
-    assertTrue(flattenedSql.contains(childTab1));
-    assertTrue(flattenedSql.contains(childTab2));
-    assertTrue(flattenedSql.contains(parentTab));
-  }
-
-  /**
-   * Test nested script formatting
-   * @throws Exception
-   */
-  public void testNestedScriptsForMySQL() throws Exception {
-    String childTab1 = "childTab1";
-    String childTab2 = "childTab2";
-    String parentTab = "fooTab";
-
-    String childTestScript1[] = {
-      "/* this is a comment code */",
-      "DROP TABLE IF EXISTS " + childTab1 + ";",
-      "CREATE TABLE " + childTab1 + "(id INTEGER);",
-      "DROP TABLE " + childTab1 + ";"
-    };
-    String childTestScript2[] = {
-        "/* this is a special exec code */;",
-        "DROP TABLE IF EXISTS " + childTab2 + ";",
-        "CREATE TABLE " + childTab2 + "(id INTEGER);",
-        "-- this is a comment",
-        "DROP TABLE " + childTab2 + ";"
-    };
-
-    String parentTestScript[] = {
-        " -- this is a comment",
-        "DROP TABLE IF EXISTS " + parentTab + ";",
-        " /* this is special exec code */;",
-        "CREATE TABLE " + parentTab + "(id INTEGER);",
-        "SOURCE " + generateTestScript(childTestScript1).getName() + ";",
-        "DROP TABLE " + parentTab + ";",
-        "SOURCE " + generateTestScript(childTestScript2).getName() + ";",
-        "--ending comment ",
-      };
-
-    File testScriptFile = generateTestScript(parentTestScript);
-    String flattenedSql = HiveSchemaHelper.getDbCommandParser("mysql", false)
-        .buildCommand(testScriptFile.getParentFile().getPath(),
-            testScriptFile.getName());
-    assertFalse(flattenedSql.contains("RUN"));
-    assertFalse(flattenedSql.contains("comment"));
-    assertTrue(flattenedSql.contains(childTab1));
-    assertTrue(flattenedSql.contains(childTab2));
-    assertTrue(flattenedSql.contains(parentTab));
-  }
-
-  /**
-   * Test script formatting
-   * @throws Exception
-   */
-  public void testScriptWithDelimiter() throws Exception {
-    String testScript[] = {
-        "-- this is a comment",
-      "DROP TABLE IF EXISTS fooTab;",
-      "DELIMITER $$",
-      "/*!1234 this is comment code like mysql */$$",
-      "CREATE TABLE fooTab(id INTEGER)$$",
-      "CREATE PROCEDURE fooProc()",
-      "SELECT * FROM fooTab;",
-      "CALL barProc();",
-      "END PROCEDURE$$",
-      "DELIMITER ;",
-      "DROP TABLE footab;",
-      "-- ending comment"
-    };
-    String resultScript[] = {
-      "DROP TABLE IF EXISTS fooTab",
-      "/*!1234 this is comment code like mysql */",
-      "CREATE TABLE fooTab(id INTEGER)",
-      "CREATE PROCEDURE fooProc()" + " " +
-      "SELECT * FROM fooTab;" + " " +
-      "CALL barProc();" + " " +
-      "END PROCEDURE",
-      "DROP TABLE footab",
-    };
-    String expectedSQL = StringUtils.join(resultScript, System.getProperty("line.separator")) +
-        System.getProperty("line.separator");
-    File testScriptFile = generateTestScript(testScript);
-    NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql", false);
-    String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
-        testScriptFile.getName());
-
-    assertEquals(expectedSQL, flattenedSql);
-  }
-
-  /**
-   * Test script formatting
-   * @throws Exception
-   */
-  public void testScriptMultiRowComment() throws Exception {
-    String testScript[] = {
-        "-- this is a comment",
-      "DROP TABLE IF EXISTS fooTab;",
-      "DELIMITER $$",
-      "/*!1234 this is comment code like mysql */$$",
-      "CREATE TABLE fooTab(id INTEGER)$$",
-      "DELIMITER ;",
-      "/* multiline comment started ",
-      " * multiline comment continue",
-      " * multiline comment ended */",
-      "DROP TABLE footab;",
-      "-- ending comment"
-    };
-    String parsedScript[] = {
-      "DROP TABLE IF EXISTS fooTab",
-      "/*!1234 this is comment code like mysql */",
-      "CREATE TABLE fooTab(id INTEGER)",
-      "DROP TABLE footab",
-    };
-
-    String expectedSQL = StringUtils.join(parsedScript, System.getProperty("line.separator")) +
-        System.getProperty("line.separator");
-    File testScriptFile = generateTestScript(testScript);
-    NestedScriptParser testDbParser = HiveSchemaHelper.getDbCommandParser("mysql", false);
-    String flattenedSql = testDbParser.buildCommand(testScriptFile.getParentFile().getPath(),
-        testScriptFile.getName());
-
-    assertEquals(expectedSQL, flattenedSql);
-  }
-
-  /**
-   * Test nested script formatting
-   * @throws Exception
-   */
-  public void testNestedScriptsForOracle() throws Exception {
-    String childTab1 = "childTab1";
-    String childTab2 = "childTab2";
-    String parentTab = "fooTab";
-
-    String childTestScript1[] = {
-      "-- this is a comment ",
-      "DROP TABLE IF EXISTS " + childTab1 + ";",
-      "CREATE TABLE " + childTab1 + "(id INTEGER);",
-      "DROP TABLE " + childTab1 + ";"
-    };
-    String childTestScript2[] = {
-        "-- this is a comment",
-        "DROP TABLE IF EXISTS " + childTab2 + ";",
-        "CREATE TABLE " + childTab2 + "(id INTEGER);",
-        "-- this is also a comment",
-        "DROP TABLE " + childTab2 + ";"
-    };
-
-    String parentTestScript[] = {
-        " -- this is a comment",
-        "DROP TABLE IF EXISTS " + parentTab + ";",
-        " -- this is another comment ",
-        "CREATE TABLE " + parentTab + "(id INTEGER);",
-        "@" + generateTestScript(childTestScript1).getName() + ";",
-        "DROP TABLE " + parentTab + ";",
-        "@" + generateTestScript(childTestScript2).getName() + ";",
-        "--ending comment ",
-      };
-
-    File testScriptFile = generateTestScript(parentTestScript);
-    String flattenedSql = HiveSchemaHelper.getDbCommandParser("oracle", false)
-        .buildCommand(testScriptFile.getParentFile().getPath(),
-            testScriptFile.getName());
-    assertFalse(flattenedSql.contains("@"));
-    assertFalse(flattenedSql.contains("comment"));
-    assertTrue(flattenedSql.contains(childTab1));
-    assertTrue(flattenedSql.contains(childTab2));
-    assertTrue(flattenedSql.contains(parentTab));
-  }
-
-  /**
-   * Test script formatting
-   * @throws Exception
-   */
-  public void testPostgresFilter() throws Exception {
-    String testScript[] = {
-        "-- this is a comment",
-        "DROP TABLE IF EXISTS fooTab;",
-        HiveSchemaHelper.PostgresCommandParser.POSTGRES_STANDARD_STRINGS_OPT + ";",
-        "CREATE TABLE fooTab(id INTEGER);",
-        "DROP TABLE footab;",
-        "-- ending comment"
-    };
-
-    String expectedScriptWithOptionPresent[] = {
-        "DROP TABLE IF EXISTS fooTab",
-        HiveSchemaHelper.PostgresCommandParser.POSTGRES_STANDARD_STRINGS_OPT,
-        "CREATE TABLE fooTab(id INTEGER)",
-        "DROP TABLE footab",
-    };
-
-    NestedScriptParser noDbOptParser = HiveSchemaHelper
-        .getDbCommandParser("postgres", false);
-    String expectedSQL = StringUtils.join(
-        expectedScriptWithOptionPresent, System.getProperty("line.separator")) +
-            System.getProperty("line.separator");
-    File testScriptFile = generateTestScript(testScript);
-    String flattenedSql = noDbOptParser.buildCommand(
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
-    assertEquals(expectedSQL, flattenedSql);
-
-    String expectedScriptWithOptionAbsent[] = {
-        "DROP TABLE IF EXISTS fooTab",
-        "CREATE TABLE fooTab(id INTEGER)",
-        "DROP TABLE footab",
-    };
-
-    NestedScriptParser dbOptParser = HiveSchemaHelper.getDbCommandParser(
-        "postgres",
-        PostgresCommandParser.POSTGRES_SKIP_STANDARD_STRINGS_DBOPT,
-        null, null, null, null, false);
-    expectedSQL = StringUtils.join(
-        expectedScriptWithOptionAbsent, System.getProperty("line.separator")) +
-            System.getProperty("line.separator");
-    testScriptFile = generateTestScript(testScript);
-    flattenedSql = dbOptParser.buildCommand(
-        testScriptFile.getParentFile().getPath(), testScriptFile.getName());
-    assertEquals(expectedSQL, flattenedSql);
-  }
-
-  /**
-   * Test validate uri of locations
-   * @throws Exception
-   */
-  public void testValidateLocations() throws Exception {
-    schemaTool.doInit();
-    URI defaultRoot = new URI("hdfs://myhost.com:8020");
-    URI defaultRoot2 = new URI("s3://myhost2.com:8888");
-    //check empty DB
-    boolean isValid = schemaTool.validateLocations(conn, null);
-    assertTrue(isValid);
-    isValid = schemaTool.validateLocations(conn, new URI[] {defaultRoot,defaultRoot2});
-    assertTrue(isValid);
-
- // Test valid case
-    String[] scripts = new String[] {
-         "insert into CTLGS values(3, 'test_cat_2', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
-         "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2')",
-         "insert into DBS values(7, 'db with bad port', 'hdfs://myhost.com:8020/', 'haDB', 'public', 'role', 'test_cat_2')",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3,null,'org.apache.hadoop.mapred.TextInputFormat','N','N',null,-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,3,'myView','VIRTUAL_VIEW','select a.col1,a.col2 from foo','select * from foo','n')",
-         "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,7,0 ,'hive',0,4000,'mytal4012','MANAGED_TABLE',NULL,NULL,'n')",
-         "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
-         "insert into SKEWED_STRING_LIST values(1)",
-         "insert into SKEWED_STRING_LIST values(2)",
-         "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')",
-         "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'s3://myhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')"
-       };
-    File scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateLocations(conn, null);
-    assertTrue(isValid);
-    isValid = schemaTool.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
-    assertTrue(isValid);
-    scripts = new String[] {
-        "delete from SKEWED_COL_VALUE_LOC_MAP",
-        "delete from SKEWED_STRING_LIST",
-        "delete from PARTITIONS",
-        "delete from TBLS",
-        "delete from SDS",
-        "delete from DBS",
-        "insert into DBS values(2, 'my db', '/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'test_cat_2')",
-        "insert into DBS values(4, 'my db2', 'hdfs://myhost.com:8020', '', 'public', 'role', 'test_cat_2')",
-        "insert into DBS values(6, 'db with bad port', 'hdfs://myhost.com:8020:', 'zDB', 'public', 'role', 'test_cat_2')",
-        "insert into DBS values(7, 'db with bad port', 'hdfs://mynameservice.com/', 'haDB', 'public', 'role', 'test_cat_2')",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (3000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','yourhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4001,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4003,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4004,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (4002,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (5000,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','file:///user/admin/2016_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3000 ,1435255431,2,0 ,'hive',0,3000,'mytal3000','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4011 ,1435255431,4,0 ,'hive',0,4001,'mytal4011','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4012 ,1435255431,4,0 ,'hive',0,4002,'','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4013 ,1435255431,4,0 ,'hive',0,4003,'mytal4013','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (4014 ,1435255431,2,0 ,'hive',0,4003,'','MANAGED_TABLE',NULL,NULL,'n')",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4001, 1441402388,0, 'd1=1/d2=4001',4001,4011)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4002, 1441402388,0, 'd1=1/d2=4002',4002,4012)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4003, 1441402388,0, 'd1=1/d2=4003',4003,4013)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(4004, 1441402388,0, 'd1=1/d2=4004',4004,4014)",
-        "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(5000, 1441402388,0, 'd1=1/d2=5000',5000,2)",
-        "insert into SKEWED_STRING_LIST values(1)",
-        "insert into SKEWED_STRING_LIST values(2)",
-        "insert into SKEWED_COL_VALUE_LOC_MAP values(1,1,'hdfs://yourhost.com:8020/user/hive/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')",
-        "insert into SKEWED_COL_VALUE_LOC_MAP values(2,2,'file:///user/admin/warehouse/mytal/HIVE_DEFAULT_LIST_BUCKETING_DIR_NAME/')"
-    };
-    scriptFile = generateTestScript(scripts);
-    schemaTool.runBeeLine(scriptFile.getPath());
-    isValid = schemaTool.validateLocations(conn, null);
-    assertFalse(isValid);
-    isValid = schemaTool.validateLocations(conn, new URI[] {defaultRoot, defaultRoot2});
-    assertFalse(isValid);
-  }
-
-  public void testHiveMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
-    schemaTool.doInit("3.0.0");
-    validateMetastoreDbPropertiesTable();
-  }
-
-  public void testMetastoreDbPropertiesAfterUpgrade() throws HiveMetaException, IOException {
-    schemaTool.doInit("2.0.0");
-    schemaTool.doUpgrade();
-    validateMetastoreDbPropertiesTable();
-  }
-
-  private File generateTestScript(String [] stmts) throws IOException {
-    File testScriptFile = File.createTempFile("schematest", ".sql");
-    testScriptFile.deleteOnExit();
-    FileWriter fstream = new FileWriter(testScriptFile.getPath());
-    BufferedWriter out = new BufferedWriter(fstream);
-    for (String line: stmts) {
-      out.write(line);
-      out.newLine();
-    }
-    out.close();
-    return testScriptFile;
-  }
-
-  private void validateMetastoreDbPropertiesTable() throws HiveMetaException, IOException {
-    boolean isValid = (boolean) schemaTool.validateSchemaTables(conn);
-    assertTrue(isValid);
-    // adding same property key twice should throw unique key constraint violation exception
-    String[] scripts = new String[] {
-        "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-1', 'dummy uuid 1')",
-        "insert into METASTORE_DB_PROPERTIES values ('guid', 'test-uuid-2', 'dummy uuid 2')", };
-    File scriptFile = generateTestScript(scripts);
-    Exception ex = null;
-    try {
-      schemaTool.runBeeLine(scriptFile.getPath());
-    } catch (Exception iox) {
-      ex = iox;
-    }
-    assertTrue(ex != null && ex instanceof IOException);
-  }
-  /**
-   * Write out a dummy pre-upgrade script with given SQL statement.
-   */
-  private String writeDummyPreUpgradeScript(int index, String upgradeScriptName,
-      String sql) throws Exception {
-    String preUpgradeScript = "pre-" + index + "-" + upgradeScriptName;
-    String dummyPreScriptPath = System.getProperty("test.tmp.dir", "target/tmp") +
-        File.separatorChar + "scripts" + File.separatorChar + "metastore" +
-        File.separatorChar + "upgrade" + File.separatorChar + "derby" +
-        File.separatorChar + preUpgradeScript;
-    FileWriter fstream = new FileWriter(dummyPreScriptPath);
-    BufferedWriter out = new BufferedWriter(fstream);
-    out.write(sql + System.getProperty("line.separator") + ";");
-    out.close();
-    return preUpgradeScript;
-  }
-
-  /**
-   * Insert the records in DB to simulate a hive table
-   * @throws IOException
-   */
-  private void createTestHiveTableSchemas() throws IOException {
-     String[] scripts = new String[] {
-          "insert into CTLGS values(2, 'my_catalog', 'description', 'hdfs://myhost.com:8020/user/hive/warehouse/mydb')",
-          "insert into DBS values(2, 'my db', 'hdfs://myhost.com:8021/user/hive/warehouse/mydb', 'mydb', 'public', 'role', 'my_catalog')",
-          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (1,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/hive/warehouse/mydb',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-          "insert into SDS(SD_ID,CD_ID,INPUT_FORMAT,IS_COMPRESSED,IS_STOREDASSUBDIRECTORIES,LOCATION,NUM_BUCKETS,OUTPUT_FORMAT,SERDE_ID) values (2,null,'org.apache.hadoop.mapred.TextInputFormat','N','N','hdfs://myhost.com:8020/user/admin/2015_11_18',-1,'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat',null)",
-          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (2 ,1435255431,2,0 ,'hive',0,1,'mytal','MANAGED_TABLE',NULL,NULL,'n')",
-          "insert into TBLS(TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT,IS_REWRITE_ENABLED) values (3 ,1435255431,2,0 ,'hive',0,2,'aTable','MANAGED_TABLE',NULL,NULL,'n')",
-          "insert into PARTITIONS(PART_ID,CREATE_TIME,LAST_ACCESS_TIME, PART_NAME,SD_ID,TBL_ID) values(1, 1441402388,0, 'd1=1/d2=1',2,2)"
-        };
-     File scriptFile = generateTestScript(scripts);
-     schemaTool.runBeeLine(scriptFile.getPath());
-  }
-
-  /**
-   * A mock Connection class that throws an exception out of getMetaData().
-   */
-  class BadMetaDataConnection extends DelegatingConnection {
-    static final String FAILURE_TEXT = "fault injected";
-
-    BadMetaDataConnection(Connection connection) {
-      super(connection);
-    }
-
-    @Override
-    public DatabaseMetaData getMetaData() throws SQLException {
-      throw new SQLException(FAILURE_TEXT);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/d83a0be9/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java
deleted file mode 100644
index f5bc570..0000000
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaToolCatalogOps.java
+++ /dev/null
@@ -1,417 +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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * 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.beeline;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaException;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
-import org.apache.hadoop.hive.metastore.api.Catalog;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.client.builder.CatalogBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.DatabaseBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.FunctionBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.PartitionBuilder;
-import org.apache.hadoop.hive.metastore.client.builder.TableBuilder;
-import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
-import org.apache.thrift.TException;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
-import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
-
-public class TestSchemaToolCatalogOps {
-  private static final Logger LOG = LoggerFactory.getLogger(TestSchemaToolCatalogOps.class);
-  private static HiveSchemaTool schemaTool;
-  private static HiveConf conf;
-  private IMetaStoreClient client;
-  private static String testMetastoreDB;
-
-  @BeforeClass
-  public static void initDb() throws HiveMetaException, IOException {
-    conf = new HiveConf();
-    MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, false);
-    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.HMS_HANDLER_ATTEMPTS, 1);
-    MetastoreConf.setLongVar(conf, MetastoreConf.ConfVars.THRIFT_CONNECTION_RETRIES, 1);
-    testMetastoreDB = System.getProperty("java.io.tmpdir") +
-        File.separator + "testschematoolcatopsdb";
-    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.CONNECT_URL_KEY,
-        "jdbc:derby:" + testMetastoreDB + ";create=true");
-    schemaTool = new HiveSchemaTool(
-        System.getProperty("test.tmp.dir", "target/tmp"), conf, "derby", null);
-    schemaTool.setUserName(MetastoreConf.getVar(conf, MetastoreConf.ConfVars.CONNECTION_USER_NAME));
-    schemaTool.setPassWord(MetastoreConf.getPassword(conf, MetastoreConf.ConfVars.PWD));
-    schemaTool.doInit();  // Pre-install the database so all the tables are there.
-
-  }
-
-  @AfterClass
-  public static void removeDb() throws Exception {
-    File metaStoreDir = new File(testMetastoreDB);
-    if (metaStoreDir.exists()) {
-      FileUtils.forceDeleteOnExit(metaStoreDir);
-    }
-  }
-
-  @Before
-  public void createClient() throws MetaException {
-    client = new HiveMetaStoreClient(conf);
-  }
-
-  @Test
-  public void createCatalog() throws HiveMetaException, TException {
-    String catName = "my_test_catalog";
-    String location = "file:///tmp/my_test_catalog";
-    String description = "very descriptive";
-    schemaTool.createCatalog(catName, location, description, false);
-
-    Catalog cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void createExistingCatalog() throws HiveMetaException {
-    schemaTool.createCatalog("hive", "somewhere", "", false);
-  }
-
-  @Test
-  public void createExistingCatalogWithIfNotExists() throws HiveMetaException {
-    String catName = "my_existing_test_catalog";
-    String location = "file:///tmp/my_test_catalog";
-    String description = "very descriptive";
-    schemaTool.createCatalog(catName, location, description, false);
-
-    schemaTool.createCatalog(catName, location, description, true);
-  }
-
-  @Test
-  public void alterCatalog() throws HiveMetaException, TException {
-    String catName = "an_alterable_catalog";
-    String location = "file:///tmp/an_alterable_catalog";
-    String description = "description";
-    schemaTool.createCatalog(catName, location, description, false);
-
-    location = "file:///tmp/somewhere_else";
-    schemaTool.alterCatalog(catName, location, null);
-    Catalog cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-
-    description = "a better description";
-    schemaTool.alterCatalog(catName, null, description);
-    cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-
-    location = "file:///tmp/a_third_location";
-    description = "best description yet";
-    schemaTool.alterCatalog(catName, location, description);
-    cat = client.getCatalog(catName);
-    Assert.assertEquals(location, cat.getLocationUri());
-    Assert.assertEquals(description, cat.getDescription());
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void alterBogusCatalog() throws HiveMetaException {
-    schemaTool.alterCatalog("nosuch", "file:///tmp/somewhere", "whatever");
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void alterCatalogNoChange() throws HiveMetaException {
-    String catName = "alter_cat_no_change";
-    String location = "file:///tmp/alter_cat_no_change";
-    String description = "description";
-    schemaTool.createCatalog(catName, location, description, false);
-
-    schemaTool.alterCatalog(catName, null, null);
-  }
-
-  @Test
-  public void moveDatabase() throws HiveMetaException, TException {
-    String toCatName = "moveDbCat";
-    String dbName = "moveDbDb";
-    String tableName = "moveDbTable";
-    String funcName = "movedbfunc";
-    String partVal = "moveDbKey";
-
-    new CatalogBuilder()
-        .setName(toCatName)
-        .setLocation("file:///tmp")
-        .create(client);
-
-    Database db = new DatabaseBuilder()
-        .setCatalogName(DEFAULT_CATALOG_NAME)
-        .setName(dbName)
-        .create(client, conf);
-
-    new FunctionBuilder()
-        .inDb(db)
-        .setName(funcName)
-        .setClass("org.apache.hive.myudf")
-        .create(client, conf);
-
-    Table table = new TableBuilder()
-        .inDb(db)
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .addPartCol("p", "string")
-        .create(client, conf);
-
-    new PartitionBuilder()
-        .inTable(table)
-        .addValue(partVal)
-        .addToTable(client, conf);
-
-    schemaTool.moveDatabase(DEFAULT_CATALOG_NAME, toCatName, dbName);
-
-    Database fetchedDb = client.getDatabase(toCatName, dbName);
-    Assert.assertNotNull(fetchedDb);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedDb.getCatalogName());
-
-    Function fetchedFunction = client.getFunction(toCatName, dbName, funcName);
-    Assert.assertNotNull(fetchedFunction);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedFunction.getCatName());
-    Assert.assertEquals(dbName.toLowerCase(), fetchedFunction.getDbName());
-
-    Table fetchedTable = client.getTable(toCatName, dbName, tableName);
-    Assert.assertNotNull(fetchedTable);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
-    Assert.assertEquals(dbName.toLowerCase(), fetchedTable.getDbName());
-
-    Partition fetchedPart =
-        client.getPartition(toCatName, dbName, tableName, Collections.singletonList(partVal));
-    Assert.assertNotNull(fetchedPart);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
-    Assert.assertEquals(dbName.toLowerCase(), fetchedPart.getDbName());
-    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
-  }
-
-  @Test
-  public void moveDatabaseWithExistingDbOfSameNameAlreadyInTargetCatalog()
-      throws TException, HiveMetaException {
-    String catName = "clobberCatalog";
-    new CatalogBuilder()
-        .setName(catName)
-        .setLocation("file:///tmp")
-        .create(client);
-    try {
-      schemaTool.moveDatabase(catName, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME);
-      Assert.fail("Attempt to move default database should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
-    Assert.assertTrue(dbNames.contains(DEFAULT_DATABASE_NAME));
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void moveNonExistentDatabase() throws TException, HiveMetaException {
-    String catName = "moveNonExistentDb";
-    new CatalogBuilder()
-        .setName(catName)
-        .setLocation("file:///tmp")
-        .create(client);
-    schemaTool.moveDatabase(catName, DEFAULT_CATALOG_NAME, "nosuch");
-  }
-
-  @Test
-  public void moveDbToNonExistentCatalog() throws TException, HiveMetaException {
-    String dbName = "doomedToHomelessness";
-    new DatabaseBuilder()
-        .setName(dbName)
-        .create(client, conf);
-    try {
-      schemaTool.moveDatabase(DEFAULT_CATALOG_NAME, "nosuch", dbName);
-      Assert.fail("Attempt to move database to non-existent catalog should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> dbNames = new HashSet<>(client.getAllDatabases(DEFAULT_CATALOG_NAME));
-    Assert.assertTrue(dbNames.contains(dbName.toLowerCase()));
-  }
-
-  @Test
-  public void moveTable() throws TException, HiveMetaException {
-    String toCatName = "moveTableCat";
-    String toDbName = "moveTableDb";
-    String tableName = "moveTableTable";
-    String partVal = "moveTableKey";
-
-    new CatalogBuilder()
-        .setName(toCatName)
-        .setLocation("file:///tmp")
-        .create(client);
-
-    new DatabaseBuilder()
-        .setCatalogName(toCatName)
-        .setName(toDbName)
-        .create(client, conf);
-
-    Table table = new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .addPartCol("p", "string")
-        .create(client, conf);
-
-    new PartitionBuilder()
-        .inTable(table)
-        .addValue(partVal)
-        .addToTable(client, conf);
-
-    schemaTool.moveTable(DEFAULT_CATALOG_NAME, toCatName, DEFAULT_DATABASE_NAME, toDbName, tableName);
-
-    Table fetchedTable = client.getTable(toCatName, toDbName, tableName);
-    Assert.assertNotNull(fetchedTable);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedTable.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
-
-    Partition fetchedPart =
-        client.getPartition(toCatName, toDbName, tableName, Collections.singletonList(partVal));
-    Assert.assertNotNull(fetchedPart);
-    Assert.assertEquals(toCatName.toLowerCase(), fetchedPart.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
-    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
-  }
-
-  @Test
-  public void moveTableWithinCatalog() throws TException, HiveMetaException {
-    String toDbName = "moveTableWithinCatalogDb";
-    String tableName = "moveTableWithinCatalogTable";
-    String partVal = "moveTableWithinCatalogKey";
-
-    new DatabaseBuilder()
-        .setName(toDbName)
-        .create(client, conf);
-
-    Table table = new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .addPartCol("p", "string")
-        .create(client, conf);
-
-    new PartitionBuilder()
-        .inTable(table)
-        .addValue(partVal)
-        .addToTable(client, conf);
-
-    schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName, tableName);
-
-    Table fetchedTable = client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
-    Assert.assertNotNull(fetchedTable);
-    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedTable.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedTable.getDbName());
-
-    Partition fetchedPart =
-        client.getPartition(DEFAULT_CATALOG_NAME, toDbName, tableName, Collections.singletonList(partVal));
-    Assert.assertNotNull(fetchedPart);
-    Assert.assertEquals(DEFAULT_CATALOG_NAME, fetchedPart.getCatName());
-    Assert.assertEquals(toDbName.toLowerCase(), fetchedPart.getDbName());
-    Assert.assertEquals(tableName.toLowerCase(), fetchedPart.getTableName());
-  }
-
-  @Test
-  public void moveTableWithExistingTableOfSameNameAlreadyInTargetDatabase()
-      throws TException, HiveMetaException {
-    String toDbName = "clobberTableDb";
-    String tableName = "clobberTableTable";
-
-    Database toDb = new DatabaseBuilder()
-        .setName(toDbName)
-        .create(client, conf);
-
-    new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .create(client, conf);
-
-    new TableBuilder()
-        .inDb(toDb)
-        .setTableName(tableName)
-        .addCol("b", "varchar(32)")
-        .create(client, conf);
-
-    try {
-      schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME,
-          toDbName, tableName);
-      Assert.fail("Attempt to move table should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
-    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
-
-    // Make sure the table in the target database didn't get clobbered
-    Table fetchedTable =  client.getTable(DEFAULT_CATALOG_NAME, toDbName, tableName);
-    Assert.assertEquals("b", fetchedTable.getSd().getCols().get(0).getName());
-  }
-
-  @Test(expected = HiveMetaException.class)
-  public void moveNonExistentTable() throws TException, HiveMetaException {
-    String toDbName = "moveNonExistentTable";
-    new DatabaseBuilder()
-        .setName(toDbName)
-        .create(client, conf);
-    schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME, toDbName,
-        "nosuch");
-  }
-
-  @Test
-  public void moveTableToNonExistentDb() throws TException, HiveMetaException {
-    String tableName = "doomedToWander";
-    new TableBuilder()
-        .setTableName(tableName)
-        .addCol("a", "int")
-        .create(client, conf);
-
-    try {
-      schemaTool.moveTable(DEFAULT_CATALOG_NAME, DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME,
-          "nosuch", tableName);
-      Assert.fail("Attempt to move table to non-existent table should have failed.");
-    } catch (HiveMetaException e) {
-      // good
-    }
-
-    // Make sure nothing really moved
-    Set<String> tableNames = new HashSet<>(client.getAllTables(DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME));
-    Assert.assertTrue(tableNames.contains(tableName.toLowerCase()));
-  }
-}


[28/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 5c6495e..f5913fc 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -445,6 +445,8 @@ class AbortTxnsRequest;
 
 class CommitTxnRequest;
 
+class WriteEventInfo;
+
 class ReplTblWriteIdStateRequest;
 
 class GetValidWriteIdsRequest;
@@ -517,6 +519,10 @@ class FireEventRequest;
 
 class FireEventResponse;
 
+class WriteNotificationLogRequest;
+
+class WriteNotificationLogResponse;
+
 class MetadataPpdResult;
 
 class GetFileMetadataByExprResult;
@@ -6985,8 +6991,9 @@ inline std::ostream& operator<<(std::ostream& out, const AbortTxnsRequest& obj)
 }
 
 typedef struct _CommitTxnRequest__isset {
-  _CommitTxnRequest__isset() : replPolicy(false) {}
+  _CommitTxnRequest__isset() : replPolicy(false), writeEventInfos(false) {}
   bool replPolicy :1;
+  bool writeEventInfos :1;
 } _CommitTxnRequest__isset;
 
 class CommitTxnRequest {
@@ -7000,6 +7007,7 @@ class CommitTxnRequest {
   virtual ~CommitTxnRequest() throw();
   int64_t txnid;
   std::string replPolicy;
+  std::vector<WriteEventInfo>  writeEventInfos;
 
   _CommitTxnRequest__isset __isset;
 
@@ -7007,6 +7015,8 @@ class CommitTxnRequest {
 
   void __set_replPolicy(const std::string& val);
 
+  void __set_writeEventInfos(const std::vector<WriteEventInfo> & val);
+
   bool operator == (const CommitTxnRequest & rhs) const
   {
     if (!(txnid == rhs.txnid))
@@ -7015,6 +7025,10 @@ class CommitTxnRequest {
       return false;
     else if (__isset.replPolicy && !(replPolicy == rhs.replPolicy))
       return false;
+    if (__isset.writeEventInfos != rhs.__isset.writeEventInfos)
+      return false;
+    else if (__isset.writeEventInfos && !(writeEventInfos == rhs.writeEventInfos))
+      return false;
     return true;
   }
   bool operator != (const CommitTxnRequest &rhs) const {
@@ -7037,6 +7051,90 @@ inline std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj)
   return out;
 }
 
+typedef struct _WriteEventInfo__isset {
+  _WriteEventInfo__isset() : partition(false), tableObj(false), partitionObj(false) {}
+  bool partition :1;
+  bool tableObj :1;
+  bool partitionObj :1;
+} _WriteEventInfo__isset;
+
+class WriteEventInfo {
+ public:
+
+  WriteEventInfo(const WriteEventInfo&);
+  WriteEventInfo& operator=(const WriteEventInfo&);
+  WriteEventInfo() : writeId(0), database(), table(), files(), partition(), tableObj(), partitionObj() {
+  }
+
+  virtual ~WriteEventInfo() throw();
+  int64_t writeId;
+  std::string database;
+  std::string table;
+  std::string files;
+  std::string partition;
+  std::string tableObj;
+  std::string partitionObj;
+
+  _WriteEventInfo__isset __isset;
+
+  void __set_writeId(const int64_t val);
+
+  void __set_database(const std::string& val);
+
+  void __set_table(const std::string& val);
+
+  void __set_files(const std::string& val);
+
+  void __set_partition(const std::string& val);
+
+  void __set_tableObj(const std::string& val);
+
+  void __set_partitionObj(const std::string& val);
+
+  bool operator == (const WriteEventInfo & rhs) const
+  {
+    if (!(writeId == rhs.writeId))
+      return false;
+    if (!(database == rhs.database))
+      return false;
+    if (!(table == rhs.table))
+      return false;
+    if (!(files == rhs.files))
+      return false;
+    if (__isset.partition != rhs.__isset.partition)
+      return false;
+    else if (__isset.partition && !(partition == rhs.partition))
+      return false;
+    if (__isset.tableObj != rhs.__isset.tableObj)
+      return false;
+    else if (__isset.tableObj && !(tableObj == rhs.tableObj))
+      return false;
+    if (__isset.partitionObj != rhs.__isset.partitionObj)
+      return false;
+    else if (__isset.partitionObj && !(partitionObj == rhs.partitionObj))
+      return false;
+    return true;
+  }
+  bool operator != (const WriteEventInfo &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WriteEventInfo & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(WriteEventInfo &a, WriteEventInfo &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WriteEventInfo& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _ReplTblWriteIdStateRequest__isset {
   _ReplTblWriteIdStateRequest__isset() : partNames(false) {}
   bool partNames :1;
@@ -9035,9 +9133,10 @@ inline std::ostream& operator<<(std::ostream& out, const NotificationEventsCount
 }
 
 typedef struct _InsertEventRequestData__isset {
-  _InsertEventRequestData__isset() : replace(false), filesAddedChecksum(false) {}
+  _InsertEventRequestData__isset() : replace(false), filesAddedChecksum(false), subDirectoryList(false) {}
   bool replace :1;
   bool filesAddedChecksum :1;
+  bool subDirectoryList :1;
 } _InsertEventRequestData__isset;
 
 class InsertEventRequestData {
@@ -9052,6 +9151,7 @@ class InsertEventRequestData {
   bool replace;
   std::vector<std::string>  filesAdded;
   std::vector<std::string>  filesAddedChecksum;
+  std::vector<std::string>  subDirectoryList;
 
   _InsertEventRequestData__isset __isset;
 
@@ -9061,6 +9161,8 @@ class InsertEventRequestData {
 
   void __set_filesAddedChecksum(const std::vector<std::string> & val);
 
+  void __set_subDirectoryList(const std::vector<std::string> & val);
+
   bool operator == (const InsertEventRequestData & rhs) const
   {
     if (__isset.replace != rhs.__isset.replace)
@@ -9073,6 +9175,10 @@ class InsertEventRequestData {
       return false;
     else if (__isset.filesAddedChecksum && !(filesAddedChecksum == rhs.filesAddedChecksum))
       return false;
+    if (__isset.subDirectoryList != rhs.__isset.subDirectoryList)
+      return false;
+    else if (__isset.subDirectoryList && !(subDirectoryList == rhs.subDirectoryList))
+      return false;
     return true;
   }
   bool operator != (const InsertEventRequestData &rhs) const {
@@ -9258,6 +9364,114 @@ inline std::ostream& operator<<(std::ostream& out, const FireEventResponse& obj)
   return out;
 }
 
+typedef struct _WriteNotificationLogRequest__isset {
+  _WriteNotificationLogRequest__isset() : partitionVals(false) {}
+  bool partitionVals :1;
+} _WriteNotificationLogRequest__isset;
+
+class WriteNotificationLogRequest {
+ public:
+
+  WriteNotificationLogRequest(const WriteNotificationLogRequest&);
+  WriteNotificationLogRequest& operator=(const WriteNotificationLogRequest&);
+  WriteNotificationLogRequest() : txnId(0), writeId(0), db(), table() {
+  }
+
+  virtual ~WriteNotificationLogRequest() throw();
+  int64_t txnId;
+  int64_t writeId;
+  std::string db;
+  std::string table;
+  InsertEventRequestData fileInfo;
+  std::vector<std::string>  partitionVals;
+
+  _WriteNotificationLogRequest__isset __isset;
+
+  void __set_txnId(const int64_t val);
+
+  void __set_writeId(const int64_t val);
+
+  void __set_db(const std::string& val);
+
+  void __set_table(const std::string& val);
+
+  void __set_fileInfo(const InsertEventRequestData& val);
+
+  void __set_partitionVals(const std::vector<std::string> & val);
+
+  bool operator == (const WriteNotificationLogRequest & rhs) const
+  {
+    if (!(txnId == rhs.txnId))
+      return false;
+    if (!(writeId == rhs.writeId))
+      return false;
+    if (!(db == rhs.db))
+      return false;
+    if (!(table == rhs.table))
+      return false;
+    if (!(fileInfo == rhs.fileInfo))
+      return false;
+    if (__isset.partitionVals != rhs.__isset.partitionVals)
+      return false;
+    else if (__isset.partitionVals && !(partitionVals == rhs.partitionVals))
+      return false;
+    return true;
+  }
+  bool operator != (const WriteNotificationLogRequest &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WriteNotificationLogRequest & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(WriteNotificationLogRequest &a, WriteNotificationLogRequest &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WriteNotificationLogRequest& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
+
+class WriteNotificationLogResponse {
+ public:
+
+  WriteNotificationLogResponse(const WriteNotificationLogResponse&);
+  WriteNotificationLogResponse& operator=(const WriteNotificationLogResponse&);
+  WriteNotificationLogResponse() {
+  }
+
+  virtual ~WriteNotificationLogResponse() throw();
+
+  bool operator == (const WriteNotificationLogResponse & /* rhs */) const
+  {
+    return true;
+  }
+  bool operator != (const WriteNotificationLogResponse &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const WriteNotificationLogResponse & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+  virtual void printTo(std::ostream& out) const;
+};
+
+void swap(WriteNotificationLogResponse &a, WriteNotificationLogResponse &b);
+
+inline std::ostream& operator<<(std::ostream& out, const WriteNotificationLogResponse& obj)
+{
+  obj.printTo(out);
+  return out;
+}
+
 typedef struct _MetadataPpdResult__isset {
   _MetadataPpdResult__isset() : metadata(false), includeBitset(false) {}
   bool metadata :1;

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 1dcc870..3ce72e9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -816,13 +816,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITIONNAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list700 = iprot.readListBegin();
-                struct.partitionnames = new ArrayList<String>(_list700.size);
-                String _elem701;
-                for (int _i702 = 0; _i702 < _list700.size; ++_i702)
+                org.apache.thrift.protocol.TList _list708 = iprot.readListBegin();
+                struct.partitionnames = new ArrayList<String>(_list708.size);
+                String _elem709;
+                for (int _i710 = 0; _i710 < _list708.size; ++_i710)
                 {
-                  _elem701 = iprot.readString();
-                  struct.partitionnames.add(_elem701);
+                  _elem709 = iprot.readString();
+                  struct.partitionnames.add(_elem709);
                 }
                 iprot.readListEnd();
               }
@@ -872,9 +872,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(PARTITIONNAMES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionnames.size()));
-          for (String _iter703 : struct.partitionnames)
+          for (String _iter711 : struct.partitionnames)
           {
-            oprot.writeString(_iter703);
+            oprot.writeString(_iter711);
           }
           oprot.writeListEnd();
         }
@@ -910,9 +910,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tablename);
       {
         oprot.writeI32(struct.partitionnames.size());
-        for (String _iter704 : struct.partitionnames)
+        for (String _iter712 : struct.partitionnames)
         {
-          oprot.writeString(_iter704);
+          oprot.writeString(_iter712);
         }
       }
       BitSet optionals = new BitSet();
@@ -937,13 +937,13 @@ import org.slf4j.LoggerFactory;
       struct.tablename = iprot.readString();
       struct.setTablenameIsSet(true);
       {
-        org.apache.thrift.protocol.TList _list705 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.partitionnames = new ArrayList<String>(_list705.size);
-        String _elem706;
-        for (int _i707 = 0; _i707 < _list705.size; ++_i707)
+        org.apache.thrift.protocol.TList _list713 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.partitionnames = new ArrayList<String>(_list713.size);
+        String _elem714;
+        for (int _i715 = 0; _i715 < _list713.size; ++_i715)
         {
-          _elem706 = iprot.readString();
-          struct.partitionnames.add(_elem706);
+          _elem714 = iprot.readString();
+          struct.partitionnames.add(_elem714);
         }
       }
       struct.setPartitionnamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
index fa33963..a0b47a9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsRequest.java
@@ -716,13 +716,13 @@ import org.slf4j.LoggerFactory;
           case 3: // TXN_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list626 = iprot.readListBegin();
-                struct.txnIds = new ArrayList<Long>(_list626.size);
-                long _elem627;
-                for (int _i628 = 0; _i628 < _list626.size; ++_i628)
+                org.apache.thrift.protocol.TList _list634 = iprot.readListBegin();
+                struct.txnIds = new ArrayList<Long>(_list634.size);
+                long _elem635;
+                for (int _i636 = 0; _i636 < _list634.size; ++_i636)
                 {
-                  _elem627 = iprot.readI64();
-                  struct.txnIds.add(_elem627);
+                  _elem635 = iprot.readI64();
+                  struct.txnIds.add(_elem635);
                 }
                 iprot.readListEnd();
               }
@@ -742,14 +742,14 @@ import org.slf4j.LoggerFactory;
           case 5: // SRC_TXN_TO_WRITE_ID_LIST
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list629 = iprot.readListBegin();
-                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list629.size);
-                TxnToWriteId _elem630;
-                for (int _i631 = 0; _i631 < _list629.size; ++_i631)
+                org.apache.thrift.protocol.TList _list637 = iprot.readListBegin();
+                struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list637.size);
+                TxnToWriteId _elem638;
+                for (int _i639 = 0; _i639 < _list637.size; ++_i639)
                 {
-                  _elem630 = new TxnToWriteId();
-                  _elem630.read(iprot);
-                  struct.srcTxnToWriteIdList.add(_elem630);
+                  _elem638 = new TxnToWriteId();
+                  _elem638.read(iprot);
+                  struct.srcTxnToWriteIdList.add(_elem638);
                 }
                 iprot.readListEnd();
               }
@@ -786,9 +786,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TXN_IDS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.txnIds.size()));
-            for (long _iter632 : struct.txnIds)
+            for (long _iter640 : struct.txnIds)
             {
-              oprot.writeI64(_iter632);
+              oprot.writeI64(_iter640);
             }
             oprot.writeListEnd();
           }
@@ -807,9 +807,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SRC_TXN_TO_WRITE_ID_LIST_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.srcTxnToWriteIdList.size()));
-            for (TxnToWriteId _iter633 : struct.srcTxnToWriteIdList)
+            for (TxnToWriteId _iter641 : struct.srcTxnToWriteIdList)
             {
-              _iter633.write(oprot);
+              _iter641.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -849,9 +849,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTxnIds()) {
         {
           oprot.writeI32(struct.txnIds.size());
-          for (long _iter634 : struct.txnIds)
+          for (long _iter642 : struct.txnIds)
           {
-            oprot.writeI64(_iter634);
+            oprot.writeI64(_iter642);
           }
         }
       }
@@ -861,9 +861,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSrcTxnToWriteIdList()) {
         {
           oprot.writeI32(struct.srcTxnToWriteIdList.size());
-          for (TxnToWriteId _iter635 : struct.srcTxnToWriteIdList)
+          for (TxnToWriteId _iter643 : struct.srcTxnToWriteIdList)
           {
-            _iter635.write(oprot);
+            _iter643.write(oprot);
           }
         }
       }
@@ -879,13 +879,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list636 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-          struct.txnIds = new ArrayList<Long>(_list636.size);
-          long _elem637;
-          for (int _i638 = 0; _i638 < _list636.size; ++_i638)
+          org.apache.thrift.protocol.TList _list644 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+          struct.txnIds = new ArrayList<Long>(_list644.size);
+          long _elem645;
+          for (int _i646 = 0; _i646 < _list644.size; ++_i646)
           {
-            _elem637 = iprot.readI64();
-            struct.txnIds.add(_elem637);
+            _elem645 = iprot.readI64();
+            struct.txnIds.add(_elem645);
           }
         }
         struct.setTxnIdsIsSet(true);
@@ -896,14 +896,14 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list639 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list639.size);
-          TxnToWriteId _elem640;
-          for (int _i641 = 0; _i641 < _list639.size; ++_i641)
+          org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.srcTxnToWriteIdList = new ArrayList<TxnToWriteId>(_list647.size);
+          TxnToWriteId _elem648;
+          for (int _i649 = 0; _i649 < _list647.size; ++_i649)
           {
-            _elem640 = new TxnToWriteId();
-            _elem640.read(iprot);
-            struct.srcTxnToWriteIdList.add(_elem640);
+            _elem648 = new TxnToWriteId();
+            _elem648.read(iprot);
+            struct.srcTxnToWriteIdList.add(_elem648);
           }
         }
         struct.setSrcTxnToWriteIdListIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
index 20dc757..13df26d 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AllocateTableWriteIdsResponse.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TXN_TO_WRITE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list642 = iprot.readListBegin();
-                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list642.size);
-                TxnToWriteId _elem643;
-                for (int _i644 = 0; _i644 < _list642.size; ++_i644)
+                org.apache.thrift.protocol.TList _list650 = iprot.readListBegin();
+                struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list650.size);
+                TxnToWriteId _elem651;
+                for (int _i652 = 0; _i652 < _list650.size; ++_i652)
                 {
-                  _elem643 = new TxnToWriteId();
-                  _elem643.read(iprot);
-                  struct.txnToWriteIds.add(_elem643);
+                  _elem651 = new TxnToWriteId();
+                  _elem651.read(iprot);
+                  struct.txnToWriteIds.add(_elem651);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TXN_TO_WRITE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.txnToWriteIds.size()));
-          for (TxnToWriteId _iter645 : struct.txnToWriteIds)
+          for (TxnToWriteId _iter653 : struct.txnToWriteIds)
           {
-            _iter645.write(oprot);
+            _iter653.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.txnToWriteIds.size());
-        for (TxnToWriteId _iter646 : struct.txnToWriteIds)
+        for (TxnToWriteId _iter654 : struct.txnToWriteIds)
         {
-          _iter646.write(oprot);
+          _iter654.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, AllocateTableWriteIdsResponse struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list647 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list647.size);
-        TxnToWriteId _elem648;
-        for (int _i649 = 0; _i649 < _list647.size; ++_i649)
+        org.apache.thrift.protocol.TList _list655 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.txnToWriteIds = new ArrayList<TxnToWriteId>(_list655.size);
+        TxnToWriteId _elem656;
+        for (int _i657 = 0; _i657 < _list655.size; ++_i657)
         {
-          _elem648 = new TxnToWriteId();
-          _elem648.read(iprot);
-          struct.txnToWriteIds.add(_elem648);
+          _elem656 = new TxnToWriteId();
+          _elem656.read(iprot);
+          struct.txnToWriteIds.add(_elem656);
         }
       }
       struct.setTxnToWriteIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
index 470a070..1af1628 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClearFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list800.size);
-                long _elem801;
-                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
+                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list824.size);
+                long _elem825;
+                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
                 {
-                  _elem801 = iprot.readI64();
-                  struct.fileIds.add(_elem801);
+                  _elem825 = iprot.readI64();
+                  struct.fileIds.add(_elem825);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter803 : struct.fileIds)
+          for (long _iter827 : struct.fileIds)
           {
-            oprot.writeI64(_iter803);
+            oprot.writeI64(_iter827);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter804 : struct.fileIds)
+        for (long _iter828 : struct.fileIds)
         {
-          oprot.writeI64(_iter804);
+          oprot.writeI64(_iter828);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClearFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list805.size);
-        long _elem806;
-        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
+        org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list829.size);
+        long _elem830;
+        for (int _i831 = 0; _i831 < _list829.size; ++_i831)
         {
-          _elem806 = iprot.readI64();
-          struct.fileIds.add(_elem806);
+          _elem830 = iprot.readI64();
+          struct.fileIds.add(_elem830);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
index af48583..4cd04f1 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ClientCapabilities.java
@@ -354,13 +354,13 @@ import org.slf4j.LoggerFactory;
           case 1: // VALUES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list816 = iprot.readListBegin();
-                struct.values = new ArrayList<ClientCapability>(_list816.size);
-                ClientCapability _elem817;
-                for (int _i818 = 0; _i818 < _list816.size; ++_i818)
+                org.apache.thrift.protocol.TList _list840 = iprot.readListBegin();
+                struct.values = new ArrayList<ClientCapability>(_list840.size);
+                ClientCapability _elem841;
+                for (int _i842 = 0; _i842 < _list840.size; ++_i842)
                 {
-                  _elem817 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-                  struct.values.add(_elem817);
+                  _elem841 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+                  struct.values.add(_elem841);
                 }
                 iprot.readListEnd();
               }
@@ -386,9 +386,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(VALUES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size()));
-          for (ClientCapability _iter819 : struct.values)
+          for (ClientCapability _iter843 : struct.values)
           {
-            oprot.writeI32(_iter819.getValue());
+            oprot.writeI32(_iter843.getValue());
           }
           oprot.writeListEnd();
         }
@@ -413,9 +413,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.values.size());
-        for (ClientCapability _iter820 : struct.values)
+        for (ClientCapability _iter844 : struct.values)
         {
-          oprot.writeI32(_iter820.getValue());
+          oprot.writeI32(_iter844.getValue());
         }
       }
     }
@@ -424,13 +424,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, ClientCapabilities struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list821 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
-        struct.values = new ArrayList<ClientCapability>(_list821.size);
-        ClientCapability _elem822;
-        for (int _i823 = 0; _i823 < _list821.size; ++_i823)
+        org.apache.thrift.protocol.TList _list845 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32());
+        struct.values = new ArrayList<ClientCapability>(_list845.size);
+        ClientCapability _elem846;
+        for (int _i847 = 0; _i847 < _list845.size; ++_i847)
         {
-          _elem822 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
-          struct.values.add(_elem822);
+          _elem846 = org.apache.hadoop.hive.metastore.api.ClientCapability.findByValue(iprot.readI32());
+          struct.values.add(_elem846);
         }
       }
       struct.setValuesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
index 3c15f84..f295958 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CommitTxnRequest.java
@@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory;
 
   private static final org.apache.thrift.protocol.TField TXNID_FIELD_DESC = new org.apache.thrift.protocol.TField("txnid", org.apache.thrift.protocol.TType.I64, (short)1);
   private static final org.apache.thrift.protocol.TField REPL_POLICY_FIELD_DESC = new org.apache.thrift.protocol.TField("replPolicy", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField WRITE_EVENT_INFOS_FIELD_DESC = new org.apache.thrift.protocol.TField("writeEventInfos", org.apache.thrift.protocol.TType.LIST, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,11 +50,13 @@ import org.slf4j.LoggerFactory;
 
   private long txnid; // required
   private String replPolicy; // optional
+  private List<WriteEventInfo> writeEventInfos; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     TXNID((short)1, "txnid"),
-    REPL_POLICY((short)2, "replPolicy");
+    REPL_POLICY((short)2, "replPolicy"),
+    WRITE_EVENT_INFOS((short)3, "writeEventInfos");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -72,6 +75,8 @@ import org.slf4j.LoggerFactory;
           return TXNID;
         case 2: // REPL_POLICY
           return REPL_POLICY;
+        case 3: // WRITE_EVENT_INFOS
+          return WRITE_EVENT_INFOS;
         default:
           return null;
       }
@@ -114,7 +119,7 @@ import org.slf4j.LoggerFactory;
   // isset id assignments
   private static final int __TXNID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.REPL_POLICY};
+  private static final _Fields optionals[] = {_Fields.REPL_POLICY,_Fields.WRITE_EVENT_INFOS};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -122,6 +127,9 @@ import org.slf4j.LoggerFactory;
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
     tmpMap.put(_Fields.REPL_POLICY, new org.apache.thrift.meta_data.FieldMetaData("replPolicy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.WRITE_EVENT_INFOS, new org.apache.thrift.meta_data.FieldMetaData("writeEventInfos", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+            new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT            , "WriteEventInfo"))));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CommitTxnRequest.class, metaDataMap);
   }
@@ -146,6 +154,13 @@ import org.slf4j.LoggerFactory;
     if (other.isSetReplPolicy()) {
       this.replPolicy = other.replPolicy;
     }
+    if (other.isSetWriteEventInfos()) {
+      List<WriteEventInfo> __this__writeEventInfos = new ArrayList<WriteEventInfo>(other.writeEventInfos.size());
+      for (WriteEventInfo other_element : other.writeEventInfos) {
+        __this__writeEventInfos.add(other_element);
+      }
+      this.writeEventInfos = __this__writeEventInfos;
+    }
   }
 
   public CommitTxnRequest deepCopy() {
@@ -157,6 +172,7 @@ import org.slf4j.LoggerFactory;
     setTxnidIsSet(false);
     this.txnid = 0;
     this.replPolicy = null;
+    this.writeEventInfos = null;
   }
 
   public long getTxnid() {
@@ -204,6 +220,44 @@ import org.slf4j.LoggerFactory;
     }
   }
 
+  public int getWriteEventInfosSize() {
+    return (this.writeEventInfos == null) ? 0 : this.writeEventInfos.size();
+  }
+
+  public java.util.Iterator<WriteEventInfo> getWriteEventInfosIterator() {
+    return (this.writeEventInfos == null) ? null : this.writeEventInfos.iterator();
+  }
+
+  public void addToWriteEventInfos(WriteEventInfo elem) {
+    if (this.writeEventInfos == null) {
+      this.writeEventInfos = new ArrayList<WriteEventInfo>();
+    }
+    this.writeEventInfos.add(elem);
+  }
+
+  public List<WriteEventInfo> getWriteEventInfos() {
+    return this.writeEventInfos;
+  }
+
+  public void setWriteEventInfos(List<WriteEventInfo> writeEventInfos) {
+    this.writeEventInfos = writeEventInfos;
+  }
+
+  public void unsetWriteEventInfos() {
+    this.writeEventInfos = null;
+  }
+
+  /** Returns true if field writeEventInfos is set (has been assigned a value) and false otherwise */
+  public boolean isSetWriteEventInfos() {
+    return this.writeEventInfos != null;
+  }
+
+  public void setWriteEventInfosIsSet(boolean value) {
+    if (!value) {
+      this.writeEventInfos = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TXNID:
@@ -222,6 +276,14 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
+    case WRITE_EVENT_INFOS:
+      if (value == null) {
+        unsetWriteEventInfos();
+      } else {
+        setWriteEventInfos((List<WriteEventInfo>)value);
+      }
+      break;
+
     }
   }
 
@@ -233,6 +295,9 @@ import org.slf4j.LoggerFactory;
     case REPL_POLICY:
       return getReplPolicy();
 
+    case WRITE_EVENT_INFOS:
+      return getWriteEventInfos();
+
     }
     throw new IllegalStateException();
   }
@@ -248,6 +313,8 @@ import org.slf4j.LoggerFactory;
       return isSetTxnid();
     case REPL_POLICY:
       return isSetReplPolicy();
+    case WRITE_EVENT_INFOS:
+      return isSetWriteEventInfos();
     }
     throw new IllegalStateException();
   }
@@ -283,6 +350,15 @@ import org.slf4j.LoggerFactory;
         return false;
     }
 
+    boolean this_present_writeEventInfos = true && this.isSetWriteEventInfos();
+    boolean that_present_writeEventInfos = true && that.isSetWriteEventInfos();
+    if (this_present_writeEventInfos || that_present_writeEventInfos) {
+      if (!(this_present_writeEventInfos && that_present_writeEventInfos))
+        return false;
+      if (!this.writeEventInfos.equals(that.writeEventInfos))
+        return false;
+    }
+
     return true;
   }
 
@@ -300,6 +376,11 @@ import org.slf4j.LoggerFactory;
     if (present_replPolicy)
       list.add(replPolicy);
 
+    boolean present_writeEventInfos = true && (isSetWriteEventInfos());
+    list.add(present_writeEventInfos);
+    if (present_writeEventInfos)
+      list.add(writeEventInfos);
+
     return list.hashCode();
   }
 
@@ -331,6 +412,16 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetWriteEventInfos()).compareTo(other.isSetWriteEventInfos());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetWriteEventInfos()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.writeEventInfos, other.writeEventInfos);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -364,6 +455,16 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
+    if (isSetWriteEventInfos()) {
+      if (!first) sb.append(", ");
+      sb.append("writeEventInfos:");
+      if (this.writeEventInfos == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.writeEventInfos);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -429,6 +530,25 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // WRITE_EVENT_INFOS
+            if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+              {
+                org.apache.thrift.protocol.TList _list594 = iprot.readListBegin();
+                struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list594.size);
+                WriteEventInfo _elem595;
+                for (int _i596 = 0; _i596 < _list594.size; ++_i596)
+                {
+                  _elem595 = new WriteEventInfo();
+                  _elem595.read(iprot);
+                  struct.writeEventInfos.add(_elem595);
+                }
+                iprot.readListEnd();
+              }
+              struct.setWriteEventInfosIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -452,6 +572,20 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
+      if (struct.writeEventInfos != null) {
+        if (struct.isSetWriteEventInfos()) {
+          oprot.writeFieldBegin(WRITE_EVENT_INFOS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.writeEventInfos.size()));
+            for (WriteEventInfo _iter597 : struct.writeEventInfos)
+            {
+              _iter597.write(oprot);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -474,10 +608,22 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetReplPolicy()) {
         optionals.set(0);
       }
-      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetWriteEventInfos()) {
+        optionals.set(1);
+      }
+      oprot.writeBitSet(optionals, 2);
       if (struct.isSetReplPolicy()) {
         oprot.writeString(struct.replPolicy);
       }
+      if (struct.isSetWriteEventInfos()) {
+        {
+          oprot.writeI32(struct.writeEventInfos.size());
+          for (WriteEventInfo _iter598 : struct.writeEventInfos)
+          {
+            _iter598.write(oprot);
+          }
+        }
+      }
     }
 
     @Override
@@ -485,11 +631,25 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol iprot = (TTupleProtocol) prot;
       struct.txnid = iprot.readI64();
       struct.setTxnidIsSet(true);
-      BitSet incoming = iprot.readBitSet(1);
+      BitSet incoming = iprot.readBitSet(2);
       if (incoming.get(0)) {
         struct.replPolicy = iprot.readString();
         struct.setReplPolicyIsSet(true);
       }
+      if (incoming.get(1)) {
+        {
+          org.apache.thrift.protocol.TList _list599 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.writeEventInfos = new ArrayList<WriteEventInfo>(_list599.size);
+          WriteEventInfo _elem600;
+          for (int _i601 = 0; _i601 < _list599.size; ++_i601)
+          {
+            _elem600 = new WriteEventInfo();
+            _elem600.read(iprot);
+            struct.writeEventInfos.add(_elem600);
+          }
+        }
+        struct.setWriteEventInfosIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
index 31f2e14..57eb5ef 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionRequest.java
@@ -814,15 +814,15 @@ import org.slf4j.LoggerFactory;
           case 6: // PROPERTIES
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map682 = iprot.readMapBegin();
-                struct.properties = new HashMap<String,String>(2*_map682.size);
-                String _key683;
-                String _val684;
-                for (int _i685 = 0; _i685 < _map682.size; ++_i685)
+                org.apache.thrift.protocol.TMap _map690 = iprot.readMapBegin();
+                struct.properties = new HashMap<String,String>(2*_map690.size);
+                String _key691;
+                String _val692;
+                for (int _i693 = 0; _i693 < _map690.size; ++_i693)
                 {
-                  _key683 = iprot.readString();
-                  _val684 = iprot.readString();
-                  struct.properties.put(_key683, _val684);
+                  _key691 = iprot.readString();
+                  _val692 = iprot.readString();
+                  struct.properties.put(_key691, _val692);
                 }
                 iprot.readMapEnd();
               }
@@ -878,10 +878,10 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PROPERTIES_FIELD_DESC);
           {
             oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.properties.size()));
-            for (Map.Entry<String, String> _iter686 : struct.properties.entrySet())
+            for (Map.Entry<String, String> _iter694 : struct.properties.entrySet())
             {
-              oprot.writeString(_iter686.getKey());
-              oprot.writeString(_iter686.getValue());
+              oprot.writeString(_iter694.getKey());
+              oprot.writeString(_iter694.getValue());
             }
             oprot.writeMapEnd();
           }
@@ -928,10 +928,10 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetProperties()) {
         {
           oprot.writeI32(struct.properties.size());
-          for (Map.Entry<String, String> _iter687 : struct.properties.entrySet())
+          for (Map.Entry<String, String> _iter695 : struct.properties.entrySet())
           {
-            oprot.writeString(_iter687.getKey());
-            oprot.writeString(_iter687.getValue());
+            oprot.writeString(_iter695.getKey());
+            oprot.writeString(_iter695.getValue());
           }
         }
       }
@@ -957,15 +957,15 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TMap _map688 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.properties = new HashMap<String,String>(2*_map688.size);
-          String _key689;
-          String _val690;
-          for (int _i691 = 0; _i691 < _map688.size; ++_i691)
+          org.apache.thrift.protocol.TMap _map696 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.properties = new HashMap<String,String>(2*_map696.size);
+          String _key697;
+          String _val698;
+          for (int _i699 = 0; _i699 < _map696.size; ++_i699)
           {
-            _key689 = iprot.readString();
-            _val690 = iprot.readString();
-            struct.properties.put(_key689, _val690);
+            _key697 = iprot.readString();
+            _val698 = iprot.readString();
+            struct.properties.put(_key697, _val698);
           }
         }
         struct.setPropertiesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
index ab7b059..611bf6f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CreationMetadata.java
@@ -712,13 +712,13 @@ import org.slf4j.LoggerFactory;
           case 4: // TABLES_USED
             if (schemeField.type == org.apache.thrift.protocol.TType.SET) {
               {
-                org.apache.thrift.protocol.TSet _set708 = iprot.readSetBegin();
-                struct.tablesUsed = new HashSet<String>(2*_set708.size);
-                String _elem709;
-                for (int _i710 = 0; _i710 < _set708.size; ++_i710)
+                org.apache.thrift.protocol.TSet _set716 = iprot.readSetBegin();
+                struct.tablesUsed = new HashSet<String>(2*_set716.size);
+                String _elem717;
+                for (int _i718 = 0; _i718 < _set716.size; ++_i718)
                 {
-                  _elem709 = iprot.readString();
-                  struct.tablesUsed.add(_elem709);
+                  _elem717 = iprot.readString();
+                  struct.tablesUsed.add(_elem717);
                 }
                 iprot.readSetEnd();
               }
@@ -767,9 +767,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_USED_FIELD_DESC);
         {
           oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, struct.tablesUsed.size()));
-          for (String _iter711 : struct.tablesUsed)
+          for (String _iter719 : struct.tablesUsed)
           {
-            oprot.writeString(_iter711);
+            oprot.writeString(_iter719);
           }
           oprot.writeSetEnd();
         }
@@ -804,9 +804,9 @@ import org.slf4j.LoggerFactory;
       oprot.writeString(struct.tblName);
       {
         oprot.writeI32(struct.tablesUsed.size());
-        for (String _iter712 : struct.tablesUsed)
+        for (String _iter720 : struct.tablesUsed)
         {
-          oprot.writeString(_iter712);
+          oprot.writeString(_iter720);
         }
       }
       BitSet optionals = new BitSet();
@@ -829,13 +829,13 @@ import org.slf4j.LoggerFactory;
       struct.tblName = iprot.readString();
       struct.setTblNameIsSet(true);
       {
-        org.apache.thrift.protocol.TSet _set713 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.tablesUsed = new HashSet<String>(2*_set713.size);
-        String _elem714;
-        for (int _i715 = 0; _i715 < _set713.size; ++_i715)
+        org.apache.thrift.protocol.TSet _set721 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.tablesUsed = new HashSet<String>(2*_set721.size);
+        String _elem722;
+        for (int _i723 = 0; _i723 < _set721.size; ++_i723)
         {
-          _elem714 = iprot.readString();
-          struct.tablesUsed.add(_elem714);
+          _elem722 = iprot.readString();
+          struct.tablesUsed.add(_elem722);
         }
       }
       struct.setTablesUsedIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
index e43493e..8f5b4e5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FindSchemasByColsResp.java
@@ -350,14 +350,14 @@ import org.slf4j.LoggerFactory;
           case 1: // SCHEMA_VERSIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list920 = iprot.readListBegin();
-                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list920.size);
-                SchemaVersionDescriptor _elem921;
-                for (int _i922 = 0; _i922 < _list920.size; ++_i922)
+                org.apache.thrift.protocol.TList _list944 = iprot.readListBegin();
+                struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list944.size);
+                SchemaVersionDescriptor _elem945;
+                for (int _i946 = 0; _i946 < _list944.size; ++_i946)
                 {
-                  _elem921 = new SchemaVersionDescriptor();
-                  _elem921.read(iprot);
-                  struct.schemaVersions.add(_elem921);
+                  _elem945 = new SchemaVersionDescriptor();
+                  _elem945.read(iprot);
+                  struct.schemaVersions.add(_elem945);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(SCHEMA_VERSIONS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.schemaVersions.size()));
-          for (SchemaVersionDescriptor _iter923 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter947 : struct.schemaVersions)
           {
-            _iter923.write(oprot);
+            _iter947.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -416,9 +416,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetSchemaVersions()) {
         {
           oprot.writeI32(struct.schemaVersions.size());
-          for (SchemaVersionDescriptor _iter924 : struct.schemaVersions)
+          for (SchemaVersionDescriptor _iter948 : struct.schemaVersions)
           {
-            _iter924.write(oprot);
+            _iter948.write(oprot);
           }
         }
       }
@@ -430,14 +430,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list925 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list925.size);
-          SchemaVersionDescriptor _elem926;
-          for (int _i927 = 0; _i927 < _list925.size; ++_i927)
+          org.apache.thrift.protocol.TList _list949 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.schemaVersions = new ArrayList<SchemaVersionDescriptor>(_list949.size);
+          SchemaVersionDescriptor _elem950;
+          for (int _i951 = 0; _i951 < _list949.size; ++_i951)
           {
-            _elem926 = new SchemaVersionDescriptor();
-            _elem926.read(iprot);
-            struct.schemaVersions.add(_elem926);
+            _elem950 = new SchemaVersionDescriptor();
+            _elem950.read(iprot);
+            struct.schemaVersions.add(_elem950);
           }
         }
         struct.setSchemaVersionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
index 7b0ec6c..2560922 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/FireEventRequest.java
@@ -794,13 +794,13 @@ import org.slf4j.LoggerFactory;
           case 5: // PARTITION_VALS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list740 = iprot.readListBegin();
-                struct.partitionVals = new ArrayList<String>(_list740.size);
-                String _elem741;
-                for (int _i742 = 0; _i742 < _list740.size; ++_i742)
+                org.apache.thrift.protocol.TList _list756 = iprot.readListBegin();
+                struct.partitionVals = new ArrayList<String>(_list756.size);
+                String _elem757;
+                for (int _i758 = 0; _i758 < _list756.size; ++_i758)
                 {
-                  _elem741 = iprot.readString();
-                  struct.partitionVals.add(_elem741);
+                  _elem757 = iprot.readString();
+                  struct.partitionVals.add(_elem757);
                 }
                 iprot.readListEnd();
               }
@@ -857,9 +857,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(PARTITION_VALS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.partitionVals.size()));
-            for (String _iter743 : struct.partitionVals)
+            for (String _iter759 : struct.partitionVals)
             {
-              oprot.writeString(_iter743);
+              oprot.writeString(_iter759);
             }
             oprot.writeListEnd();
           }
@@ -915,9 +915,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetPartitionVals()) {
         {
           oprot.writeI32(struct.partitionVals.size());
-          for (String _iter744 : struct.partitionVals)
+          for (String _iter760 : struct.partitionVals)
           {
-            oprot.writeString(_iter744);
+            oprot.writeString(_iter760);
           }
         }
       }
@@ -945,13 +945,13 @@ import org.slf4j.LoggerFactory;
       }
       if (incoming.get(2)) {
         {
-          org.apache.thrift.protocol.TList _list745 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.partitionVals = new ArrayList<String>(_list745.size);
-          String _elem746;
-          for (int _i747 = 0; _i747 < _list745.size; ++_i747)
+          org.apache.thrift.protocol.TList _list761 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.partitionVals = new ArrayList<String>(_list761.size);
+          String _elem762;
+          for (int _i763 = 0; _i763 < _list761.size; ++_i763)
           {
-            _elem746 = iprot.readString();
-            struct.partitionVals.add(_elem746);
+            _elem762 = iprot.readString();
+            struct.partitionVals.add(_elem762);
           }
         }
         struct.setPartitionValsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
index 544ba19..f68afe8 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetAllFunctionsResponse.java
@@ -346,14 +346,14 @@ import org.slf4j.LoggerFactory;
           case 1: // FUNCTIONS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list808 = iprot.readListBegin();
-                struct.functions = new ArrayList<Function>(_list808.size);
-                Function _elem809;
-                for (int _i810 = 0; _i810 < _list808.size; ++_i810)
+                org.apache.thrift.protocol.TList _list832 = iprot.readListBegin();
+                struct.functions = new ArrayList<Function>(_list832.size);
+                Function _elem833;
+                for (int _i834 = 0; _i834 < _list832.size; ++_i834)
                 {
-                  _elem809 = new Function();
-                  _elem809.read(iprot);
-                  struct.functions.add(_elem809);
+                  _elem833 = new Function();
+                  _elem833.read(iprot);
+                  struct.functions.add(_elem833);
                 }
                 iprot.readListEnd();
               }
@@ -380,9 +380,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(FUNCTIONS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.functions.size()));
-            for (Function _iter811 : struct.functions)
+            for (Function _iter835 : struct.functions)
             {
-              _iter811.write(oprot);
+              _iter835.write(oprot);
             }
             oprot.writeListEnd();
           }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetFunctions()) {
         {
           oprot.writeI32(struct.functions.size());
-          for (Function _iter812 : struct.functions)
+          for (Function _iter836 : struct.functions)
           {
-            _iter812.write(oprot);
+            _iter836.write(oprot);
           }
         }
       }
@@ -428,14 +428,14 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(1);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list813 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-          struct.functions = new ArrayList<Function>(_list813.size);
-          Function _elem814;
-          for (int _i815 = 0; _i815 < _list813.size; ++_i815)
+          org.apache.thrift.protocol.TList _list837 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+          struct.functions = new ArrayList<Function>(_list837.size);
+          Function _elem838;
+          for (int _i839 = 0; _i839 < _list837.size; ++_i839)
           {
-            _elem814 = new Function();
-            _elem814.read(iprot);
-            struct.functions.add(_elem814);
+            _elem838 = new Function();
+            _elem838.read(iprot);
+            struct.functions.add(_elem838);
           }
         }
         struct.setFunctionsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
index 0a94f2f..836f35f 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprRequest.java
@@ -619,13 +619,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list758 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list758.size);
-                long _elem759;
-                for (int _i760 = 0; _i760 < _list758.size; ++_i760)
+                org.apache.thrift.protocol.TList _list782 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list782.size);
+                long _elem783;
+                for (int _i784 = 0; _i784 < _list782.size; ++_i784)
                 {
-                  _elem759 = iprot.readI64();
-                  struct.fileIds.add(_elem759);
+                  _elem783 = iprot.readI64();
+                  struct.fileIds.add(_elem783);
                 }
                 iprot.readListEnd();
               }
@@ -675,9 +675,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter761 : struct.fileIds)
+          for (long _iter785 : struct.fileIds)
           {
-            oprot.writeI64(_iter761);
+            oprot.writeI64(_iter785);
           }
           oprot.writeListEnd();
         }
@@ -719,9 +719,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter762 : struct.fileIds)
+        for (long _iter786 : struct.fileIds)
         {
-          oprot.writeI64(_iter762);
+          oprot.writeI64(_iter786);
         }
       }
       oprot.writeBinary(struct.expr);
@@ -745,13 +745,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list763 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list763.size);
-        long _elem764;
-        for (int _i765 = 0; _i765 < _list763.size; ++_i765)
+        org.apache.thrift.protocol.TList _list787 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list787.size);
+        long _elem788;
+        for (int _i789 = 0; _i789 < _list787.size; ++_i789)
         {
-          _elem764 = iprot.readI64();
-          struct.fileIds.add(_elem764);
+          _elem788 = iprot.readI64();
+          struct.fileIds.add(_elem788);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
index e07d2e5..17f0ee5 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataByExprResult.java
@@ -444,16 +444,16 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map748 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map748.size);
-                long _key749;
-                MetadataPpdResult _val750;
-                for (int _i751 = 0; _i751 < _map748.size; ++_i751)
+                org.apache.thrift.protocol.TMap _map772 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map772.size);
+                long _key773;
+                MetadataPpdResult _val774;
+                for (int _i775 = 0; _i775 < _map772.size; ++_i775)
                 {
-                  _key749 = iprot.readI64();
-                  _val750 = new MetadataPpdResult();
-                  _val750.read(iprot);
-                  struct.metadata.put(_key749, _val750);
+                  _key773 = iprot.readI64();
+                  _val774 = new MetadataPpdResult();
+                  _val774.read(iprot);
+                  struct.metadata.put(_key773, _val774);
                 }
                 iprot.readMapEnd();
               }
@@ -487,10 +487,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, struct.metadata.size()));
-          for (Map.Entry<Long, MetadataPpdResult> _iter752 : struct.metadata.entrySet())
+          for (Map.Entry<Long, MetadataPpdResult> _iter776 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter752.getKey());
-            _iter752.getValue().write(oprot);
+            oprot.writeI64(_iter776.getKey());
+            _iter776.getValue().write(oprot);
           }
           oprot.writeMapEnd();
         }
@@ -518,10 +518,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, MetadataPpdResult> _iter753 : struct.metadata.entrySet())
+        for (Map.Entry<Long, MetadataPpdResult> _iter777 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter753.getKey());
-          _iter753.getValue().write(oprot);
+          oprot.writeI64(_iter777.getKey());
+          _iter777.getValue().write(oprot);
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -531,16 +531,16 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataByExprResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map754 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map754.size);
-        long _key755;
-        MetadataPpdResult _val756;
-        for (int _i757 = 0; _i757 < _map754.size; ++_i757)
+        org.apache.thrift.protocol.TMap _map778 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.metadata = new HashMap<Long,MetadataPpdResult>(2*_map778.size);
+        long _key779;
+        MetadataPpdResult _val780;
+        for (int _i781 = 0; _i781 < _map778.size; ++_i781)
         {
-          _key755 = iprot.readI64();
-          _val756 = new MetadataPpdResult();
-          _val756.read(iprot);
-          struct.metadata.put(_key755, _val756);
+          _key779 = iprot.readI64();
+          _val780 = new MetadataPpdResult();
+          _val780.read(iprot);
+          struct.metadata.put(_key779, _val780);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
index ebb6639..12b4392 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataRequest.java
@@ -351,13 +351,13 @@ import org.slf4j.LoggerFactory;
           case 1: // FILE_IDS
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list776 = iprot.readListBegin();
-                struct.fileIds = new ArrayList<Long>(_list776.size);
-                long _elem777;
-                for (int _i778 = 0; _i778 < _list776.size; ++_i778)
+                org.apache.thrift.protocol.TList _list800 = iprot.readListBegin();
+                struct.fileIds = new ArrayList<Long>(_list800.size);
+                long _elem801;
+                for (int _i802 = 0; _i802 < _list800.size; ++_i802)
                 {
-                  _elem777 = iprot.readI64();
-                  struct.fileIds.add(_elem777);
+                  _elem801 = iprot.readI64();
+                  struct.fileIds.add(_elem801);
                 }
                 iprot.readListEnd();
               }
@@ -383,9 +383,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(FILE_IDS_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.fileIds.size()));
-          for (long _iter779 : struct.fileIds)
+          for (long _iter803 : struct.fileIds)
           {
-            oprot.writeI64(_iter779);
+            oprot.writeI64(_iter803);
           }
           oprot.writeListEnd();
         }
@@ -410,9 +410,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.fileIds.size());
-        for (long _iter780 : struct.fileIds)
+        for (long _iter804 : struct.fileIds)
         {
-          oprot.writeI64(_iter780);
+          oprot.writeI64(_iter804);
         }
       }
     }
@@ -421,13 +421,13 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataRequest struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list781 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
-        struct.fileIds = new ArrayList<Long>(_list781.size);
-        long _elem782;
-        for (int _i783 = 0; _i783 < _list781.size; ++_i783)
+        org.apache.thrift.protocol.TList _list805 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32());
+        struct.fileIds = new ArrayList<Long>(_list805.size);
+        long _elem806;
+        for (int _i807 = 0; _i807 < _list805.size; ++_i807)
         {
-          _elem782 = iprot.readI64();
-          struct.fileIds.add(_elem782);
+          _elem806 = iprot.readI64();
+          struct.fileIds.add(_elem806);
         }
       }
       struct.setFileIdsIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
index 67981cd..65708d7 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetFileMetadataResult.java
@@ -433,15 +433,15 @@ import org.slf4j.LoggerFactory;
           case 1: // METADATA
             if (schemeField.type == org.apache.thrift.protocol.TType.MAP) {
               {
-                org.apache.thrift.protocol.TMap _map766 = iprot.readMapBegin();
-                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map766.size);
-                long _key767;
-                ByteBuffer _val768;
-                for (int _i769 = 0; _i769 < _map766.size; ++_i769)
+                org.apache.thrift.protocol.TMap _map790 = iprot.readMapBegin();
+                struct.metadata = new HashMap<Long,ByteBuffer>(2*_map790.size);
+                long _key791;
+                ByteBuffer _val792;
+                for (int _i793 = 0; _i793 < _map790.size; ++_i793)
                 {
-                  _key767 = iprot.readI64();
-                  _val768 = iprot.readBinary();
-                  struct.metadata.put(_key767, _val768);
+                  _key791 = iprot.readI64();
+                  _val792 = iprot.readBinary();
+                  struct.metadata.put(_key791, _val792);
                 }
                 iprot.readMapEnd();
               }
@@ -475,10 +475,10 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(METADATA_FIELD_DESC);
         {
           oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, struct.metadata.size()));
-          for (Map.Entry<Long, ByteBuffer> _iter770 : struct.metadata.entrySet())
+          for (Map.Entry<Long, ByteBuffer> _iter794 : struct.metadata.entrySet())
           {
-            oprot.writeI64(_iter770.getKey());
-            oprot.writeBinary(_iter770.getValue());
+            oprot.writeI64(_iter794.getKey());
+            oprot.writeBinary(_iter794.getValue());
           }
           oprot.writeMapEnd();
         }
@@ -506,10 +506,10 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.metadata.size());
-        for (Map.Entry<Long, ByteBuffer> _iter771 : struct.metadata.entrySet())
+        for (Map.Entry<Long, ByteBuffer> _iter795 : struct.metadata.entrySet())
         {
-          oprot.writeI64(_iter771.getKey());
-          oprot.writeBinary(_iter771.getValue());
+          oprot.writeI64(_iter795.getKey());
+          oprot.writeBinary(_iter795.getValue());
         }
       }
       oprot.writeBool(struct.isSupported);
@@ -519,15 +519,15 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetFileMetadataResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TMap _map772 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map772.size);
-        long _key773;
-        ByteBuffer _val774;
-        for (int _i775 = 0; _i775 < _map772.size; ++_i775)
+        org.apache.thrift.protocol.TMap _map796 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I64, org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+        struct.metadata = new HashMap<Long,ByteBuffer>(2*_map796.size);
+        long _key797;
+        ByteBuffer _val798;
+        for (int _i799 = 0; _i799 < _map796.size; ++_i799)
         {
-          _key773 = iprot.readI64();
-          _val774 = iprot.readBinary();
-          struct.metadata.put(_key773, _val774);
+          _key797 = iprot.readI64();
+          _val798 = iprot.readBinary();
+          struct.metadata.put(_key797, _val798);
         }
       }
       struct.setMetadataIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
index 6a78b77..09ca865 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesRequest.java
@@ -606,13 +606,13 @@ import org.slf4j.LoggerFactory;
           case 2: // TBL_NAMES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list824 = iprot.readListBegin();
-                struct.tblNames = new ArrayList<String>(_list824.size);
-                String _elem825;
-                for (int _i826 = 0; _i826 < _list824.size; ++_i826)
+                org.apache.thrift.protocol.TList _list848 = iprot.readListBegin();
+                struct.tblNames = new ArrayList<String>(_list848.size);
+                String _elem849;
+                for (int _i850 = 0; _i850 < _list848.size; ++_i850)
                 {
-                  _elem825 = iprot.readString();
-                  struct.tblNames.add(_elem825);
+                  _elem849 = iprot.readString();
+                  struct.tblNames.add(_elem849);
                 }
                 iprot.readListEnd();
               }
@@ -661,9 +661,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(TBL_NAMES_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tblNames.size()));
-            for (String _iter827 : struct.tblNames)
+            for (String _iter851 : struct.tblNames)
             {
-              oprot.writeString(_iter827);
+              oprot.writeString(_iter851);
             }
             oprot.writeListEnd();
           }
@@ -716,9 +716,9 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetTblNames()) {
         {
           oprot.writeI32(struct.tblNames.size());
-          for (String _iter828 : struct.tblNames)
+          for (String _iter852 : struct.tblNames)
           {
-            oprot.writeString(_iter828);
+            oprot.writeString(_iter852);
           }
         }
       }
@@ -738,13 +738,13 @@ import org.slf4j.LoggerFactory;
       BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         {
-          org.apache.thrift.protocol.TList _list829 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-          struct.tblNames = new ArrayList<String>(_list829.size);
-          String _elem830;
-          for (int _i831 = 0; _i831 < _list829.size; ++_i831)
+          org.apache.thrift.protocol.TList _list853 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+          struct.tblNames = new ArrayList<String>(_list853.size);
+          String _elem854;
+          for (int _i855 = 0; _i855 < _list853.size; ++_i855)
           {
-            _elem830 = iprot.readString();
-            struct.tblNames.add(_elem830);
+            _elem854 = iprot.readString();
+            struct.tblNames.add(_elem854);
           }
         }
         struct.setTblNamesIsSet(true);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
index 13be2ed..72256e6 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/GetTablesResult.java
@@ -354,14 +354,14 @@ import org.slf4j.LoggerFactory;
           case 1: // TABLES
             if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
               {
-                org.apache.thrift.protocol.TList _list832 = iprot.readListBegin();
-                struct.tables = new ArrayList<Table>(_list832.size);
-                Table _elem833;
-                for (int _i834 = 0; _i834 < _list832.size; ++_i834)
+                org.apache.thrift.protocol.TList _list856 = iprot.readListBegin();
+                struct.tables = new ArrayList<Table>(_list856.size);
+                Table _elem857;
+                for (int _i858 = 0; _i858 < _list856.size; ++_i858)
                 {
-                  _elem833 = new Table();
-                  _elem833.read(iprot);
-                  struct.tables.add(_elem833);
+                  _elem857 = new Table();
+                  _elem857.read(iprot);
+                  struct.tables.add(_elem857);
                 }
                 iprot.readListEnd();
               }
@@ -387,9 +387,9 @@ import org.slf4j.LoggerFactory;
         oprot.writeFieldBegin(TABLES_FIELD_DESC);
         {
           oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.tables.size()));
-          for (Table _iter835 : struct.tables)
+          for (Table _iter859 : struct.tables)
           {
-            _iter835.write(oprot);
+            _iter859.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -414,9 +414,9 @@ import org.slf4j.LoggerFactory;
       TTupleProtocol oprot = (TTupleProtocol) prot;
       {
         oprot.writeI32(struct.tables.size());
-        for (Table _iter836 : struct.tables)
+        for (Table _iter860 : struct.tables)
         {
-          _iter836.write(oprot);
+          _iter860.write(oprot);
         }
       }
     }
@@ -425,14 +425,14 @@ import org.slf4j.LoggerFactory;
     public void read(org.apache.thrift.protocol.TProtocol prot, GetTablesResult struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
       {
-        org.apache.thrift.protocol.TList _list837 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
-        struct.tables = new ArrayList<Table>(_list837.size);
-        Table _elem838;
-        for (int _i839 = 0; _i839 < _list837.size; ++_i839)
+        org.apache.thrift.protocol.TList _list861 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32());
+        struct.tables = new ArrayList<Table>(_list861.size);
+        Table _elem862;
+        for (int _i863 = 0; _i863 < _list861.size; ++_i863)
         {
-          _elem838 = new Table();
-          _elem838.read(iprot);
-          struct.tables.add(_elem838);
+          _elem862 = new Table();
+          _elem862.read(iprot);
+          struct.tables.add(_elem862);
         }
       }
       struct.setTablesIsSet(true);


[22/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index a29ebb7..93b5780 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -16558,6 +16558,10 @@ class CommitTxnRequest {
    * @var string
    */
   public $replPolicy = null;
+  /**
+   * @var \metastore\WriteEventInfo[]
+   */
+  public $writeEventInfos = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -16570,6 +16574,15 @@ class CommitTxnRequest {
           'var' => 'replPolicy',
           'type' => TType::STRING,
           ),
+        3 => array(
+          'var' => 'writeEventInfos',
+          'type' => TType::LST,
+          'etype' => TType::STRUCT,
+          'elem' => array(
+            'type' => TType::STRUCT,
+            'class' => '\metastore\WriteEventInfo',
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -16579,6 +16592,9 @@ class CommitTxnRequest {
       if (isset($vals['replPolicy'])) {
         $this->replPolicy = $vals['replPolicy'];
       }
+      if (isset($vals['writeEventInfos'])) {
+        $this->writeEventInfos = $vals['writeEventInfos'];
+      }
     }
   }
 
@@ -16615,6 +16631,24 @@ class CommitTxnRequest {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 3:
+          if ($ftype == TType::LST) {
+            $this->writeEventInfos = array();
+            $_size523 = 0;
+            $_etype526 = 0;
+            $xfer += $input->readListBegin($_etype526, $_size523);
+            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            {
+              $elem528 = null;
+              $elem528 = new \metastore\WriteEventInfo();
+              $xfer += $elem528->read($input);
+              $this->writeEventInfos []= $elem528;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -16638,6 +16672,236 @@ class CommitTxnRequest {
       $xfer += $output->writeString($this->replPolicy);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->writeEventInfos !== null) {
+      if (!is_array($this->writeEventInfos)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('writeEventInfos', TType::LST, 3);
+      {
+        $output->writeListBegin(TType::STRUCT, count($this->writeEventInfos));
+        {
+          foreach ($this->writeEventInfos as $iter529)
+          {
+            $xfer += $iter529->write($output);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class WriteEventInfo {
+  static $_TSPEC;
+
+  /**
+   * @var int
+   */
+  public $writeId = null;
+  /**
+   * @var string
+   */
+  public $database = null;
+  /**
+   * @var string
+   */
+  public $table = null;
+  /**
+   * @var string
+   */
+  public $files = null;
+  /**
+   * @var string
+   */
+  public $partition = null;
+  /**
+   * @var string
+   */
+  public $tableObj = null;
+  /**
+   * @var string
+   */
+  public $partitionObj = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'writeId',
+          'type' => TType::I64,
+          ),
+        2 => array(
+          'var' => 'database',
+          'type' => TType::STRING,
+          ),
+        3 => array(
+          'var' => 'table',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'files',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'partition',
+          'type' => TType::STRING,
+          ),
+        6 => array(
+          'var' => 'tableObj',
+          'type' => TType::STRING,
+          ),
+        7 => array(
+          'var' => 'partitionObj',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['writeId'])) {
+        $this->writeId = $vals['writeId'];
+      }
+      if (isset($vals['database'])) {
+        $this->database = $vals['database'];
+      }
+      if (isset($vals['table'])) {
+        $this->table = $vals['table'];
+      }
+      if (isset($vals['files'])) {
+        $this->files = $vals['files'];
+      }
+      if (isset($vals['partition'])) {
+        $this->partition = $vals['partition'];
+      }
+      if (isset($vals['tableObj'])) {
+        $this->tableObj = $vals['tableObj'];
+      }
+      if (isset($vals['partitionObj'])) {
+        $this->partitionObj = $vals['partitionObj'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'WriteEventInfo';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->writeId);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->database);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->table);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->files);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->partition);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableObj);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->partitionObj);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('WriteEventInfo');
+    if ($this->writeId !== null) {
+      $xfer += $output->writeFieldBegin('writeId', TType::I64, 1);
+      $xfer += $output->writeI64($this->writeId);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->database !== null) {
+      $xfer += $output->writeFieldBegin('database', TType::STRING, 2);
+      $xfer += $output->writeString($this->database);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->table !== null) {
+      $xfer += $output->writeFieldBegin('table', TType::STRING, 3);
+      $xfer += $output->writeString($this->table);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->files !== null) {
+      $xfer += $output->writeFieldBegin('files', TType::STRING, 4);
+      $xfer += $output->writeString($this->files);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partition !== null) {
+      $xfer += $output->writeFieldBegin('partition', TType::STRING, 5);
+      $xfer += $output->writeString($this->partition);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableObj !== null) {
+      $xfer += $output->writeFieldBegin('tableObj', TType::STRING, 6);
+      $xfer += $output->writeString($this->tableObj);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partitionObj !== null) {
+      $xfer += $output->writeFieldBegin('partitionObj', TType::STRING, 7);
+      $xfer += $output->writeString($this->partitionObj);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -16785,14 +17049,14 @@ class ReplTblWriteIdStateRequest {
         case 6:
           if ($ftype == TType::LST) {
             $this->partNames = array();
-            $_size523 = 0;
-            $_etype526 = 0;
-            $xfer += $input->readListBegin($_etype526, $_size523);
-            for ($_i527 = 0; $_i527 < $_size523; ++$_i527)
+            $_size530 = 0;
+            $_etype533 = 0;
+            $xfer += $input->readListBegin($_etype533, $_size530);
+            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
             {
-              $elem528 = null;
-              $xfer += $input->readString($elem528);
-              $this->partNames []= $elem528;
+              $elem535 = null;
+              $xfer += $input->readString($elem535);
+              $this->partNames []= $elem535;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16845,9 +17109,9 @@ class ReplTblWriteIdStateRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->partNames));
         {
-          foreach ($this->partNames as $iter529)
+          foreach ($this->partNames as $iter536)
           {
-            $xfer += $output->writeString($iter529);
+            $xfer += $output->writeString($iter536);
           }
         }
         $output->writeListEnd();
@@ -16922,14 +17186,14 @@ class GetValidWriteIdsRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fullTableNames = array();
-            $_size530 = 0;
-            $_etype533 = 0;
-            $xfer += $input->readListBegin($_etype533, $_size530);
-            for ($_i534 = 0; $_i534 < $_size530; ++$_i534)
+            $_size537 = 0;
+            $_etype540 = 0;
+            $xfer += $input->readListBegin($_etype540, $_size537);
+            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
             {
-              $elem535 = null;
-              $xfer += $input->readString($elem535);
-              $this->fullTableNames []= $elem535;
+              $elem542 = null;
+              $xfer += $input->readString($elem542);
+              $this->fullTableNames []= $elem542;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -16964,9 +17228,9 @@ class GetValidWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->fullTableNames));
         {
-          foreach ($this->fullTableNames as $iter536)
+          foreach ($this->fullTableNames as $iter543)
           {
-            $xfer += $output->writeString($iter536);
+            $xfer += $output->writeString($iter543);
           }
         }
         $output->writeListEnd();
@@ -17093,14 +17357,14 @@ class TableValidWriteIds {
         case 3:
           if ($ftype == TType::LST) {
             $this->invalidWriteIds = array();
-            $_size537 = 0;
-            $_etype540 = 0;
-            $xfer += $input->readListBegin($_etype540, $_size537);
-            for ($_i541 = 0; $_i541 < $_size537; ++$_i541)
+            $_size544 = 0;
+            $_etype547 = 0;
+            $xfer += $input->readListBegin($_etype547, $_size544);
+            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
             {
-              $elem542 = null;
-              $xfer += $input->readI64($elem542);
-              $this->invalidWriteIds []= $elem542;
+              $elem549 = null;
+              $xfer += $input->readI64($elem549);
+              $this->invalidWriteIds []= $elem549;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17152,9 +17416,9 @@ class TableValidWriteIds {
       {
         $output->writeListBegin(TType::I64, count($this->invalidWriteIds));
         {
-          foreach ($this->invalidWriteIds as $iter543)
+          foreach ($this->invalidWriteIds as $iter550)
           {
-            $xfer += $output->writeI64($iter543);
+            $xfer += $output->writeI64($iter550);
           }
         }
         $output->writeListEnd();
@@ -17229,15 +17493,15 @@ class GetValidWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->tblValidWriteIds = array();
-            $_size544 = 0;
-            $_etype547 = 0;
-            $xfer += $input->readListBegin($_etype547, $_size544);
-            for ($_i548 = 0; $_i548 < $_size544; ++$_i548)
+            $_size551 = 0;
+            $_etype554 = 0;
+            $xfer += $input->readListBegin($_etype554, $_size551);
+            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
             {
-              $elem549 = null;
-              $elem549 = new \metastore\TableValidWriteIds();
-              $xfer += $elem549->read($input);
-              $this->tblValidWriteIds []= $elem549;
+              $elem556 = null;
+              $elem556 = new \metastore\TableValidWriteIds();
+              $xfer += $elem556->read($input);
+              $this->tblValidWriteIds []= $elem556;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17265,9 +17529,9 @@ class GetValidWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tblValidWriteIds));
         {
-          foreach ($this->tblValidWriteIds as $iter550)
+          foreach ($this->tblValidWriteIds as $iter557)
           {
-            $xfer += $iter550->write($output);
+            $xfer += $iter557->write($output);
           }
         }
         $output->writeListEnd();
@@ -17394,14 +17658,14 @@ class AllocateTableWriteIdsRequest {
         case 3:
           if ($ftype == TType::LST) {
             $this->txnIds = array();
-            $_size551 = 0;
-            $_etype554 = 0;
-            $xfer += $input->readListBegin($_etype554, $_size551);
-            for ($_i555 = 0; $_i555 < $_size551; ++$_i555)
+            $_size558 = 0;
+            $_etype561 = 0;
+            $xfer += $input->readListBegin($_etype561, $_size558);
+            for ($_i562 = 0; $_i562 < $_size558; ++$_i562)
             {
-              $elem556 = null;
-              $xfer += $input->readI64($elem556);
-              $this->txnIds []= $elem556;
+              $elem563 = null;
+              $xfer += $input->readI64($elem563);
+              $this->txnIds []= $elem563;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17418,15 +17682,15 @@ class AllocateTableWriteIdsRequest {
         case 5:
           if ($ftype == TType::LST) {
             $this->srcTxnToWriteIdList = array();
-            $_size557 = 0;
-            $_etype560 = 0;
-            $xfer += $input->readListBegin($_etype560, $_size557);
-            for ($_i561 = 0; $_i561 < $_size557; ++$_i561)
+            $_size564 = 0;
+            $_etype567 = 0;
+            $xfer += $input->readListBegin($_etype567, $_size564);
+            for ($_i568 = 0; $_i568 < $_size564; ++$_i568)
             {
-              $elem562 = null;
-              $elem562 = new \metastore\TxnToWriteId();
-              $xfer += $elem562->read($input);
-              $this->srcTxnToWriteIdList []= $elem562;
+              $elem569 = null;
+              $elem569 = new \metastore\TxnToWriteId();
+              $xfer += $elem569->read($input);
+              $this->srcTxnToWriteIdList []= $elem569;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -17464,9 +17728,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::I64, count($this->txnIds));
         {
-          foreach ($this->txnIds as $iter563)
+          foreach ($this->txnIds as $iter570)
           {
-            $xfer += $output->writeI64($iter563);
+            $xfer += $output->writeI64($iter570);
           }
         }
         $output->writeListEnd();
@@ -17486,9 +17750,9 @@ class AllocateTableWriteIdsRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->srcTxnToWriteIdList));
         {
-          foreach ($this->srcTxnToWriteIdList as $iter564)
+          foreach ($this->srcTxnToWriteIdList as $iter571)
           {
-            $xfer += $iter564->write($output);
+            $xfer += $iter571->write($output);
           }
         }
         $output->writeListEnd();
@@ -17651,16 +17915,16 @@ class AllocateTableWriteIdsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->txnToWriteIds = array();
-            $_size565 = 0;
-            $_etype568 = 0;
-            $xfer += $input->readListBegin($_etype568, $_size565);
-            for ($_i569 = 0; $_i569 < $_size565; ++$_i569)
+            $_size572 = 0;
+            $_etype575 = 0;
+            $xfer += $input->readListBegin($_etype575, $_size572);
+            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
             {
-              $elem570 = null;
-              $elem570 = new \metastore\TxnToWriteId();
-              $xfer += $elem570->read($input);
-              $this->txnToWriteIds []= $elem570;
-            }
+              $elem577 = null;
+              $elem577 = new \metastore\TxnToWriteId();
+              $xfer += $elem577->read($input);
+              $this->txnToWriteIds []= $elem577;
+            }
             $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
@@ -17687,9 +17951,9 @@ class AllocateTableWriteIdsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->txnToWriteIds));
         {
-          foreach ($this->txnToWriteIds as $iter571)
+          foreach ($this->txnToWriteIds as $iter578)
           {
-            $xfer += $iter571->write($output);
+            $xfer += $iter578->write($output);
           }
         }
         $output->writeListEnd();
@@ -18034,15 +18298,15 @@ class LockRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->component = array();
-            $_size572 = 0;
-            $_etype575 = 0;
-            $xfer += $input->readListBegin($_etype575, $_size572);
-            for ($_i576 = 0; $_i576 < $_size572; ++$_i576)
+            $_size579 = 0;
+            $_etype582 = 0;
+            $xfer += $input->readListBegin($_etype582, $_size579);
+            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
             {
-              $elem577 = null;
-              $elem577 = new \metastore\LockComponent();
-              $xfer += $elem577->read($input);
-              $this->component []= $elem577;
+              $elem584 = null;
+              $elem584 = new \metastore\LockComponent();
+              $xfer += $elem584->read($input);
+              $this->component []= $elem584;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -18098,9 +18362,9 @@ class LockRequest {
       {
         $output->writeListBegin(TType::STRUCT, count($this->component));
         {
-          foreach ($this->component as $iter578)
+          foreach ($this->component as $iter585)
           {
-            $xfer += $iter578->write($output);
+            $xfer += $iter585->write($output);
           }
         }
         $output->writeListEnd();
@@ -19043,15 +19307,15 @@ class ShowLocksResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->locks = array();
-            $_size579 = 0;
-            $_etype582 = 0;
-            $xfer += $input->readListBegin($_etype582, $_size579);
-            for ($_i583 = 0; $_i583 < $_size579; ++$_i583)
+            $_size586 = 0;
+            $_etype589 = 0;
+            $xfer += $input->readListBegin($_etype589, $_size586);
+            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
             {
-              $elem584 = null;
-              $elem584 = new \metastore\ShowLocksResponseElement();
-              $xfer += $elem584->read($input);
-              $this->locks []= $elem584;
+              $elem591 = null;
+              $elem591 = new \metastore\ShowLocksResponseElement();
+              $xfer += $elem591->read($input);
+              $this->locks []= $elem591;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -19079,9 +19343,9 @@ class ShowLocksResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->locks));
         {
-          foreach ($this->locks as $iter585)
+          foreach ($this->locks as $iter592)
           {
-            $xfer += $iter585->write($output);
+            $xfer += $iter592->write($output);
           }
         }
         $output->writeListEnd();
@@ -19356,17 +19620,17 @@ class HeartbeatTxnRangeResponse {
         case 1:
           if ($ftype == TType::SET) {
             $this->aborted = array();
-            $_size586 = 0;
-            $_etype589 = 0;
-            $xfer += $input->readSetBegin($_etype589, $_size586);
-            for ($_i590 = 0; $_i590 < $_size586; ++$_i590)
+            $_size593 = 0;
+            $_etype596 = 0;
+            $xfer += $input->readSetBegin($_etype596, $_size593);
+            for ($_i597 = 0; $_i597 < $_size593; ++$_i597)
             {
-              $elem591 = null;
-              $xfer += $input->readI64($elem591);
-              if (is_scalar($elem591)) {
-                $this->aborted[$elem591] = true;
+              $elem598 = null;
+              $xfer += $input->readI64($elem598);
+              if (is_scalar($elem598)) {
+                $this->aborted[$elem598] = true;
               } else {
-                $this->aborted []= $elem591;
+                $this->aborted []= $elem598;
               }
             }
             $xfer += $input->readSetEnd();
@@ -19377,17 +19641,17 @@ class HeartbeatTxnRangeResponse {
         case 2:
           if ($ftype == TType::SET) {
             $this->nosuch = array();
-            $_size592 = 0;
-            $_etype595 = 0;
-            $xfer += $input->readSetBegin($_etype595, $_size592);
-            for ($_i596 = 0; $_i596 < $_size592; ++$_i596)
+            $_size599 = 0;
+            $_etype602 = 0;
+            $xfer += $input->readSetBegin($_etype602, $_size599);
+            for ($_i603 = 0; $_i603 < $_size599; ++$_i603)
             {
-              $elem597 = null;
-              $xfer += $input->readI64($elem597);
-              if (is_scalar($elem597)) {
-                $this->nosuch[$elem597] = true;
+              $elem604 = null;
+              $xfer += $input->readI64($elem604);
+              if (is_scalar($elem604)) {
+                $this->nosuch[$elem604] = true;
               } else {
-                $this->nosuch []= $elem597;
+                $this->nosuch []= $elem604;
               }
             }
             $xfer += $input->readSetEnd();
@@ -19416,12 +19680,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->aborted));
         {
-          foreach ($this->aborted as $iter598 => $iter599)
+          foreach ($this->aborted as $iter605 => $iter606)
           {
-            if (is_scalar($iter599)) {
-            $xfer += $output->writeI64($iter598);
+            if (is_scalar($iter606)) {
+            $xfer += $output->writeI64($iter605);
             } else {
-            $xfer += $output->writeI64($iter599);
+            $xfer += $output->writeI64($iter606);
             }
           }
         }
@@ -19437,12 +19701,12 @@ class HeartbeatTxnRangeResponse {
       {
         $output->writeSetBegin(TType::I64, count($this->nosuch));
         {
-          foreach ($this->nosuch as $iter600 => $iter601)
+          foreach ($this->nosuch as $iter607 => $iter608)
           {
-            if (is_scalar($iter601)) {
-            $xfer += $output->writeI64($iter600);
+            if (is_scalar($iter608)) {
+            $xfer += $output->writeI64($iter607);
             } else {
-            $xfer += $output->writeI64($iter601);
+            $xfer += $output->writeI64($iter608);
             }
           }
         }
@@ -19601,17 +19865,17 @@ class CompactionRequest {
         case 6:
           if ($ftype == TType::MAP) {
             $this->properties = array();
-            $_size602 = 0;
-            $_ktype603 = 0;
-            $_vtype604 = 0;
-            $xfer += $input->readMapBegin($_ktype603, $_vtype604, $_size602);
-            for ($_i606 = 0; $_i606 < $_size602; ++$_i606)
+            $_size609 = 0;
+            $_ktype610 = 0;
+            $_vtype611 = 0;
+            $xfer += $input->readMapBegin($_ktype610, $_vtype611, $_size609);
+            for ($_i613 = 0; $_i613 < $_size609; ++$_i613)
             {
-              $key607 = '';
-              $val608 = '';
-              $xfer += $input->readString($key607);
-              $xfer += $input->readString($val608);
-              $this->properties[$key607] = $val608;
+              $key614 = '';
+              $val615 = '';
+              $xfer += $input->readString($key614);
+              $xfer += $input->readString($val615);
+              $this->properties[$key614] = $val615;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -19664,10 +19928,10 @@ class CompactionRequest {
       {
         $output->writeMapBegin(TType::STRING, TType::STRING, count($this->properties));
         {
-          foreach ($this->properties as $kiter609 => $viter610)
+          foreach ($this->properties as $kiter616 => $viter617)
           {
-            $xfer += $output->writeString($kiter609);
-            $xfer += $output->writeString($viter610);
+            $xfer += $output->writeString($kiter616);
+            $xfer += $output->writeString($viter617);
           }
         }
         $output->writeMapEnd();
@@ -20254,15 +20518,15 @@ class ShowCompactResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->compacts = array();
-            $_size611 = 0;
-            $_etype614 = 0;
-            $xfer += $input->readListBegin($_etype614, $_size611);
-            for ($_i615 = 0; $_i615 < $_size611; ++$_i615)
+            $_size618 = 0;
+            $_etype621 = 0;
+            $xfer += $input->readListBegin($_etype621, $_size618);
+            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
             {
-              $elem616 = null;
-              $elem616 = new \metastore\ShowCompactResponseElement();
-              $xfer += $elem616->read($input);
-              $this->compacts []= $elem616;
+              $elem623 = null;
+              $elem623 = new \metastore\ShowCompactResponseElement();
+              $xfer += $elem623->read($input);
+              $this->compacts []= $elem623;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20290,9 +20554,9 @@ class ShowCompactResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->compacts));
         {
-          foreach ($this->compacts as $iter617)
+          foreach ($this->compacts as $iter624)
           {
-            $xfer += $iter617->write($output);
+            $xfer += $iter624->write($output);
           }
         }
         $output->writeListEnd();
@@ -20439,14 +20703,14 @@ class AddDynamicPartitions {
         case 5:
           if ($ftype == TType::LST) {
             $this->partitionnames = array();
-            $_size618 = 0;
-            $_etype621 = 0;
-            $xfer += $input->readListBegin($_etype621, $_size618);
-            for ($_i622 = 0; $_i622 < $_size618; ++$_i622)
+            $_size625 = 0;
+            $_etype628 = 0;
+            $xfer += $input->readListBegin($_etype628, $_size625);
+            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
             {
-              $elem623 = null;
-              $xfer += $input->readString($elem623);
-              $this->partitionnames []= $elem623;
+              $elem630 = null;
+              $xfer += $input->readString($elem630);
+              $this->partitionnames []= $elem630;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -20501,9 +20765,9 @@ class AddDynamicPartitions {
       {
         $output->writeListBegin(TType::STRING, count($this->partitionnames));
         {
-          foreach ($this->partitionnames as $iter624)
+          foreach ($this->partitionnames as $iter631)
           {
-            $xfer += $output->writeString($iter624);
+            $xfer += $output->writeString($iter631);
           }
         }
         $output->writeListEnd();
@@ -20827,17 +21091,17 @@ class CreationMetadata {
         case 4:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size625 = 0;
-            $_etype628 = 0;
-            $xfer += $input->readSetBegin($_etype628, $_size625);
-            for ($_i629 = 0; $_i629 < $_size625; ++$_i629)
+            $_size632 = 0;
+            $_etype635 = 0;
+            $xfer += $input->readSetBegin($_etype635, $_size632);
+            for ($_i636 = 0; $_i636 < $_size632; ++$_i636)
             {
-              $elem630 = null;
-              $xfer += $input->readString($elem630);
-              if (is_scalar($elem630)) {
-                $this->tablesUsed[$elem630] = true;
+              $elem637 = null;
+              $xfer += $input->readString($elem637);
+              if (is_scalar($elem637)) {
+                $this->tablesUsed[$elem637] = true;
               } else {
-                $this->tablesUsed []= $elem630;
+                $this->tablesUsed []= $elem637;
               }
             }
             $xfer += $input->readSetEnd();
@@ -20888,12 +21152,12 @@ class CreationMetadata {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter631 => $iter632)
+          foreach ($this->tablesUsed as $iter638 => $iter639)
           {
-            if (is_scalar($iter632)) {
-            $xfer += $output->writeString($iter631);
+            if (is_scalar($iter639)) {
+            $xfer += $output->writeString($iter638);
             } else {
-            $xfer += $output->writeString($iter632);
+            $xfer += $output->writeString($iter639);
             }
           }
         }
@@ -21298,15 +21562,15 @@ class NotificationEventResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->events = array();
-            $_size633 = 0;
-            $_etype636 = 0;
-            $xfer += $input->readListBegin($_etype636, $_size633);
-            for ($_i637 = 0; $_i637 < $_size633; ++$_i637)
+            $_size640 = 0;
+            $_etype643 = 0;
+            $xfer += $input->readListBegin($_etype643, $_size640);
+            for ($_i644 = 0; $_i644 < $_size640; ++$_i644)
             {
-              $elem638 = null;
-              $elem638 = new \metastore\NotificationEvent();
-              $xfer += $elem638->read($input);
-              $this->events []= $elem638;
+              $elem645 = null;
+              $elem645 = new \metastore\NotificationEvent();
+              $xfer += $elem645->read($input);
+              $this->events []= $elem645;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21334,9 +21598,9 @@ class NotificationEventResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->events));
         {
-          foreach ($this->events as $iter639)
+          foreach ($this->events as $iter646)
           {
-            $xfer += $iter639->write($output);
+            $xfer += $iter646->write($output);
           }
         }
         $output->writeListEnd();
@@ -21636,6 +21900,10 @@ class InsertEventRequestData {
    * @var string[]
    */
   public $filesAddedChecksum = null;
+  /**
+   * @var string[]
+   */
+  public $subDirectoryList = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -21660,6 +21928,14 @@ class InsertEventRequestData {
             'type' => TType::STRING,
             ),
           ),
+        4 => array(
+          'var' => 'subDirectoryList',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
         );
     }
     if (is_array($vals)) {
@@ -21672,6 +21948,9 @@ class InsertEventRequestData {
       if (isset($vals['filesAddedChecksum'])) {
         $this->filesAddedChecksum = $vals['filesAddedChecksum'];
       }
+      if (isset($vals['subDirectoryList'])) {
+        $this->subDirectoryList = $vals['subDirectoryList'];
+      }
     }
   }
 
@@ -21704,14 +21983,14 @@ class InsertEventRequestData {
         case 2:
           if ($ftype == TType::LST) {
             $this->filesAdded = array();
-            $_size640 = 0;
-            $_etype643 = 0;
-            $xfer += $input->readListBegin($_etype643, $_size640);
-            for ($_i644 = 0; $_i644 < $_size640; ++$_i644)
+            $_size647 = 0;
+            $_etype650 = 0;
+            $xfer += $input->readListBegin($_etype650, $_size647);
+            for ($_i651 = 0; $_i651 < $_size647; ++$_i651)
             {
-              $elem645 = null;
-              $xfer += $input->readString($elem645);
-              $this->filesAdded []= $elem645;
+              $elem652 = null;
+              $xfer += $input->readString($elem652);
+              $this->filesAdded []= $elem652;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21721,14 +22000,31 @@ class InsertEventRequestData {
         case 3:
           if ($ftype == TType::LST) {
             $this->filesAddedChecksum = array();
-            $_size646 = 0;
-            $_etype649 = 0;
-            $xfer += $input->readListBegin($_etype649, $_size646);
-            for ($_i650 = 0; $_i650 < $_size646; ++$_i650)
+            $_size653 = 0;
+            $_etype656 = 0;
+            $xfer += $input->readListBegin($_etype656, $_size653);
+            for ($_i657 = 0; $_i657 < $_size653; ++$_i657)
+            {
+              $elem658 = null;
+              $xfer += $input->readString($elem658);
+              $this->filesAddedChecksum []= $elem658;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::LST) {
+            $this->subDirectoryList = array();
+            $_size659 = 0;
+            $_etype662 = 0;
+            $xfer += $input->readListBegin($_etype662, $_size659);
+            for ($_i663 = 0; $_i663 < $_size659; ++$_i663)
             {
-              $elem651 = null;
-              $xfer += $input->readString($elem651);
-              $this->filesAddedChecksum []= $elem651;
+              $elem664 = null;
+              $xfer += $input->readString($elem664);
+              $this->subDirectoryList []= $elem664;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -21761,9 +22057,9 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAdded));
         {
-          foreach ($this->filesAdded as $iter652)
+          foreach ($this->filesAdded as $iter665)
           {
-            $xfer += $output->writeString($iter652);
+            $xfer += $output->writeString($iter665);
           }
         }
         $output->writeListEnd();
@@ -21778,9 +22074,26 @@ class InsertEventRequestData {
       {
         $output->writeListBegin(TType::STRING, count($this->filesAddedChecksum));
         {
-          foreach ($this->filesAddedChecksum as $iter653)
+          foreach ($this->filesAddedChecksum as $iter666)
+          {
+            $xfer += $output->writeString($iter666);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->subDirectoryList !== null) {
+      if (!is_array($this->subDirectoryList)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('subDirectoryList', TType::LST, 4);
+      {
+        $output->writeListBegin(TType::STRING, count($this->subDirectoryList));
+        {
+          foreach ($this->subDirectoryList as $iter667)
           {
-            $xfer += $output->writeString($iter653);
+            $xfer += $output->writeString($iter667);
           }
         }
         $output->writeListEnd();
@@ -21794,33 +22107,320 @@ class InsertEventRequestData {
 
 }
 
-class FireEventRequestData {
+class FireEventRequestData {
+  static $_TSPEC;
+
+  /**
+   * @var \metastore\InsertEventRequestData
+   */
+  public $insertData = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'insertData',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InsertEventRequestData',
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['insertData'])) {
+        $this->insertData = $vals['insertData'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'FireEventRequestData';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::STRUCT) {
+            $this->insertData = new \metastore\InsertEventRequestData();
+            $xfer += $this->insertData->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('FireEventRequestData');
+    if ($this->insertData !== null) {
+      if (!is_object($this->insertData)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('insertData', TType::STRUCT, 1);
+      $xfer += $this->insertData->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class FireEventRequest {
+  static $_TSPEC;
+
+  /**
+   * @var bool
+   */
+  public $successful = null;
+  /**
+   * @var \metastore\FireEventRequestData
+   */
+  public $data = null;
+  /**
+   * @var string
+   */
+  public $dbName = null;
+  /**
+   * @var string
+   */
+  public $tableName = null;
+  /**
+   * @var string[]
+   */
+  public $partitionVals = null;
+  /**
+   * @var string
+   */
+  public $catName = null;
+
+  public function __construct($vals=null) {
+    if (!isset(self::$_TSPEC)) {
+      self::$_TSPEC = array(
+        1 => array(
+          'var' => 'successful',
+          'type' => TType::BOOL,
+          ),
+        2 => array(
+          'var' => 'data',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\FireEventRequestData',
+          ),
+        3 => array(
+          'var' => 'dbName',
+          'type' => TType::STRING,
+          ),
+        4 => array(
+          'var' => 'tableName',
+          'type' => TType::STRING,
+          ),
+        5 => array(
+          'var' => 'partitionVals',
+          'type' => TType::LST,
+          'etype' => TType::STRING,
+          'elem' => array(
+            'type' => TType::STRING,
+            ),
+          ),
+        6 => array(
+          'var' => 'catName',
+          'type' => TType::STRING,
+          ),
+        );
+    }
+    if (is_array($vals)) {
+      if (isset($vals['successful'])) {
+        $this->successful = $vals['successful'];
+      }
+      if (isset($vals['data'])) {
+        $this->data = $vals['data'];
+      }
+      if (isset($vals['dbName'])) {
+        $this->dbName = $vals['dbName'];
+      }
+      if (isset($vals['tableName'])) {
+        $this->tableName = $vals['tableName'];
+      }
+      if (isset($vals['partitionVals'])) {
+        $this->partitionVals = $vals['partitionVals'];
+      }
+      if (isset($vals['catName'])) {
+        $this->catName = $vals['catName'];
+      }
+    }
+  }
+
+  public function getName() {
+    return 'FireEventRequest';
+  }
+
+  public function read($input)
+  {
+    $xfer = 0;
+    $fname = null;
+    $ftype = 0;
+    $fid = 0;
+    $xfer += $input->readStructBegin($fname);
+    while (true)
+    {
+      $xfer += $input->readFieldBegin($fname, $ftype, $fid);
+      if ($ftype == TType::STOP) {
+        break;
+      }
+      switch ($fid)
+      {
+        case 1:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->successful);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 2:
+          if ($ftype == TType::STRUCT) {
+            $this->data = new \metastore\FireEventRequestData();
+            $xfer += $this->data->read($input);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 3:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->dbName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 4:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->tableName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 5:
+          if ($ftype == TType::LST) {
+            $this->partitionVals = array();
+            $_size668 = 0;
+            $_etype671 = 0;
+            $xfer += $input->readListBegin($_etype671, $_size668);
+            for ($_i672 = 0; $_i672 < $_size668; ++$_i672)
+            {
+              $elem673 = null;
+              $xfer += $input->readString($elem673);
+              $this->partitionVals []= $elem673;
+            }
+            $xfer += $input->readListEnd();
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 6:
+          if ($ftype == TType::STRING) {
+            $xfer += $input->readString($this->catName);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        default:
+          $xfer += $input->skip($ftype);
+          break;
+      }
+      $xfer += $input->readFieldEnd();
+    }
+    $xfer += $input->readStructEnd();
+    return $xfer;
+  }
+
+  public function write($output) {
+    $xfer = 0;
+    $xfer += $output->writeStructBegin('FireEventRequest');
+    if ($this->successful !== null) {
+      $xfer += $output->writeFieldBegin('successful', TType::BOOL, 1);
+      $xfer += $output->writeBool($this->successful);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->data !== null) {
+      if (!is_object($this->data)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('data', TType::STRUCT, 2);
+      $xfer += $this->data->write($output);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->dbName !== null) {
+      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 3);
+      $xfer += $output->writeString($this->dbName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->tableName !== null) {
+      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 4);
+      $xfer += $output->writeString($this->tableName);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->partitionVals !== null) {
+      if (!is_array($this->partitionVals)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 5);
+      {
+        $output->writeListBegin(TType::STRING, count($this->partitionVals));
+        {
+          foreach ($this->partitionVals as $iter674)
+          {
+            $xfer += $output->writeString($iter674);
+          }
+        }
+        $output->writeListEnd();
+      }
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->catName !== null) {
+      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
+      $xfer += $output->writeString($this->catName);
+      $xfer += $output->writeFieldEnd();
+    }
+    $xfer += $output->writeFieldStop();
+    $xfer += $output->writeStructEnd();
+    return $xfer;
+  }
+
+}
+
+class FireEventResponse {
   static $_TSPEC;
 
-  /**
-   * @var \metastore\InsertEventRequestData
-   */
-  public $insertData = null;
 
-  public function __construct($vals=null) {
+  public function __construct() {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
-        1 => array(
-          'var' => 'insertData',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\InsertEventRequestData',
-          ),
         );
     }
-    if (is_array($vals)) {
-      if (isset($vals['insertData'])) {
-        $this->insertData = $vals['insertData'];
-      }
-    }
   }
 
   public function getName() {
-    return 'FireEventRequestData';
+    return 'FireEventResponse';
   }
 
   public function read($input)
@@ -21838,14 +22438,6 @@ class FireEventRequestData {
       }
       switch ($fid)
       {
-        case 1:
-          if ($ftype == TType::STRUCT) {
-            $this->insertData = new \metastore\InsertEventRequestData();
-            $xfer += $this->insertData->read($input);
-          } else {
-            $xfer += $input->skip($ftype);
-          }
-          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -21858,15 +22450,7 @@ class FireEventRequestData {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('FireEventRequestData');
-    if ($this->insertData !== null) {
-      if (!is_object($this->insertData)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('insertData', TType::STRUCT, 1);
-      $xfer += $this->insertData->write($output);
-      $xfer += $output->writeFieldEnd();
-    }
+    $xfer += $output->writeStructBegin('FireEventResponse');
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -21874,55 +22458,59 @@ class FireEventRequestData {
 
 }
 
-class FireEventRequest {
+class WriteNotificationLogRequest {
   static $_TSPEC;
 
   /**
-   * @var bool
+   * @var int
    */
-  public $successful = null;
+  public $txnId = null;
   /**
-   * @var \metastore\FireEventRequestData
+   * @var int
    */
-  public $data = null;
+  public $writeId = null;
   /**
    * @var string
    */
-  public $dbName = null;
+  public $db = null;
   /**
    * @var string
    */
-  public $tableName = null;
+  public $table = null;
   /**
-   * @var string[]
+   * @var \metastore\InsertEventRequestData
    */
-  public $partitionVals = null;
+  public $fileInfo = null;
   /**
-   * @var string
+   * @var string[]
    */
-  public $catName = null;
+  public $partitionVals = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
       self::$_TSPEC = array(
         1 => array(
-          'var' => 'successful',
-          'type' => TType::BOOL,
+          'var' => 'txnId',
+          'type' => TType::I64,
           ),
         2 => array(
-          'var' => 'data',
-          'type' => TType::STRUCT,
-          'class' => '\metastore\FireEventRequestData',
+          'var' => 'writeId',
+          'type' => TType::I64,
           ),
         3 => array(
-          'var' => 'dbName',
+          'var' => 'db',
           'type' => TType::STRING,
           ),
         4 => array(
-          'var' => 'tableName',
+          'var' => 'table',
           'type' => TType::STRING,
           ),
         5 => array(
+          'var' => 'fileInfo',
+          'type' => TType::STRUCT,
+          'class' => '\metastore\InsertEventRequestData',
+          ),
+        6 => array(
           'var' => 'partitionVals',
           'type' => TType::LST,
           'etype' => TType::STRING,
@@ -21930,36 +22518,32 @@ class FireEventRequest {
             'type' => TType::STRING,
             ),
           ),
-        6 => array(
-          'var' => 'catName',
-          'type' => TType::STRING,
-          ),
         );
     }
     if (is_array($vals)) {
-      if (isset($vals['successful'])) {
-        $this->successful = $vals['successful'];
+      if (isset($vals['txnId'])) {
+        $this->txnId = $vals['txnId'];
       }
-      if (isset($vals['data'])) {
-        $this->data = $vals['data'];
+      if (isset($vals['writeId'])) {
+        $this->writeId = $vals['writeId'];
       }
-      if (isset($vals['dbName'])) {
-        $this->dbName = $vals['dbName'];
+      if (isset($vals['db'])) {
+        $this->db = $vals['db'];
       }
-      if (isset($vals['tableName'])) {
-        $this->tableName = $vals['tableName'];
+      if (isset($vals['table'])) {
+        $this->table = $vals['table'];
+      }
+      if (isset($vals['fileInfo'])) {
+        $this->fileInfo = $vals['fileInfo'];
       }
       if (isset($vals['partitionVals'])) {
         $this->partitionVals = $vals['partitionVals'];
       }
-      if (isset($vals['catName'])) {
-        $this->catName = $vals['catName'];
-      }
     }
   }
 
   public function getName() {
-    return 'FireEventRequest';
+    return 'WriteNotificationLogRequest';
   }
 
   public function read($input)
@@ -21978,54 +22562,54 @@ class FireEventRequest {
       switch ($fid)
       {
         case 1:
-          if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->successful);
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->txnId);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 2:
-          if ($ftype == TType::STRUCT) {
-            $this->data = new \metastore\FireEventRequestData();
-            $xfer += $this->data->read($input);
+          if ($ftype == TType::I64) {
+            $xfer += $input->readI64($this->writeId);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 3:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->dbName);
+            $xfer += $input->readString($this->db);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 4:
           if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->tableName);
+            $xfer += $input->readString($this->table);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 5:
-          if ($ftype == TType::LST) {
-            $this->partitionVals = array();
-            $_size654 = 0;
-            $_etype657 = 0;
-            $xfer += $input->readListBegin($_etype657, $_size654);
-            for ($_i658 = 0; $_i658 < $_size654; ++$_i658)
-            {
-              $elem659 = null;
-              $xfer += $input->readString($elem659);
-              $this->partitionVals []= $elem659;
-            }
-            $xfer += $input->readListEnd();
+          if ($ftype == TType::STRUCT) {
+            $this->fileInfo = new \metastore\InsertEventRequestData();
+            $xfer += $this->fileInfo->read($input);
           } else {
             $xfer += $input->skip($ftype);
           }
           break;
         case 6:
-          if ($ftype == TType::STRING) {
-            $xfer += $input->readString($this->catName);
+          if ($ftype == TType::LST) {
+            $this->partitionVals = array();
+            $_size675 = 0;
+            $_etype678 = 0;
+            $xfer += $input->readListBegin($_etype678, $_size675);
+            for ($_i679 = 0; $_i679 < $_size675; ++$_i679)
+            {
+              $elem680 = null;
+              $xfer += $input->readString($elem680);
+              $this->partitionVals []= $elem680;
+            }
+            $xfer += $input->readListEnd();
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -22042,52 +22626,52 @@ class FireEventRequest {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('FireEventRequest');
-    if ($this->successful !== null) {
-      $xfer += $output->writeFieldBegin('successful', TType::BOOL, 1);
-      $xfer += $output->writeBool($this->successful);
+    $xfer += $output->writeStructBegin('WriteNotificationLogRequest');
+    if ($this->txnId !== null) {
+      $xfer += $output->writeFieldBegin('txnId', TType::I64, 1);
+      $xfer += $output->writeI64($this->txnId);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->data !== null) {
-      if (!is_object($this->data)) {
-        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
-      }
-      $xfer += $output->writeFieldBegin('data', TType::STRUCT, 2);
-      $xfer += $this->data->write($output);
+    if ($this->writeId !== null) {
+      $xfer += $output->writeFieldBegin('writeId', TType::I64, 2);
+      $xfer += $output->writeI64($this->writeId);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->dbName !== null) {
-      $xfer += $output->writeFieldBegin('dbName', TType::STRING, 3);
-      $xfer += $output->writeString($this->dbName);
+    if ($this->db !== null) {
+      $xfer += $output->writeFieldBegin('db', TType::STRING, 3);
+      $xfer += $output->writeString($this->db);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->tableName !== null) {
-      $xfer += $output->writeFieldBegin('tableName', TType::STRING, 4);
-      $xfer += $output->writeString($this->tableName);
+    if ($this->table !== null) {
+      $xfer += $output->writeFieldBegin('table', TType::STRING, 4);
+      $xfer += $output->writeString($this->table);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->fileInfo !== null) {
+      if (!is_object($this->fileInfo)) {
+        throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
+      }
+      $xfer += $output->writeFieldBegin('fileInfo', TType::STRUCT, 5);
+      $xfer += $this->fileInfo->write($output);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->partitionVals !== null) {
       if (!is_array($this->partitionVals)) {
         throw new TProtocolException('Bad type in structure.', TProtocolException::INVALID_DATA);
       }
-      $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 5);
+      $xfer += $output->writeFieldBegin('partitionVals', TType::LST, 6);
       {
         $output->writeListBegin(TType::STRING, count($this->partitionVals));
         {
-          foreach ($this->partitionVals as $iter660)
+          foreach ($this->partitionVals as $iter681)
           {
-            $xfer += $output->writeString($iter660);
+            $xfer += $output->writeString($iter681);
           }
         }
         $output->writeListEnd();
       }
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->catName !== null) {
-      $xfer += $output->writeFieldBegin('catName', TType::STRING, 6);
-      $xfer += $output->writeString($this->catName);
-      $xfer += $output->writeFieldEnd();
-    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -22095,7 +22679,7 @@ class FireEventRequest {
 
 }
 
-class FireEventResponse {
+class WriteNotificationLogResponse {
   static $_TSPEC;
 
 
@@ -22107,7 +22691,7 @@ class FireEventResponse {
   }
 
   public function getName() {
-    return 'FireEventResponse';
+    return 'WriteNotificationLogResponse';
   }
 
   public function read($input)
@@ -22137,7 +22721,7 @@ class FireEventResponse {
 
   public function write($output) {
     $xfer = 0;
-    $xfer += $output->writeStructBegin('FireEventResponse');
+    $xfer += $output->writeStructBegin('WriteNotificationLogResponse');
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -22309,18 +22893,18 @@ class GetFileMetadataByExprResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size661 = 0;
-            $_ktype662 = 0;
-            $_vtype663 = 0;
-            $xfer += $input->readMapBegin($_ktype662, $_vtype663, $_size661);
-            for ($_i665 = 0; $_i665 < $_size661; ++$_i665)
+            $_size682 = 0;
+            $_ktype683 = 0;
+            $_vtype684 = 0;
+            $xfer += $input->readMapBegin($_ktype683, $_vtype684, $_size682);
+            for ($_i686 = 0; $_i686 < $_size682; ++$_i686)
             {
-              $key666 = 0;
-              $val667 = new \metastore\MetadataPpdResult();
-              $xfer += $input->readI64($key666);
-              $val667 = new \metastore\MetadataPpdResult();
-              $xfer += $val667->read($input);
-              $this->metadata[$key666] = $val667;
+              $key687 = 0;
+              $val688 = new \metastore\MetadataPpdResult();
+              $xfer += $input->readI64($key687);
+              $val688 = new \metastore\MetadataPpdResult();
+              $xfer += $val688->read($input);
+              $this->metadata[$key687] = $val688;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22355,10 +22939,10 @@ class GetFileMetadataByExprResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRUCT, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter668 => $viter669)
+          foreach ($this->metadata as $kiter689 => $viter690)
           {
-            $xfer += $output->writeI64($kiter668);
-            $xfer += $viter669->write($output);
+            $xfer += $output->writeI64($kiter689);
+            $xfer += $viter690->write($output);
           }
         }
         $output->writeMapEnd();
@@ -22460,14 +23044,14 @@ class GetFileMetadataByExprRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size670 = 0;
-            $_etype673 = 0;
-            $xfer += $input->readListBegin($_etype673, $_size670);
-            for ($_i674 = 0; $_i674 < $_size670; ++$_i674)
+            $_size691 = 0;
+            $_etype694 = 0;
+            $xfer += $input->readListBegin($_etype694, $_size691);
+            for ($_i695 = 0; $_i695 < $_size691; ++$_i695)
             {
-              $elem675 = null;
-              $xfer += $input->readI64($elem675);
-              $this->fileIds []= $elem675;
+              $elem696 = null;
+              $xfer += $input->readI64($elem696);
+              $this->fileIds []= $elem696;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22516,9 +23100,9 @@ class GetFileMetadataByExprRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter676)
+          foreach ($this->fileIds as $iter697)
           {
-            $xfer += $output->writeI64($iter676);
+            $xfer += $output->writeI64($iter697);
           }
         }
         $output->writeListEnd();
@@ -22612,17 +23196,17 @@ class GetFileMetadataResult {
         case 1:
           if ($ftype == TType::MAP) {
             $this->metadata = array();
-            $_size677 = 0;
-            $_ktype678 = 0;
-            $_vtype679 = 0;
-            $xfer += $input->readMapBegin($_ktype678, $_vtype679, $_size677);
-            for ($_i681 = 0; $_i681 < $_size677; ++$_i681)
+            $_size698 = 0;
+            $_ktype699 = 0;
+            $_vtype700 = 0;
+            $xfer += $input->readMapBegin($_ktype699, $_vtype700, $_size698);
+            for ($_i702 = 0; $_i702 < $_size698; ++$_i702)
             {
-              $key682 = 0;
-              $val683 = '';
-              $xfer += $input->readI64($key682);
-              $xfer += $input->readString($val683);
-              $this->metadata[$key682] = $val683;
+              $key703 = 0;
+              $val704 = '';
+              $xfer += $input->readI64($key703);
+              $xfer += $input->readString($val704);
+              $this->metadata[$key703] = $val704;
             }
             $xfer += $input->readMapEnd();
           } else {
@@ -22657,10 +23241,10 @@ class GetFileMetadataResult {
       {
         $output->writeMapBegin(TType::I64, TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $kiter684 => $viter685)
+          foreach ($this->metadata as $kiter705 => $viter706)
           {
-            $xfer += $output->writeI64($kiter684);
-            $xfer += $output->writeString($viter685);
+            $xfer += $output->writeI64($kiter705);
+            $xfer += $output->writeString($viter706);
           }
         }
         $output->writeMapEnd();
@@ -22729,14 +23313,14 @@ class GetFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size686 = 0;
-            $_etype689 = 0;
-            $xfer += $input->readListBegin($_etype689, $_size686);
-            for ($_i690 = 0; $_i690 < $_size686; ++$_i690)
+            $_size707 = 0;
+            $_etype710 = 0;
+            $xfer += $input->readListBegin($_etype710, $_size707);
+            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
             {
-              $elem691 = null;
-              $xfer += $input->readI64($elem691);
-              $this->fileIds []= $elem691;
+              $elem712 = null;
+              $xfer += $input->readI64($elem712);
+              $this->fileIds []= $elem712;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22764,9 +23348,9 @@ class GetFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter692)
+          foreach ($this->fileIds as $iter713)
           {
-            $xfer += $output->writeI64($iter692);
+            $xfer += $output->writeI64($iter713);
           }
         }
         $output->writeListEnd();
@@ -22906,14 +23490,14 @@ class PutFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size693 = 0;
-            $_etype696 = 0;
-            $xfer += $input->readListBegin($_etype696, $_size693);
-            for ($_i697 = 0; $_i697 < $_size693; ++$_i697)
+            $_size714 = 0;
+            $_etype717 = 0;
+            $xfer += $input->readListBegin($_etype717, $_size714);
+            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
             {
-              $elem698 = null;
-              $xfer += $input->readI64($elem698);
-              $this->fileIds []= $elem698;
+              $elem719 = null;
+              $xfer += $input->readI64($elem719);
+              $this->fileIds []= $elem719;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22923,14 +23507,14 @@ class PutFileMetadataRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->metadata = array();
-            $_size699 = 0;
-            $_etype702 = 0;
-            $xfer += $input->readListBegin($_etype702, $_size699);
-            for ($_i703 = 0; $_i703 < $_size699; ++$_i703)
+            $_size720 = 0;
+            $_etype723 = 0;
+            $xfer += $input->readListBegin($_etype723, $_size720);
+            for ($_i724 = 0; $_i724 < $_size720; ++$_i724)
             {
-              $elem704 = null;
-              $xfer += $input->readString($elem704);
-              $this->metadata []= $elem704;
+              $elem725 = null;
+              $xfer += $input->readString($elem725);
+              $this->metadata []= $elem725;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -22965,9 +23549,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter705)
+          foreach ($this->fileIds as $iter726)
           {
-            $xfer += $output->writeI64($iter705);
+            $xfer += $output->writeI64($iter726);
           }
         }
         $output->writeListEnd();
@@ -22982,9 +23566,9 @@ class PutFileMetadataRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->metadata));
         {
-          foreach ($this->metadata as $iter706)
+          foreach ($this->metadata as $iter727)
           {
-            $xfer += $output->writeString($iter706);
+            $xfer += $output->writeString($iter727);
           }
         }
         $output->writeListEnd();
@@ -23103,14 +23687,14 @@ class ClearFileMetadataRequest {
         case 1:
           if ($ftype == TType::LST) {
             $this->fileIds = array();
-            $_size707 = 0;
-            $_etype710 = 0;
-            $xfer += $input->readListBegin($_etype710, $_size707);
-            for ($_i711 = 0; $_i711 < $_size707; ++$_i711)
+            $_size728 = 0;
+            $_etype731 = 0;
+            $xfer += $input->readListBegin($_etype731, $_size728);
+            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
             {
-              $elem712 = null;
-              $xfer += $input->readI64($elem712);
-              $this->fileIds []= $elem712;
+              $elem733 = null;
+              $xfer += $input->readI64($elem733);
+              $this->fileIds []= $elem733;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23138,9 +23722,9 @@ class ClearFileMetadataRequest {
       {
         $output->writeListBegin(TType::I64, count($this->fileIds));
         {
-          foreach ($this->fileIds as $iter713)
+          foreach ($this->fileIds as $iter734)
           {
-            $xfer += $output->writeI64($iter713);
+            $xfer += $output->writeI64($iter734);
           }
         }
         $output->writeListEnd();
@@ -23424,15 +24008,15 @@ class GetAllFunctionsResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->functions = array();
-            $_size714 = 0;
-            $_etype717 = 0;
-            $xfer += $input->readListBegin($_etype717, $_size714);
-            for ($_i718 = 0; $_i718 < $_size714; ++$_i718)
+            $_size735 = 0;
+            $_etype738 = 0;
+            $xfer += $input->readListBegin($_etype738, $_size735);
+            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
             {
-              $elem719 = null;
-              $elem719 = new \metastore\Function();
-              $xfer += $elem719->read($input);
-              $this->functions []= $elem719;
+              $elem740 = null;
+              $elem740 = new \metastore\Function();
+              $xfer += $elem740->read($input);
+              $this->functions []= $elem740;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23460,9 +24044,9 @@ class GetAllFunctionsResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->functions));
         {
-          foreach ($this->functions as $iter720)
+          foreach ($this->functions as $iter741)
           {
-            $xfer += $iter720->write($output);
+            $xfer += $iter741->write($output);
           }
         }
         $output->writeListEnd();
@@ -23526,14 +24110,14 @@ class ClientCapabilities {
         case 1:
           if ($ftype == TType::LST) {
             $this->values = array();
-            $_size721 = 0;
-            $_etype724 = 0;
-            $xfer += $input->readListBegin($_etype724, $_size721);
-            for ($_i725 = 0; $_i725 < $_size721; ++$_i725)
+            $_size742 = 0;
+            $_etype745 = 0;
+            $xfer += $input->readListBegin($_etype745, $_size742);
+            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
             {
-              $elem726 = null;
-              $xfer += $input->readI32($elem726);
-              $this->values []= $elem726;
+              $elem747 = null;
+              $xfer += $input->readI32($elem747);
+              $this->values []= $elem747;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23561,9 +24145,9 @@ class ClientCapabilities {
       {
         $output->writeListBegin(TType::I32, count($this->values));
         {
-          foreach ($this->values as $iter727)
+          foreach ($this->values as $iter748)
           {
-            $xfer += $output->writeI32($iter727);
+            $xfer += $output->writeI32($iter748);
           }
         }
         $output->writeListEnd();
@@ -23897,14 +24481,14 @@ class GetTablesRequest {
         case 2:
           if ($ftype == TType::LST) {
             $this->tblNames = array();
-            $_size728 = 0;
-            $_etype731 = 0;
-            $xfer += $input->readListBegin($_etype731, $_size728);
-            for ($_i732 = 0; $_i732 < $_size728; ++$_i732)
+            $_size749 = 0;
+            $_etype752 = 0;
+            $xfer += $input->readListBegin($_etype752, $_size749);
+            for ($_i753 = 0; $_i753 < $_size749; ++$_i753)
             {
-              $elem733 = null;
-              $xfer += $input->readString($elem733);
-              $this->tblNames []= $elem733;
+              $elem754 = null;
+              $xfer += $input->readString($elem754);
+              $this->tblNames []= $elem754;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -23952,9 +24536,9 @@ class GetTablesRequest {
       {
         $output->writeListBegin(TType::STRING, count($this->tblNames));
         {
-          foreach ($this->tblNames as $iter734)
+          foreach ($this->tblNames as $iter755)
           {
-            $xfer += $output->writeString($iter734);
+            $xfer += $output->writeString($iter755);
           }
         }
         $output->writeListEnd();
@@ -24032,15 +24616,15 @@ class GetTablesResult {
         case 1:
           if ($ftype == TType::LST) {
             $this->tables = array();
-            $_size735 = 0;
-            $_etype738 = 0;
-            $xfer += $input->readListBegin($_etype738, $_size735);
-            for ($_i739 = 0; $_i739 < $_size735; ++$_i739)
+            $_size756 = 0;
+            $_etype759 = 0;
+            $xfer += $input->readListBegin($_etype759, $_size756);
+            for ($_i760 = 0; $_i760 < $_size756; ++$_i760)
             {
-              $elem740 = null;
-              $elem740 = new \metastore\Table();
-              $xfer += $elem740->read($input);
-              $this->tables []= $elem740;
+              $elem761 = null;
+              $elem761 = new \metastore\Table();
+              $xfer += $elem761->read($input);
+              $this->tables []= $elem761;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -24068,9 +24652,9 @@ class GetTablesResult {
       {
         $output->writeListBegin(TType::STRUCT, count($this->tables));
         {
-          foreach ($this->tables as $iter741)
+          foreach ($this->tables as $iter762)
           {
-            $xfer += $iter741->write($output);
+            $xfer += $iter762->write($output);
           }
         }
         $output->writeListEnd();
@@ -24482,17 +25066,17 @@ class Materialization {
         case 1:
           if ($ftype == TType::SET) {
             $this->tablesUsed = array();
-            $_size742 = 0;
-            $_etype745 = 0;
-            $xfer += $input->readSetBegin($_etype745, $_size742);
-            for ($_i746 = 0; $_i746 < $_size742; ++$_i746)
+            $_size763 = 0;
+            $_etype766 = 0;
+            $xfer += $input->readSetBegin($_etype766, $_size763);
+            for ($_i767 = 0; $_i767 < $_size763; ++$_i767)
             {
-              $elem747 = null;
-              $xfer += $input->readString($elem747);
-              if (is_scalar($elem747)) {
-                $this->tablesUsed[$elem747] = true;
+              $elem768 = null;
+              $xfer += $input->readString($elem768);
+              if (is_scalar($elem768)) {
+                $this->tablesUsed[$elem768] = true;
               } else {
-                $this->tablesUsed []= $elem747;
+                $this->tablesUsed []= $elem768;
               }
             }
             $xfer += $input->readSetEnd();
@@ -24542,12 +25126,12 @@ class Materialization {
       {
         $output->writeSetBegin(TType::STRING, count($this->tablesUsed));
         {
-          foreach ($this->tablesUsed as $iter748 => $iter749)
+          foreach ($this->tablesUsed as $iter769 => $iter770)
           {
-            if (is_scalar($iter749)) {
-            $xfer += $output->writeString($iter748);
+            if (is_scalar($iter770)) {
+            $xfer += $output->writeString($iter769);
             } else {
-            $xfer += $output->writeString($iter749);
+            $xfer += $output->writeString($iter770);
             }
           }
         }
@@ -25819,15 +26403,15 @@ class WMFullResourcePlan {
         case 2:
           if ($ftype == TType::LST) {
             $this->pools = array();
-            $_size750 = 0;
-            $_etype753 = 0;
-            $xfer += $input->readListBegin($_etype753, $_size750);
-            for ($_i754 = 0; $_i754 < $_size750; ++$_i754)
+            $_size771 = 0;
+            $_etype774 = 0;
+            $xfer += $input->readListBegin($_etype774, $_size771);
+            for ($_i775 = 0; $_i775 < $_size771; ++$_i775)
             {
-              $elem755 = null;
-              $elem755 = new \metastore\WMPool();
-              $xfer += $elem755->read($input);
-              $this->pools []= $elem755;
+              $elem776 = null;
+              $elem776 = new \metastore\WMPool();
+              $xfer += $elem776->read($input);
+              $this->pools []= $elem776;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25837,15 +26421,15 @@ class WMFullResourcePlan {
         case 3:
           if ($ftype == TType::LST) {
             $this->mappings = array();
-            $_size756 = 0;
-            $_etype759 = 0;
-            $xfer += $input->readListBegin($_etype759, $_size756);
-            for ($_i760 = 0; $_i760 < $_size756; ++$_i760)
+            $_size777 = 0;
+            $_etype780 = 0;
+            $xfer += $input->readListBegin($_etype780, $_size777);
+            for ($_i781 = 0; $_i781 < $_size777; ++$_i781)
             {
-              $elem761 = null;
-              $elem761 = new \metastore\WMMapping();
-              $xfer += $elem761->read($input);
-              $this->mappings []= $elem761;
+              $elem782 = null;
+              $elem782 = new \metastore\WMMapping();
+              $xfer += $elem782->read($input);
+              $this->mappings []= $elem782;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25855,15 +26439,15 @@ class WMFullResourcePlan {
         case 4:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size762 = 0;
-            $_etype765 = 0;
-            $xfer += $input->readListBegin($_etype765, $_size762);
-            for ($_i766 = 0; $_i766 < $_size762; ++$_i766)
+            $_size783 = 0;
+            $_etype786 = 0;
+            $xfer += $input->readListBegin($_etype786, $_size783);
+            for ($_i787 = 0; $_i787 < $_size783; ++$_i787)
             {
-              $elem767 = null;
-              $elem767 = new \metastore\WMTrigger();
-              $xfer += $elem767->read($input);
-              $this->triggers []= $elem767;
+              $elem788 = null;
+              $elem788 = new \metastore\WMTrigger();
+              $xfer += $elem788->read($input);
+              $this->triggers []= $elem788;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25873,15 +26457,15 @@ class WMFullResourcePlan {
         case 5:
           if ($ftype == TType::LST) {
             $this->poolTriggers = array();
-            $_size768 = 0;
-            $_etype771 = 0;
-            $xfer += $input->readListBegin($_etype771, $_size768);
-            for ($_i772 = 0; $_i772 < $_size768; ++$_i772)
+            $_size789 = 0;
+            $_etype792 = 0;
+            $xfer += $input->readListBegin($_etype792, $_size789);
+            for ($_i793 = 0; $_i793 < $_size789; ++$_i793)
             {
-              $elem773 = null;
-              $elem773 = new \metastore\WMPoolTrigger();
-              $xfer += $elem773->read($input);
-              $this->poolTriggers []= $elem773;
+              $elem794 = null;
+              $elem794 = new \metastore\WMPoolTrigger();
+              $xfer += $elem794->read($input);
+              $this->poolTriggers []= $elem794;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -25917,9 +26501,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->pools));
         {
-          foreach ($this->pools as $iter774)
+          foreach ($this->pools as $iter795)
           {
-            $xfer += $iter774->write($output);
+            $xfer += $iter795->write($output);
           }
         }
         $output->writeListEnd();
@@ -25934,9 +26518,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->mappings));
         {
-          foreach ($this->mappings as $iter775)
+          foreach ($this->mappings as $iter796)
           {
-            $xfer += $iter775->write($output);
+            $xfer += $iter796->write($output);
           }
         }
         $output->writeListEnd();
@@ -25951,9 +26535,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter776)
+          foreach ($this->triggers as $iter797)
           {
-            $xfer += $iter776->write($output);
+            $xfer += $iter797->write($output);
           }
         }
         $output->writeListEnd();
@@ -25968,9 +26552,9 @@ class WMFullResourcePlan {
       {
         $output->writeListBegin(TType::STRUCT, count($this->poolTriggers));
         {
-          foreach ($this->poolTriggers as $iter777)
+          foreach ($this->poolTriggers as $iter798)
           {
-            $xfer += $iter777->write($output);
+            $xfer += $iter798->write($output);
           }
         }
         $output->writeListEnd();
@@ -26523,15 +27107,15 @@ class WMGetAllResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->resourcePlans = array();
-            $_size778 = 0;
-            $_etype781 = 0;
-            $xfer += $input->readListBegin($_etype781, $_size778);
-            for ($_i782 = 0; $_i782 < $_size778; ++$_i782)
+            $_size799 = 0;
+            $_etype802 = 0;
+            $xfer += $input->readListBegin($_etype802, $_size799);
+            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
             {
-              $elem783 = null;
-              $elem783 = new \metastore\WMResourcePlan();
-              $xfer += $elem783->read($input);
-              $this->resourcePlans []= $elem783;
+              $elem804 = null;
+              $elem804 = new \metastore\WMResourcePlan();
+              $xfer += $elem804->read($input);
+              $this->resourcePlans []= $elem804;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26559,9 +27143,9 @@ class WMGetAllResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->resourcePlans));
         {
-          foreach ($this->resourcePlans as $iter784)
+          foreach ($this->resourcePlans as $iter805)
           {
-            $xfer += $iter784->write($output);
+            $xfer += $iter805->write($output);
           }
         }
         $output->writeListEnd();
@@ -26967,14 +27551,14 @@ class WMValidateResourcePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->errors = array();
-            $_size785 = 0;
-            $_etype788 = 0;
-            $xfer += $input->readListBegin($_etype788, $_size785);
-            for ($_i789 = 0; $_i789 < $_size785; ++$_i789)
+            $_size806 = 0;
+            $_etype809 = 0;
+            $xfer += $input->readListBegin($_etype809, $_size806);
+            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
             {
-              $elem790 = null;
-              $xfer += $input->readString($elem790);
-              $this->errors []= $elem790;
+              $elem811 = null;
+              $xfer += $input->readString($elem811);
+              $this->errors []= $elem811;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -26984,14 +27568,14 @@ class WMValidateResourcePlanResponse {
         case 2:
           if ($ftype == TType::LST) {
             $this->warnings = array();
-            $_size791 = 0;
-            $_etype794 = 0;
-            $xfer += $input->readListBegin($_etype794, $_size791);
-            for ($_i795 = 0; $_i795 < $_size791; ++$_i795)
+            $_size812 = 0;
+            $_etype815 = 0;
+            $xfer += $input->readListBegin($_etype815, $_size812);
+            for ($_i816 = 0; $_i816 < $_size812; ++$_i816)
             {
-              $elem796 = null;
-              $xfer += $input->readString($elem796);
-              $this->warnings []= $elem796;
+              $elem817 = null;
+              $xfer += $input->readString($elem817);
+              $this->warnings []= $elem817;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27019,9 +27603,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->errors));
         {
-          foreach ($this->errors as $iter797)
+          foreach ($this->errors as $iter818)
           {
-            $xfer += $output->writeString($iter797);
+            $xfer += $output->writeString($iter818);
           }
         }
         $output->writeListEnd();
@@ -27036,9 +27620,9 @@ class WMValidateResourcePlanResponse {
       {
         $output->writeListBegin(TType::STRING, count($this->warnings));
         {
-          foreach ($this->warnings as $iter798)
+          foreach ($this->warnings as $iter819)
           {
-            $xfer += $output->writeString($iter798);
+            $xfer += $output->writeString($iter819);
           }
         }
         $output->writeListEnd();
@@ -27711,15 +28295,15 @@ class WMGetTriggersForResourePlanResponse {
         case 1:
           if ($ftype == TType::LST) {
             $this->triggers = array();
-            $_size799 = 0;
-            $_etype802 = 0;
-            $xfer += $input->readListBegin($_etype802, $_size799);
-            for ($_i803 = 0; $_i803 < $_size799; ++$_i803)
+            $_size820 = 0;
+            $_etype823 = 0;
+            $xfer += $input->readListBegin($_etype823, $_size820);
+            for ($_i824 = 0; $_i824 < $_size820; ++$_i824)
             {
-              $elem804 = null;
-              $elem804 = new \metastore\WMTrigger();
-              $xfer += $elem804->read($input);
-              $this->triggers []= $elem804;
+              $elem825 = null;
+              $elem825 = new \metastore\WMTrigger();
+              $xfer += $elem825->read($input);
+              $this->triggers []= $elem825;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -27747,9 +28331,9 @@ class WMGetTriggersForResourePlanResponse {
       {
         $output->writeListBegin(TType::STRUCT, count($this->triggers));
         {
-          foreach ($this->triggers as $iter805)
+          foreach ($this->triggers as $iter826)
           {
-            $xfer += $iter805->write($output);
+            $xfer += $iter826->write($output);
           }
         }
         $output->writeListEnd();
@@ -29333,15 +29917,15 @@ class SchemaVersion {
         case 4:
           if ($ftype == TType::LST) {
             $this->cols = array();
-            $_size806 = 0;
-            $_etype809 = 0;
-            $xfer += $input->readListBegin($_etype809, $_size806);
-            for ($_i810 = 0; $_i810 < $_size806; ++$_i810)
+            $_size827 = 0;
+            $_etype830 = 0;
+            $xfer += $input->readListBegin($_etype830, $_size827);
+            for ($_i831 = 0; $_i831 < $_size827; ++$_i831)
             {
-              $elem811 = null;
-              $elem811 = new \metastore\FieldSchema();
-              $xfer += $elem811->read($input);
-              $this->cols []= $elem811;
+              $elem832 = null;
+              $elem832 = new \metastore\FieldSchema();
+              $xfer += $elem832->read($input);
+              $this->cols []= $elem832;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29430,9 +30014,9 @@ class SchemaVersion {
       {
         $output->writeListBegin(TType::STRUCT, count($this->cols));
         {
-          foreach ($this->cols as $iter812)
+          foreach ($this->cols as $iter833)
           {
-            $xfer += $iter812->write($output);
+            $xfer += $iter833->write($output);
           }
         }
         $output->writeListEnd();
@@ -29754,15 +30338,15 @@ class FindSchemasByColsResp {
         case 1:
           if ($ftype == TType::LST) {
             $this->schemaVersions = array();
-            $_size813 = 0;
-            $_etype816 = 0;
-            $xfer += $input->readListBegin($_etype816, $_size813);
-            for ($_i817 = 0; $_i817 < $_size813; ++$_i817)
+            $_size834 = 0;
+            $_etype837 = 0;
+            $xfer += $input->readListBegin($_etype837, $_size834);
+            for ($_i838 = 0; $_i838 < $_size834; ++$_i838)
             {
-              $elem818 = null;
-              $elem818 = new \metastore\SchemaVersionDescriptor();
-              $xfer += $elem818->read($input);
-              $this->schemaVersions []= $elem818;
+              $elem839 = null;
+              $elem839 = new \metastore\SchemaVersionDescriptor();
+              $xfer += $elem839->read($input);
+              $this->schemaVersions []= $elem839;
             }
             $xfer += $input->readListEnd();
           } else {
@@ -29790,9 +30374,9 @@ class FindSchemasByColsResp {
       {
         $output->writeListBegin(TType::STRUCT, count($this->schemaVersions));
         {
-          foreach ($this->schemaVersions as $iter819)
+          foreach ($this->schemaVersions as $iter840)
           {
-            $xfer += $iter819->write($output);
+            $xfer += $iter840->write($output);
           }
         }
         $output->writeListEnd();


[08/46] hive git commit: HIVE-18916: SparkClientImpl doesn't error out if spark-submit fails (Sahil Takiar, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-18916: SparkClientImpl doesn't error out if spark-submit fails (Sahil Takiar, reviewed by Aihua Xu)


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

Branch: refs/heads/master-txnstats
Commit: e19b861cfbcb15166f9255f8b375ff5d8056b417
Parents: 88da0e8
Author: Sahil Takiar <ta...@gmail.com>
Authored: Mon Jul 2 11:30:14 2018 -0700
Committer: Sahil Takiar <st...@cloudera.com>
Committed: Mon Jul 2 11:30:14 2018 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |  4 +-
 .../apache/hadoop/hive/ql/QOutProcessor.java    |  2 +
 .../hadoop/hive/ql/exec/spark/SparkTask.java    | 15 ++--
 .../ql/exec/spark/session/SparkSessionImpl.java | 10 ++-
 .../spark/status/RemoteSparkJobMonitor.java     |  1 -
 .../exec/spark/TestSparkInvalidFileFormat.java  | 81 ++++++++++++++++++++
 .../spark_submit_negative_executor_cores.q      |  5 ++
 .../spark_submit_negative_executor_memory.q     |  5 ++
 .../spark_submit_negative_executor_cores.q.out  |  5 ++
 .../spark_submit_negative_executor_memory.q.out |  5 ++
 .../hive/spark/client/AbstractSparkClient.java  | 25 +++---
 .../spark/client/SparkSubmitSparkClient.java    | 22 +++---
 .../apache/hive/spark/client/rpc/RpcServer.java | 21 +++--
 13 files changed, 159 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index d415b7d..385b71e 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1684,7 +1684,9 @@ spark.query.negative.files=groupby2_map_skew_multi_distinct.q,\
 
 spark.only.query.negative.files=spark_job_max_tasks.q,\
   spark_stage_max_tasks.q,\
-  spark_task_failure.q
+  spark_task_failure.q,\
+  spark_submit_negative_executor_cores.q,\
+  spark_submit_negative_executor_memory.q
 
 spark.perf.disabled.query.files=query14.q,\
   query64.q

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java
index 359f027..1e4cddd 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QOutProcessor.java
@@ -273,6 +273,8 @@ public class QOutProcessor {
     ppm.add(new PatternReplacementPair(Pattern.compile("attempt_[0-9_]+"), "attempt_#ID#"));
     ppm.add(new PatternReplacementPair(Pattern.compile("vertex_[0-9_]+"), "vertex_#ID#"));
     ppm.add(new PatternReplacementPair(Pattern.compile("task_[0-9_]+"), "task_#ID#"));
+    ppm.add(new PatternReplacementPair(Pattern.compile("for Spark session.*?:"),
+            "#SPARK_SESSION_ID#:"));
     partialPlanMask = ppm.toArray(new PatternReplacementPair[ppm.size()]);
   }
   /* This list may be modified by specific cli drivers to mask strings that change on every test */

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
index 02613f2..ad5049a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
@@ -180,7 +180,7 @@ public class SparkTask extends Task<SparkWork> {
         killJob();
       } else if (rc == 4) {
         LOG.info("The Spark job or one stage of it has too many tasks" +
-            ". Cancelling Spark job " + sparkJobID + " with application ID " + jobID );
+            ". Cancelling Spark job " + sparkJobID + " with application ID " + jobID);
         killJob();
       }
 
@@ -189,12 +189,7 @@ public class SparkTask extends Task<SparkWork> {
       }
       sparkJobStatus.cleanup();
     } catch (Exception e) {
-      String msg = "Failed to execute Spark task " + getId() + ", with exception '" + Utilities.getNameMessage(e) + "'";
-
-      // Has to use full name to make sure it does not conflict with
-      // org.apache.commons.lang.StringUtils
-      console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e));
-      LOG.error(msg, e);
+      LOG.error("Failed to execute Spark task \"" + getId() + "\"", e);
       setException(e);
       if (e instanceof HiveException) {
         HiveException he = (HiveException) e;
@@ -609,7 +604,7 @@ public class SparkTask extends Task<SparkWork> {
   private boolean isTaskFailure(Throwable error) {
     Pattern taskFailedPattern = Pattern.compile("Task.*in stage.*failed.*times");
     while (error != null) {
-      if (taskFailedPattern.matcher(error.getMessage()).find()) {
+      if (error.getMessage() != null && taskFailedPattern.matcher(error.getMessage()).find()) {
         return true;
       }
       error = error.getCause();
@@ -621,8 +616,8 @@ public class SparkTask extends Task<SparkWork> {
     while (error != null) {
       if (error instanceof OutOfMemoryError) {
         return true;
-      } else if (error.getMessage().contains("Container killed by YARN for exceeding memory " +
-              "limits")) {
+      } else if (error.getMessage() != null && error.getMessage().contains("Container killed by " +
+              "YARN for exceeding memory limits")) {
         return true;
       }
       error = error.getCause();

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
index c8cb1ac..6e37d93 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java
@@ -214,13 +214,19 @@ public class SparkSessionImpl implements SparkSession {
           return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_RESOURCE_REQUEST,
               sessionId, matchedString.toString());
         } else {
-          return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_ERROR, sessionId, Throwables.getRootCause(e).getMessage());
+          return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_ERROR, sessionId,
+                  getRootCause(oe));
         }
       }
       e = e.getCause();
     }
 
-    return new HiveException(oe, ErrorMsg.SPARK_CREATE_CLIENT_ERROR, sessionId, Throwables.getRootCause(oe).getMessage());
+    return new HiveException(oe, ErrorMsg.SPARK_CREATE_CLIENT_ERROR, sessionId, getRootCause(oe));
+  }
+
+  private String getRootCause(Throwable e) {
+    Throwable rootCause = Throwables.getRootCause(e);
+    return rootCause.getClass().getName() + ": " + rootCause.getMessage();
   }
 
   private boolean matches(String input, String regex, StringBuilder matchedString) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
index 004b50b..560fb58 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java
@@ -76,7 +76,6 @@ public class RemoteSparkJobMonitor extends SparkJobMonitor {
           if ((timeCount > monitorTimeoutInterval)) {
             HiveException he = new HiveException(ErrorMsg.SPARK_JOB_MONITOR_TIMEOUT,
                 Long.toString(timeCount));
-            console.printError(he.getMessage());
             sparkJobStatus.setMonitorError(he);
             running = false;
             done = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkInvalidFileFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkInvalidFileFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkInvalidFileFormat.java
new file mode 100644
index 0000000..bcc0924
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkInvalidFileFormat.java
@@ -0,0 +1,81 @@
+/*
+ * 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.spark;
+
+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.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+public class TestSparkInvalidFileFormat {
+
+  @Test
+  public void readTextFileAsParquet() throws IOException {
+    HiveConf conf = new HiveConf();
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+            SQLStdHiveAuthorizerFactory.class.getName());
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "spark");
+    conf.set("spark.master", "local");
+
+    FileSystem fs = FileSystem.getLocal(conf);
+    Path tmpDir = new Path("TestSparkInvalidFileFormat-tmp");
+
+    File testFile = new File(conf.get("test.data.files"), "kv1.txt");
+
+    SessionState.start(conf);
+
+    IDriver driver = null;
+
+    try {
+      driver = DriverFactory.newDriver(conf);
+      Assert.assertEquals(0,
+              driver.run("CREATE TABLE test_table (key STRING, value STRING)").getResponseCode());
+      Assert.assertEquals(0, driver.run(
+              "LOAD DATA LOCAL INPATH '" + testFile + "' INTO TABLE test_table").getResponseCode());
+      Assert.assertEquals(0,
+              driver.run("ALTER TABLE test_table SET FILEFORMAT parquet").getResponseCode());
+      Throwable exception = driver.run(
+              "SELECT * FROM test_table ORDER BY key LIMIT 10").getException();
+      Assert.assertTrue(exception instanceof HiveException);
+      Assert.assertTrue(exception.getMessage().contains("Spark job failed due to task failures"));
+      Assert.assertTrue(exception.getMessage().contains("kv1.txt is not a Parquet file. expected " +
+              "magic number at tail [80, 65, 82, 49] but found [95, 57, 55, 10]"));
+    } finally {
+      if (driver != null) {
+        Assert.assertEquals(0, driver.run("DROP TABLE IF EXISTS test_table").getResponseCode());
+        driver.destroy();
+      }
+      if (fs.exists(tmpDir)) {
+        fs.delete(tmpDir, true);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/test/queries/clientnegative/spark_submit_negative_executor_cores.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/spark_submit_negative_executor_cores.q b/ql/src/test/queries/clientnegative/spark_submit_negative_executor_cores.q
new file mode 100644
index 0000000..5a92390
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/spark_submit_negative_executor_cores.q
@@ -0,0 +1,5 @@
+--! qt:dataset:src
+
+set spark.executor.cores=-1;
+
+select * from src order by key limit 10;

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/test/queries/clientnegative/spark_submit_negative_executor_memory.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/spark_submit_negative_executor_memory.q b/ql/src/test/queries/clientnegative/spark_submit_negative_executor_memory.q
new file mode 100644
index 0000000..55bc4b8
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/spark_submit_negative_executor_memory.q
@@ -0,0 +1,5 @@
+--! qt:dataset:src
+
+set spark.executor.memory=-1;
+
+select * from src order by key limit 10;

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_cores.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_cores.q.out b/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_cores.q.out
new file mode 100644
index 0000000..47ac8b2
--- /dev/null
+++ b/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_cores.q.out
@@ -0,0 +1,5 @@
+PREHOOK: query: select * from src order by key limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 30041 from org.apache.hadoop.hive.ql.exec.spark.SparkTask. Failed to create Spark client #SPARK_SESSION_ID#: java.lang.RuntimeException: spark-submit process failed with exit code 1 and error "Error: Executor cores must be a positive number"

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_memory.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_memory.q.out b/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_memory.q.out
new file mode 100644
index 0000000..4aa1337
--- /dev/null
+++ b/ql/src/test/results/clientnegative/spark/spark_submit_negative_executor_memory.q.out
@@ -0,0 +1,5 @@
+PREHOOK: query: select * from src order by key limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 30041 from org.apache.hadoop.hive.ql.exec.spark.SparkTask. Failed to create Spark client #SPARK_SESSION_ID#: java.lang.RuntimeException: spark-submit process failed with exit code 1 and error "Error: Executor Memory cores must be a positive number"

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/spark-client/src/main/java/org/apache/hive/spark/client/AbstractSparkClient.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/AbstractSparkClient.java b/spark-client/src/main/java/org/apache/hive/spark/client/AbstractSparkClient.java
index ed9222c..b2b5201 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/AbstractSparkClient.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/AbstractSparkClient.java
@@ -24,7 +24,6 @@ import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
 import com.google.common.collect.Maps;
 import com.google.common.io.Resources;
 
@@ -41,8 +40,6 @@ import java.io.Serializable;
 import java.io.Writer;
 import java.net.URI;
 import java.net.URL;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -125,17 +122,19 @@ abstract class AbstractSparkClient implements SparkClient {
       } else {
         errorMsg = "Error while waiting for Remote Spark Driver to connect back to HiveServer2.";
       }
-      LOG.error(errorMsg, e);
-      driverFuture.cancel(true);
-      try {
-        driverFuture.get();
-      } catch (InterruptedException ie) {
-        // Give up.
-        LOG.warn("Interrupted before driver thread was finished.", ie);
-      } catch (ExecutionException ee) {
-        LOG.error("Driver thread failed", ee);
+      if (driverFuture.isDone()) {
+        try {
+          driverFuture.get();
+        } catch (InterruptedException ie) {
+          // Give up.
+          LOG.warn("Interrupted before driver thread was finished.", ie);
+        } catch (ExecutionException ee) {
+          LOG.error("Driver thread failed", ee);
+        }
+      } else {
+        driverFuture.cancel(true);
       }
-      throw Throwables.propagate(e);
+      throw new RuntimeException(errorMsg, e);
     }
 
     LOG.info("Successfully connected to Remote Spark Driver at: " + this.driverRpc.getRemoteAddress());

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java
index 31e89b8..7a6e77b 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkSubmitSparkClient.java
@@ -32,6 +32,8 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 import java.util.concurrent.FutureTask;
 
+import org.apache.commons.lang3.StringUtils;
+
 import org.apache.hadoop.hive.common.log.LogRedirector;
 import org.apache.hadoop.hive.conf.Constants;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -200,19 +202,19 @@ class SparkSubmitSparkClient extends AbstractSparkClient {
       try {
         int exitCode = child.waitFor();
         if (exitCode != 0) {
-          StringBuilder errStr = new StringBuilder();
-          synchronized(childErrorLog) {
-            for (Object aChildErrorLog : childErrorLog) {
-              errStr.append(aChildErrorLog);
-              errStr.append('\n');
+          List<String> errorMessages = new ArrayList<>();
+          synchronized (childErrorLog) {
+            for (String line : childErrorLog) {
+              if (StringUtils.containsIgnoreCase(line, "Error")) {
+                errorMessages.add("\"" + line + "\"");
+              }
             }
           }
 
-          LOG.warn("Child process exited with code {}", exitCode);
-          rpcServer.cancelClient(clientId,
-              "Child process (spark-submit) exited before connecting back with error log " + errStr.toString());
-        } else {
-          LOG.info("Child process (spark-submit) exited successfully.");
+          String errStr = errorMessages.isEmpty() ? "?" : Joiner.on(',').join(errorMessages);
+
+          rpcServer.cancelClient(clientId, new RuntimeException("spark-submit process failed " +
+                  "with exit code " + exitCode + " and error " + errStr));
         }
       } catch (InterruptedException ie) {
         LOG.warn("Thread waiting on the child process (spark-submit) is interrupted, killing the child process.");

http://git-wip-us.apache.org/repos/asf/hive/blob/e19b861c/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
----------------------------------------------------------------------
diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
index babcb54..0c67ffd 100644
--- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
+++ b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java
@@ -202,11 +202,12 @@ public class RpcServer implements Closeable {
   }
 
   /**
-   * Tells the RPC server to cancel the connection from an existing pending client
+   * Tells the RPC server to cancel the connection from an existing pending client.
+   *
    * @param clientId The identifier for the client
-   * @param msg The error message about why the connection should be canceled
+   * @param failure The error about why the connection should be canceled
    */
-  public void cancelClient(final String clientId, final String msg) {
+  public void cancelClient(final String clientId, final Throwable failure) {
     final ClientInfo cinfo = pendingClients.remove(clientId);
     if (cinfo == null) {
       // Nothing to be done here.
@@ -214,12 +215,22 @@ public class RpcServer implements Closeable {
     }
     cinfo.timeoutFuture.cancel(true);
     if (!cinfo.promise.isDone()) {
-      cinfo.promise.setFailure(new RuntimeException(
-          String.format("Cancelling Remote Spark Driver client connection '%s' with error: " + msg, clientId)));
+      cinfo.promise.setFailure(failure);
     }
   }
 
   /**
+   * Tells the RPC server to cancel the connection from an existing pending client.
+   *
+   * @param clientId The identifier for the client
+   * @param msg The error message about why the connection should be canceled
+   */
+  public void cancelClient(final String clientId, final String msg) {
+    cancelClient(clientId, new RuntimeException(String.format(
+            "Cancelling Remote Spark Driver client connection '%s' with error: " + msg, clientId)));
+  }
+
+  /**
    * Creates a secret for identifying a client connection.
    */
   public String createSecret() {


[37/46] hive git commit: HIVE-19326 : stats auto gather: incorrect aggregation during UNION queries (may lead to incorrect results) (Zoltan Haindrich via Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/ql/src/test/results/clientpositive/union_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_stats.q.out b/ql/src/test/results/clientpositive/union_stats.q.out
index 73e880b..1acf5c8 100644
--- a/ql/src/test/results/clientpositive/union_stats.q.out
+++ b/ql/src/test/results/clientpositive/union_stats.q.out
@@ -454,6 +454,249 @@ POSTHOOK: query: create table t2 like src
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@t2
+PREHOOK: query: explain from (select * from src union all select * from src)s
+insert overwrite table t1 select *
+insert overwrite table t2 select *
+PREHOOK: type: QUERY
+POSTHOOK: query: explain from (select * from src union all select * from src)s
+insert overwrite table t1 select *
+insert overwrite table t2 select *
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-2 is a root stage
+  Stage-8 depends on stages: Stage-2 , consists of Stage-5, Stage-4, Stage-6
+  Stage-5
+  Stage-0 depends on stages: Stage-5, Stage-4, Stage-7
+  Stage-3 depends on stages: Stage-0, Stage-10
+  Stage-4
+  Stage-6
+  Stage-7 depends on stages: Stage-6
+  Stage-1 depends on stages: Stage-2
+  Stage-9 depends on stages: Stage-1, Stage-10
+  Stage-10 depends on stages: Stage-2
+
+STAGE PLANS:
+  Stage: Stage-2
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Union
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1000 Data size: 10624 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
+                      name: default.t1
+                Select Operator
+                  expressions: _col0 (type: string), _col1 (type: string)
+                  outputColumnNames: key, value
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1000 Data size: 10624 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
+                      name: default.t2
+                Select Operator
+                  expressions: _col0 (type: string), _col1 (type: string)
+                  outputColumnNames: key, value
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+                    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
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Union
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1000 Data size: 10624 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
+                      name: default.t1
+                Select Operator
+                  expressions: _col0 (type: string), _col1 (type: string)
+                  outputColumnNames: key, value
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+                    Reduce Output Operator
+                      sort order: 
+                      Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+                      value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1000 Data size: 10624 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
+                      name: default.t2
+                Select Operator
+                  expressions: _col0 (type: string), _col1 (type: string)
+                  outputColumnNames: key, value
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  Group By Operator
+                    aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
+                    mode: hash
+                    outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+                    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
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-8
+    Conditional Operator
+
+  Stage: Stage-5
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: true
+          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
+              name: default.t1
+
+  Stage: Stage-3
+    Stats Work
+      Basic Stats Work:
+      Column Stats Desc:
+          Columns: key, value
+          Column Types: string, string
+          Table: default.t1
+
+  Stage: Stage-4
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            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
+                  name: default.t1
+
+  Stage: Stage-6
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            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
+                  name: default.t1
+
+  Stage: Stage-7
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-1
+    Move Operator
+      tables:
+          replace: true
+          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
+              name: default.t2
+
+  Stage: Stage-9
+    Stats Work
+      Basic Stats Work:
+      Column Stats Desc:
+          Columns: key, value
+          Column Types: string, string
+          Table: default.t2
+
+  Stage: Stage-10
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            Reduce Output Operator
+              sort order: 
+              Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+              value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
+      Execution mode: vectorized
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
 PREHOOK: query: from (select * from src union all select * from src)s
 insert overwrite table t1 select *
 insert overwrite table t2 select *

http://git-wip-us.apache.org/repos/asf/hive/blob/2ca70b91/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java b/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
index 3b10987..09dddac 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/Deserializer.java
@@ -68,6 +68,8 @@ public interface Deserializer {
 
   /**
    * Returns statistics collected when serializing
+   *
+   * @return {@link SerDeStats} object; or in case not supported: null
    */
   SerDeStats getSerDeStats();
 }


[30/46] hive git commit: HIVE-19267: Replicate ACID/MM tables write operations (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
index 1d57aee..352f5c7 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.h
@@ -187,6 +187,7 @@ class ThriftHiveMetastoreIf : virtual public  ::facebook::fb303::FacebookService
   virtual void get_notification_events_count(NotificationEventsCountResponse& _return, const NotificationEventsCountRequest& rqst) = 0;
   virtual void fire_listener_event(FireEventResponse& _return, const FireEventRequest& rqst) = 0;
   virtual void flushCache() = 0;
+  virtual void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst) = 0;
   virtual void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) = 0;
   virtual void get_file_metadata_by_expr(GetFileMetadataByExprResult& _return, const GetFileMetadataByExprRequest& req) = 0;
   virtual void get_file_metadata(GetFileMetadataResult& _return, const GetFileMetadataRequest& req) = 0;
@@ -780,6 +781,9 @@ class ThriftHiveMetastoreNull : virtual public ThriftHiveMetastoreIf , virtual p
   void flushCache() {
     return;
   }
+  void add_write_notification_log(WriteNotificationLogResponse& /* _return */, const WriteNotificationLogRequest& /* rqst */) {
+    return;
+  }
   void cm_recycle(CmRecycleResponse& /* _return */, const CmRecycleRequest& /* request */) {
     return;
   }
@@ -21228,6 +21232,110 @@ class ThriftHiveMetastore_flushCache_presult {
 
 };
 
+typedef struct _ThriftHiveMetastore_add_write_notification_log_args__isset {
+  _ThriftHiveMetastore_add_write_notification_log_args__isset() : rqst(false) {}
+  bool rqst :1;
+} _ThriftHiveMetastore_add_write_notification_log_args__isset;
+
+class ThriftHiveMetastore_add_write_notification_log_args {
+ public:
+
+  ThriftHiveMetastore_add_write_notification_log_args(const ThriftHiveMetastore_add_write_notification_log_args&);
+  ThriftHiveMetastore_add_write_notification_log_args& operator=(const ThriftHiveMetastore_add_write_notification_log_args&);
+  ThriftHiveMetastore_add_write_notification_log_args() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_args() throw();
+  WriteNotificationLogRequest rqst;
+
+  _ThriftHiveMetastore_add_write_notification_log_args__isset __isset;
+
+  void __set_rqst(const WriteNotificationLogRequest& val);
+
+  bool operator == (const ThriftHiveMetastore_add_write_notification_log_args & rhs) const
+  {
+    if (!(rqst == rhs.rqst))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_write_notification_log_args &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_write_notification_log_args & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+
+class ThriftHiveMetastore_add_write_notification_log_pargs {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_pargs() throw();
+  const WriteNotificationLogRequest* rqst;
+
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_write_notification_log_result__isset {
+  _ThriftHiveMetastore_add_write_notification_log_result__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_add_write_notification_log_result__isset;
+
+class ThriftHiveMetastore_add_write_notification_log_result {
+ public:
+
+  ThriftHiveMetastore_add_write_notification_log_result(const ThriftHiveMetastore_add_write_notification_log_result&);
+  ThriftHiveMetastore_add_write_notification_log_result& operator=(const ThriftHiveMetastore_add_write_notification_log_result&);
+  ThriftHiveMetastore_add_write_notification_log_result() {
+  }
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_result() throw();
+  WriteNotificationLogResponse success;
+
+  _ThriftHiveMetastore_add_write_notification_log_result__isset __isset;
+
+  void __set_success(const WriteNotificationLogResponse& val);
+
+  bool operator == (const ThriftHiveMetastore_add_write_notification_log_result & rhs) const
+  {
+    if (!(success == rhs.success))
+      return false;
+    return true;
+  }
+  bool operator != (const ThriftHiveMetastore_add_write_notification_log_result &rhs) const {
+    return !(*this == rhs);
+  }
+
+  bool operator < (const ThriftHiveMetastore_add_write_notification_log_result & ) const;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+  uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const;
+
+};
+
+typedef struct _ThriftHiveMetastore_add_write_notification_log_presult__isset {
+  _ThriftHiveMetastore_add_write_notification_log_presult__isset() : success(false) {}
+  bool success :1;
+} _ThriftHiveMetastore_add_write_notification_log_presult__isset;
+
+class ThriftHiveMetastore_add_write_notification_log_presult {
+ public:
+
+
+  virtual ~ThriftHiveMetastore_add_write_notification_log_presult() throw();
+  WriteNotificationLogResponse* success;
+
+  _ThriftHiveMetastore_add_write_notification_log_presult__isset __isset;
+
+  uint32_t read(::apache::thrift::protocol::TProtocol* iprot);
+
+};
+
 typedef struct _ThriftHiveMetastore_cm_recycle_args__isset {
   _ThriftHiveMetastore_cm_recycle_args__isset() : request(false) {}
   bool request :1;
@@ -26718,6 +26826,9 @@ class ThriftHiveMetastoreClient : virtual public ThriftHiveMetastoreIf, public
   void flushCache();
   void send_flushCache();
   void recv_flushCache();
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst);
+  void send_add_write_notification_log(const WriteNotificationLogRequest& rqst);
+  void recv_add_write_notification_log(WriteNotificationLogResponse& _return);
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request);
   void send_cm_recycle(const CmRecycleRequest& request);
   void recv_cm_recycle(CmRecycleResponse& _return);
@@ -27019,6 +27130,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
   void process_get_notification_events_count(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_fire_listener_event(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_flushCache(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
+  void process_add_write_notification_log(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_cm_recycle(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_file_metadata_by_expr(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
   void process_get_file_metadata(int32_t seqid, ::apache::thrift::protocol::TProtocol* iprot, ::apache::thrift::protocol::TProtocol* oprot, void* callContext);
@@ -27230,6 +27342,7 @@ class ThriftHiveMetastoreProcessor : public  ::facebook::fb303::FacebookServiceP
     processMap_["get_notification_events_count"] = &ThriftHiveMetastoreProcessor::process_get_notification_events_count;
     processMap_["fire_listener_event"] = &ThriftHiveMetastoreProcessor::process_fire_listener_event;
     processMap_["flushCache"] = &ThriftHiveMetastoreProcessor::process_flushCache;
+    processMap_["add_write_notification_log"] = &ThriftHiveMetastoreProcessor::process_add_write_notification_log;
     processMap_["cm_recycle"] = &ThriftHiveMetastoreProcessor::process_cm_recycle;
     processMap_["get_file_metadata_by_expr"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata_by_expr;
     processMap_["get_file_metadata"] = &ThriftHiveMetastoreProcessor::process_get_file_metadata;
@@ -28884,6 +28997,16 @@ class ThriftHiveMetastoreMultiface : virtual public ThriftHiveMetastoreIf, publi
     ifaces_[i]->flushCache();
   }
 
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst) {
+    size_t sz = ifaces_.size();
+    size_t i = 0;
+    for (; i < (sz - 1); ++i) {
+      ifaces_[i]->add_write_notification_log(_return, rqst);
+    }
+    ifaces_[i]->add_write_notification_log(_return, rqst);
+    return;
+  }
+
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) {
     size_t sz = ifaces_.size();
     size_t i = 0;
@@ -29805,6 +29928,9 @@ class ThriftHiveMetastoreConcurrentClient : virtual public ThriftHiveMetastoreIf
   void flushCache();
   int32_t send_flushCache();
   void recv_flushCache(const int32_t seqid);
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst);
+  int32_t send_add_write_notification_log(const WriteNotificationLogRequest& rqst);
+  void recv_add_write_notification_log(WriteNotificationLogResponse& _return, const int32_t seqid);
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request);
   int32_t send_cm_recycle(const CmRecycleRequest& request);
   void recv_cm_recycle(CmRecycleResponse& _return, const int32_t seqid);

http://git-wip-us.apache.org/repos/asf/hive/blob/f519db7e/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
index d45ec81..789c150 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore_server.skeleton.cpp
@@ -847,6 +847,11 @@ class ThriftHiveMetastoreHandler : virtual public ThriftHiveMetastoreIf {
     printf("flushCache\n");
   }
 
+  void add_write_notification_log(WriteNotificationLogResponse& _return, const WriteNotificationLogRequest& rqst) {
+    // Your implementation goes here
+    printf("add_write_notification_log\n");
+  }
+
   void cm_recycle(CmRecycleResponse& _return, const CmRecycleRequest& request) {
     // Your implementation goes here
     printf("cm_recycle\n");