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

[hive] branch master updated: HIVE-23671: MSCK repair should handle transactional tables (Peter Varga, reviewed by Denys Kuzmenko)

This is an automated email from the ASF dual-hosted git repository.

dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 005d99c  HIVE-23671: MSCK repair should handle transactional tables (Peter Varga, reviewed by Denys Kuzmenko)
005d99c is described below

commit 005d99c03be505035d25d2f415f0420ba703355b
Author: Peter Varga <pv...@cloudera.com>
AuthorDate: Tue Jul 14 16:51:31 2020 +0200

    HIVE-23671: MSCK repair should handle transactional tables (Peter Varga, reviewed by Denys Kuzmenko)
    
    Closes (#1087)
---
 .../hive/ql/ddl/misc/msck/MsckOperation.java       |   11 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java    |   48 +-
 .../org/apache/hadoop/hive/ql/TestTxnCommands.java |    4 +-
 .../apache/hadoop/hive/ql/TestTxnCommands2.java    |   29 +-
 .../hadoop/hive/ql/TxnCommandsBaseForTests.java    |   42 +-
 .../hive/ql/metadata/TestHiveMetaStoreChecker.java |  390 +-
 .../hive/ql/metadata/TestMSCKRepairOnAcid.java     |  509 ++
 .../api/MaxAllocatedTableWriteIdRequest.java       |  490 ++
 .../api/MaxAllocatedTableWriteIdResponse.java      |  387 ++
 .../metastore/api/SeedTableWriteIdsRequest.java    |  589 ++
 .../hive/metastore/api/SeedTxnIdRequest.java       |  387 ++
 .../hive/metastore/api/ThriftHiveMetastore.java    | 5899 ++++++++++++++------
 .../gen-php/metastore/ThriftHiveMetastore.php      |  671 +++
 .../src/gen/thrift/gen-php/metastore/Types.php     |  369 ++
 .../hive_metastore/ThriftHiveMetastore-remote      |   21 +
 .../gen-py/hive_metastore/ThriftHiveMetastore.py   |  594 ++
 .../src/gen/thrift/gen-py/hive_metastore/ttypes.py |  313 ++
 .../src/gen/thrift/gen-rb/hive_metastore_types.rb  |   77 +
 .../src/gen/thrift/gen-rb/thrift_hive_metastore.rb |  177 +
 .../apache/hadoop/hive/metastore/CheckResult.java  |   47 +-
 .../hadoop/hive/metastore/HiveMetaStoreClient.java |   15 +
 .../hadoop/hive/metastore/IMetaStoreClient.java    |   27 +
 .../api/InitializeTableWriteIdsRequest.java        |   42 -
 .../src/main/thrift/hive_metastore.thrift          |   21 +
 .../hadoop/hive/metastore/HiveMetaStore.java       |   16 +
 .../hive/metastore/HiveMetaStoreChecker.java       |  102 +-
 .../org/apache/hadoop/hive/metastore/Msck.java     |  248 +-
 .../metastore/TransactionalValidationListener.java |    4 +-
 .../hadoop/hive/metastore/txn/TxnDbUtil.java       |   23 +
 .../hadoop/hive/metastore/txn/TxnHandler.java      |   99 +-
 .../apache/hadoop/hive/metastore/txn/TxnStore.java |   18 +-
 .../metastore/HiveMetaStoreClientPreCatalog.java   |   16 +
 32 files changed, 9723 insertions(+), 1962 deletions(-)

diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/msck/MsckOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/msck/MsckOperation.java
index f121fe8..b04e1ba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/msck/MsckOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/msck/MsckOperation.java
@@ -18,8 +18,8 @@
 
 package org.apache.hadoop.hive.ql.ddl.misc.msck;
 
+import org.apache.hadoop.hive.common.TableName;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
-import org.apache.hadoop.hive.ql.exec.Utilities;
 
 import java.io.IOException;
 
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.HiveTableName;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.thrift.TException;
@@ -54,22 +55,22 @@ public class MsckOperation extends DDLOperation<MsckDesc> {
       Msck msck = new Msck(false, false);
       msck.init(Msck.getMsckConf(context.getDb().getConf()));
 
-      String[] names = Utilities.getDbTableName(desc.getTableName());
+      TableName tableName = HiveTableName.of(desc.getTableName());
 
       long partitionExpirySeconds = -1L;
       try (HiveMetaStoreClient msc = new HiveMetaStoreClient(context.getConf())) {
-        Table table = msc.getTable(SessionState.get().getCurrentCatalog(), names[0], names[1]);
-        String qualifiedTableName = Warehouse.getCatalogQualifiedTableName(table);
         boolean msckEnablePartitionRetention = MetastoreConf.getBoolVar(context.getConf(),
             MetastoreConf.ConfVars.MSCK_REPAIR_ENABLE_PARTITION_RETENTION);
         if (msckEnablePartitionRetention) {
+          Table table = msc.getTable(SessionState.get().getCurrentCatalog(), tableName.getDb(), tableName.getTable());
+          String qualifiedTableName = Warehouse.getCatalogQualifiedTableName(table);
           partitionExpirySeconds = PartitionManagementTask.getRetentionPeriodInSeconds(table);
           LOG.info("{} - Retention period ({}s) for partition is enabled for MSCK REPAIR..", qualifiedTableName,
               partitionExpirySeconds);
         }
       }
 
-      MsckInfo msckInfo = new MsckInfo(SessionState.get().getCurrentCatalog(), names[0], names[1],
+      MsckInfo msckInfo = new MsckInfo(SessionState.get().getCurrentCatalog(), tableName.getDb(), tableName.getTable(),
           desc.getFilterExp(), desc.getResFile(), desc.isRepairPartitions(),
           desc.isAddPartitions(), desc.isDropPartitions(), partitionExpirySeconds);
       return msck.repair(msckInfo);
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 4fe1664..36d6c97 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
@@ -378,7 +378,7 @@ public class AcidUtils {
    * Since Hive 4.0, compactor produces directories with {@link #VISIBILITY_PATTERN} suffix.
    * _v0 is equivalent to no suffix, for backwards compatibility.
    */
-  static String addVisibilitySuffix(String baseOrDeltaDir, long visibilityTxnId) {
+  public static String addVisibilitySuffix(String baseOrDeltaDir, long visibilityTxnId) {
     if(visibilityTxnId == 0) {
       return baseOrDeltaDir;
     }
@@ -2358,33 +2358,29 @@ public class AcidUtils {
     long writeId = -1;
     ValidWriteIdList validWriteIdList = null;
 
-    HiveTxnManager sessionTxnMgr = SessionState.get().getTxnMgr();
-    String fullTableName = getFullTableName(dbName, tblName);
-    if (sessionTxnMgr != null && sessionTxnMgr.getCurrentTxnId() > 0) {
-      validWriteIdList = getTableValidWriteIdList(conf, fullTableName);
-      if (isStatsUpdater) {
-        writeId = SessionState.get().getTxnMgr() != null ?
-                SessionState.get().getTxnMgr().getAllocatedTableWriteId(
-                  dbName, tblName) : -1;
-        if (writeId < 1) {
-          // TODO: this is not ideal... stats updater that doesn't have write ID is currently
-          //       "create table"; writeId would be 0/-1 here. No need to call this w/true.
-          LOG.debug("Stats updater for {}.{} doesn't have a write ID ({})",
-              dbName, tblName, writeId);
+    if (SessionState.get() != null) {
+      HiveTxnManager sessionTxnMgr = SessionState.get().getTxnMgr();
+      String fullTableName = getFullTableName(dbName, tblName);
+      if (sessionTxnMgr != null && sessionTxnMgr.getCurrentTxnId() > 0) {
+        validWriteIdList = getTableValidWriteIdList(conf, fullTableName);
+        if (isStatsUpdater) {
+          writeId = sessionTxnMgr.getAllocatedTableWriteId(dbName, tblName);
+          if (writeId < 1) {
+            // TODO: this is not ideal... stats updater that doesn't have write ID is currently
+            //       "create table"; writeId would be 0/-1 here. No need to call this w/true.
+            LOG.debug("Stats updater for {}.{} doesn't have a write ID ({})", dbName, tblName, writeId);
+          }
         }
-      }
 
-
-      if (HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST)
-          && conf.get(ValidTxnList.VALID_TXNS_KEY) == null) {
-        return null;
-      }
-      if (validWriteIdList == null) {
-        validWriteIdList = getTableValidWriteIdListWithTxnList(
-            conf, dbName, tblName);
-      }
-      if (validWriteIdList == null) {
-        throw new AssertionError("Cannot find valid write ID list for " + tblName);
+        if (HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) && conf.get(ValidTxnList.VALID_TXNS_KEY) == null) {
+          return null;
+        }
+        if (validWriteIdList == null) {
+          validWriteIdList = getTableValidWriteIdListWithTxnList(conf, dbName, tblName);
+        }
+        if (validWriteIdList == null) {
+          throw new AssertionError("Cannot find valid write ID list for " + tblName);
+        }
       }
     }
     return new TableSnapshot(writeId,
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 98805df..4043212 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -1188,7 +1188,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
 
     //run Compaction
     runStatementOnDriver("alter table "+ TestTxnCommands2.Table.NONACIDORCTBL +" compact 'major'");
-    TestTxnCommands2.runWorker(hiveConf);
+    runWorker(hiveConf);
 
     query = "select ROW__ID, a, b" + (isVectorized ? "" : ", INPUT__FILE__NAME") + " from "
         + Table.NONACIDORCTBL + " order by ROW__ID";
@@ -1277,7 +1277,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
 
     runStatementOnDriver("insert into T" + makeValuesClause(data));
     runStatementOnDriver("alter table T compact 'major'");
-    TestTxnCommands2.runWorker(hiveConf);
+    runWorker(hiveConf);
 
     //check status of compaction job
     TxnStore txnHandler = TxnUtils.getTxnStore(hiveConf);
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
index 6cc8489..fcaea79 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
@@ -22,7 +22,6 @@ import java.io.File;
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -2211,20 +2210,7 @@ public class TestTxnCommands2 {
    * sorts rows in dictionary order
    */
   static List<String> stringifyValues(int[][] rowsIn) {
-    assert rowsIn.length > 0;
-    int[][] rows = rowsIn.clone();
-    Arrays.sort(rows, new RowComp());
-    List<String> rs = new ArrayList<String>();
-    for(int[] row : rows) {
-      assert row.length > 0;
-      StringBuilder sb = new StringBuilder();
-      for(int value : row) {
-        sb.append(value).append("\t");
-      }
-      sb.setLength(sb.length() - 1);
-      rs.add(sb.toString());
-    }
-    return rs;
+    return TxnCommandsBaseForTests.stringifyValues(rowsIn);
   }
 
   /**
@@ -2272,19 +2258,6 @@ public class TestTxnCommands2 {
     return rs;
   }
 
-  static class RowComp implements Comparator<int[]> {
-    @Override
-    public int compare(int[] row1, int[] row2) {
-      assert row1 != null && row2 != null && row1.length == row2.length;
-      for(int i = 0; i < row1.length; i++) {
-        int comp = Integer.compare(row1[i], row2[i]);
-        if(comp != 0) {
-          return comp;
-        }
-      }
-      return 0;
-    }
-  }
   static String makeValuesClause(int[][] rows) {
     assert rows.length > 0;
     StringBuilder sb = new StringBuilder(" values");
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
index 91752e8..a59b364 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hive.ql;
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -58,8 +60,8 @@ public abstract class TxnCommandsBaseForTests {
   @Rule
   public TestName testName = new TestName();
   protected HiveConf hiveConf;
-  Driver d;
-  private TxnStore txnHandler;
+  protected Driver d;
+  protected TxnStore txnHandler;
 
   public enum Table {
     ACIDTBL("acidTbl"),
@@ -162,8 +164,34 @@ public abstract class TxnCommandsBaseForTests {
    * takes raw data and turns it into a string as if from Driver.getResults()
    * sorts rows in dictionary order
    */
-  List<String> stringifyValues(int[][] rowsIn) {
-    return TestTxnCommands2.stringifyValues(rowsIn);
+  public static List<String> stringifyValues(int[][] rowsIn) {
+    assert rowsIn.length > 0;
+    int[][] rows = rowsIn.clone();
+    Arrays.sort(rows, new RowComp());
+    List<String> rs = new ArrayList<>();
+    for(int[] row : rows) {
+      assert row.length > 0;
+      StringBuilder sb = new StringBuilder();
+      for(int value : row) {
+        sb.append(value).append("\t");
+      }
+      sb.setLength(sb.length() - 1);
+      rs.add(sb.toString());
+    }
+    return rs;
+  }
+  static class RowComp implements Comparator<int[]> {
+    @Override
+    public int compare(int[] row1, int[] row2) {
+      assert row1 != null && row2 != null && row1.length == row2.length;
+      for(int i = 0; i < row1.length; i++) {
+        int comp = Integer.compare(row1[i], row2[i]);
+        if(comp != 0) {
+          return comp;
+        }
+      }
+      return 0;
+    }
   }
   protected String makeValuesClause(int[][] rows) {
     return TestTxnCommands2.makeValuesClause(rows);
@@ -181,7 +209,7 @@ public abstract class TxnCommandsBaseForTests {
   private static void runCompactorThread(HiveConf hiveConf, CompactorThreadType type)
       throws Exception {
     AtomicBoolean stop = new AtomicBoolean(true);
-    CompactorThread t = null;
+    CompactorThread t;
     switch (type) {
       case INITIATOR:
         t = new Initiator();
@@ -208,12 +236,12 @@ public abstract class TxnCommandsBaseForTests {
     } catch (CommandProcessorException e) {
       throw new RuntimeException(stmt + " failed: " + e);
     }
-    List<String> rs = new ArrayList<String>();
+    List<String> rs = new ArrayList<>();
     d.getResults(rs);
     return rs;
   }
 
-  CommandProcessorException runStatementOnDriverNegative(String stmt) throws Exception {
+  protected CommandProcessorException runStatementOnDriverNegative(String stmt) {
     try {
       d.run(stmt);
     } catch (CommandProcessorException e) {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
index c6b2fee..01f86ce 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestHiveMetaStoreChecker.java
@@ -31,7 +31,6 @@ import java.util.Random;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.CheckResult;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreChecker;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
@@ -40,7 +39,10 @@ import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetastoreException;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.thrift.TException;
@@ -75,21 +77,21 @@ public class TestHiveMetaStoreChecker {
   @Before
   public void setUp() throws Exception {
     hive = Hive.get();
-    hive.getConf().setIntVar(HiveConf.ConfVars.METASTORE_FS_HANDLER_THREADS_COUNT, 15);
-    hive.getConf().set(HiveConf.ConfVars.HIVE_MSCK_PATH_VALIDATION.varname, "throw");
+    hive.getConf().set(MetastoreConf.ConfVars.FS_HANDLER_THREADS_COUNT.getVarname(), "15");
+    hive.getConf().set(MetastoreConf.ConfVars.MSCK_PATH_VALIDATION.getVarname(), "throw");
     msc = new HiveMetaStoreClient(hive.getConf());
     checker = new HiveMetaStoreChecker(msc, hive.getConf());
 
-    partCols = new ArrayList<FieldSchema>();
+    partCols = new ArrayList<>();
     partCols.add(new FieldSchema(partDateName, serdeConstants.STRING_TYPE_NAME, ""));
     partCols.add(new FieldSchema(partCityName, serdeConstants.STRING_TYPE_NAME, ""));
 
-    parts = new ArrayList<Map<String, String>>();
-    Map<String, String> part1 = new HashMap<String, String>();
+    parts = new ArrayList<>();
+    Map<String, String> part1 = new HashMap<>();
     part1.put(partDateName, "2008-01-01");
     part1.put(partCityName, "london");
     parts.add(part1);
-    Map<String, String> part2 = new HashMap<String, String>();
+    Map<String, String> part2 = new HashMap<>();
     part2.put(partDateName, "2008-01-02");
     part2.put(partCityName, "stockholm");
     parts.add(part2);
@@ -116,8 +118,7 @@ public class TestHiveMetaStoreChecker {
 
   @Test
   public void testTableCheck() throws HiveException, IOException, TException, MetastoreException {
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, null, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, null, null, null);
     // we haven't added anything so should return an all ok
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
@@ -125,8 +126,7 @@ public class TestHiveMetaStoreChecker {
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotInMs());
 
     // check table only, should not exist in ms
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(1, result.getTablesNotInMs().size());
     assertEquals(tableName, result.getTablesNotInMs().iterator().next());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
@@ -148,16 +148,14 @@ public class TestHiveMetaStoreChecker {
     table.getTTable().unsetId();
     // now we've got a table, check that it works
     // first check all (1) tables
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, null, null, null, result);
+    result = checker.checkMetastore(catName, dbName, null, null, null);
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotInMs());
 
     // then let's check the one we know about
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotOnFs());
@@ -168,8 +166,7 @@ public class TestHiveMetaStoreChecker {
     fs.delete(table.getPath(), true);
 
     // now this shouldn't find the path on the fs
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(1, result.getTablesNotOnFs().size());
     assertEquals(tableName, result.getTablesNotOnFs().iterator().next());
@@ -184,8 +181,7 @@ public class TestHiveMetaStoreChecker {
     fs.deleteOnExit(fakeTable);
 
     // find the extra table
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, null, null, null, result);
+    result = checker.checkMetastore(catName, dbName, null, null, null);
     assertEquals(1, result.getTablesNotInMs().size());
     assertEquals(fakeTable.getName(), Lists.newArrayList(result.getTablesNotInMs()).get(0));
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
@@ -198,8 +194,7 @@ public class TestHiveMetaStoreChecker {
     hive.createTable(table);
 
     // should return all ok
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, null, null, null, result);
+    result = checker.checkMetastore(catName, dbName, null, null, null);
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotOnFs());
@@ -213,17 +208,14 @@ public class TestHiveMetaStoreChecker {
   @Test
   public void testAdditionalPartitionDirs()
     throws HiveException, AlreadyExistsException, IOException, MetastoreException {
-    Table table = createTestTable();
+    Table table = createTestTable(false);
     List<Partition> partitions = hive.getPartitions(table);
     assertEquals(2, partitions.size());
     // add a fake partition dir on fs
     fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
-    Path fakePart = new Path(table.getDataLocation().toString(),
+    addFolderToPath(fs, table.getDataLocation().toString(),
         partDateName + "=2017-01-01/" + partCityName + "=paloalto/fakePartCol=fakepartValue");
-    fs.mkdirs(fakePart);
-    fs.deleteOnExit(fakePart);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String> emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String> emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
@@ -234,16 +226,13 @@ public class TestHiveMetaStoreChecker {
   @Test(expected = MetastoreException.class)
   public void testInvalidPartitionKeyName()
     throws HiveException, AlreadyExistsException, IOException, MetastoreException {
-    Table table = createTestTable();
+    Table table = createTestTable(false);
     List<Partition> partitions = hive.getPartitions(table);
     assertEquals(2, partitions.size());
     // add a fake partition dir on fs
     fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
-    Path fakePart = new Path(table.getDataLocation().toString(),
-        "fakedate=2009-01-01/fakecity=sanjose");
-    fs.mkdirs(fakePart);
-    fs.deleteOnExit(fakePart);
-    checker.checkMetastore(catName, dbName, tableName, null, null, new CheckResult());
+    addFolderToPath(fs, table.getDataLocation().toString(),"fakedate=2009-01-01/fakecity=sanjose");
+    checker.checkMetastore(catName, dbName, tableName, null, null);
   }
 
   /*
@@ -253,20 +242,16 @@ public class TestHiveMetaStoreChecker {
   @Test
   public void testSkipInvalidPartitionKeyName()
     throws HiveException, AlreadyExistsException, IOException, MetastoreException {
-    hive.getConf().set(HiveConf.ConfVars.HIVE_MSCK_PATH_VALIDATION.varname, "skip");
+    hive.getConf().set(MetastoreConf.ConfVars.MSCK_PATH_VALIDATION.getVarname(), "skip");
     checker = new HiveMetaStoreChecker(msc, hive.getConf());
-    Table table = createTestTable();
+    Table table = createTestTable(false);
     List<Partition> partitions = hive.getPartitions(table);
     assertEquals(2, partitions.size());
     // add a fake partition dir on fs
     fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
-    Path fakePart =
-        new Path(table.getDataLocation().toString(), "fakedate=2009-01-01/fakecity=sanjose");
-    fs.mkdirs(fakePart);
-    fs.deleteOnExit(fakePart);
+    addFolderToPath(fs, table.getDataLocation().toString(),"fakedate=2009-01-01/fakecity=sanjose");
     createPartitionsDirectoriesOnFS(table, 2);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String> emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String> emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
@@ -274,35 +259,129 @@ public class TestHiveMetaStoreChecker {
     assertEquals(2, result.getPartitionsNotInMs().size());
   }
 
-  private Table createTestTable() throws HiveException, AlreadyExistsException {
-    Database db = new Database();
-    db.setName(dbName);
-    hive.createDatabase(db, true);
-
-    Table table = new Table(dbName, tableName);
-    table.setDbName(dbName);
-    table.setInputFormatClass(TextInputFormat.class);
-    table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
-    table.setPartCols(partCols);
+  /*
+   * Tests the case when we have normal delta_dirs in the partition folder
+   * does not throw HiveException
+   */
+  @Test
+  public void testAddPartitionNormalDeltas() throws Exception {
+    Table table = createTestTable(true);
+    List<Partition> partitions = hive.getPartitions(table);
+    assertEquals(2, partitions.size());
+    // add a partition dir on fs
+    fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
+    Path newPart = addFolderToPath(fs, table.getDataLocation().toString(),
+        partDateName + "=2017-01-01/" + partCityName + "=paloalto");
+
+    // Add a few deltas
+    addFolderToPath(fs, newPart.toString(), "delta_0000001_0000001_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000010_0000010_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000101_0000101_0000");
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
+    assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
+    assertEquals(1, result.getPartitionsNotInMs().size());
+    // Found the highest writeId
+    assertEquals(101, result.getPartitionsNotInMs().iterator().next().getMaxWriteId());
+    assertEquals(0, result.getPartitionsNotInMs().iterator().next().getMaxTxnId());
+  }
+  /*
+   * Tests the case when we have normal delta_dirs in the partition folder
+   * does not throw HiveException
+   */
+  @Test
+  public void testAddPartitionCompactedDeltas() throws Exception {
+    Table table = createTestTable(true);
+    List<Partition> partitions = hive.getPartitions(table);
+    assertEquals(2, partitions.size());
+    // add a partition dir on fs
+    fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
+    Path newPart = addFolderToPath(fs, table.getDataLocation().toString(),
+        partDateName + "=2017-01-01/" + partCityName + "=paloalto");
+
+    // Add a few deltas
+    addFolderToPath(fs, newPart.toString(), "delta_0000001_0000001_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000010_0000015_v0000067");
+    addFolderToPath(fs, newPart.toString(), "delta_0000101_0000120_v0000087");
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
+    assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
+    assertEquals(1, result.getPartitionsNotInMs().size());
+    // Found the highest writeId
+    assertEquals(120, result.getPartitionsNotInMs().iterator().next().getMaxWriteId());
+    assertEquals(87, result.getPartitionsNotInMs().iterator().next().getMaxTxnId());
+  }
+  @Test
+  public void testAddPartitionCompactedBase() throws Exception {
+    Table table = createTestTable(true);
+    List<Partition> partitions = hive.getPartitions(table);
+    assertEquals(2, partitions.size());
+    // add a partition dir on fs
+    fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
+    Path newPart = addFolderToPath(fs, table.getDataLocation().toString(),
+        partDateName + "=2017-01-01/" + partCityName + "=paloalto");
+
+    // Add a few deltas
+    addFolderToPath(fs, newPart.toString(), "delta_0000001_0000001_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000002_0000002_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000003_0000003_0000");
+    addFolderToPath(fs, newPart.toString(), "base_0000003_v0000200");
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
+    assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
+    assertEquals(1, result.getPartitionsNotInMs().size());
+    // Found the highest writeId
+    assertEquals(3, result.getPartitionsNotInMs().iterator().next().getMaxWriteId());
+    assertEquals(200, result.getPartitionsNotInMs().iterator().next().getMaxTxnId());
+  }
 
-    hive.createTable(table);
-    table = hive.getTable(dbName, tableName);
-    Assert.assertTrue(table.getTTable().isSetId());
-    table.getTTable().unsetId();
+  @Test
+  public void testAddPartitionMMBase() throws Exception {
+    Table table = createTestTable(true);
+    List<Partition> partitions = hive.getPartitions(table);
+    assertEquals(2, partitions.size());
+    // add a partition dir on fs
+    fs = partitions.get(0).getDataLocation().getFileSystem(hive.getConf());
+    Path newPart = addFolderToPath(fs, table.getDataLocation().toString(),
+        partDateName + "=2017-01-01/" + partCityName + "=paloalto");
+
+    // Add a few deltas
+    addFolderToPath(fs, newPart.toString(), "delta_0000001_0000001_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000002_0000002_0000");
+    addFolderToPath(fs, newPart.toString(), "delta_0000003_0000003_0000");
+    addFolderToPath(fs, newPart.toString(), "base_0000004");
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
+    assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
+    assertEquals(1, result.getPartitionsNotInMs().size());
+    // Found the highest writeId
+    assertEquals(4, result.getPartitionsNotInMs().iterator().next().getMaxWriteId());
+    assertEquals(0, result.getPartitionsNotInMs().iterator().next().getMaxTxnId());
+  }
 
-    for (Map<String, String> partSpec : parts) {
-      hive.createPartition(table, partSpec);
-    }
-    return table;
+  @Test
+  public void testNoNPartitionedTable() throws Exception {
+    Table table = createNonPartitionedTable();
+    // add a partition dir on fs
+    fs = table.getDataLocation().getFileSystem(hive.getConf());
+
+    Path tablePath = table.getDataLocation();
+
+    // Add a few deltas
+    addFolderToPath(fs, tablePath.toString(), "delta_0000001_0000001_0000");
+    addFolderToPath(fs, tablePath.toString(), "delta_0000002_0000002_0000");
+    addFolderToPath(fs, tablePath.toString(), "delta_0000003_0000003_0000");
+    addFolderToPath(fs, tablePath.toString(), "base_0000003_v0000200");
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
+    assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
+    assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotInMs());
+    // Found the highest writeId
+    assertEquals(3, result.getMaxWriteId());
+    assertEquals(200, result.getMaxTxnId());
   }
 
   @Test
   public void testPartitionsCheck() throws HiveException,
     IOException, TException, MetastoreException {
-    Table table = createTestTable();
+    Table table = createTestTable(false);
 
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     // all is well
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
@@ -320,8 +399,7 @@ public class TestHiveMetaStoreChecker {
     fs = partToRemovePath.getFileSystem(hive.getConf());
     fs.delete(partToRemovePath, true);
 
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    result = checker.checkMetastore(catName, dbName, tableName, null, null);
     // missing one partition on fs
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
@@ -336,8 +414,7 @@ public class TestHiveMetaStoreChecker {
     // cleanup
     hive.dropTable(dbName, tableName, true, true);
     hive.createTable(table);
-    result = new CheckResult();
-    checker.checkMetastore(catName, dbName, null, null, null, result);
+    result = checker.checkMetastore(catName, dbName, null, null, null);
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotOnFs());
@@ -379,35 +456,34 @@ public class TestHiveMetaStoreChecker {
     assertFalse(fs.exists(fakeTable));
   }
 
-  /*
-   * Test multi-threaded implementation of checker to find out missing partitions
+  /**
+   * Test multi-threaded implementation of checker to find out missing partitions.
+   * @throws Exception ex
    */
   @Test
-  public void testPartitionsNotInMs() throws HiveException, AlreadyExistsException, IOException, MetastoreException {
+  public void testPartitionsNotInMs() throws Exception {
     Table testTable = createPartitionedTestTable(dbName, tableName, 2, 0);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 10);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String>emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String>emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult>emptySet(), result.getPartitionsNotOnFs());
     assertEquals(10, result.getPartitionsNotInMs().size());
   }
 
-  /*
-   * Tests single threaded implementation of checkMetastore
+  /**
+   * Tests single threaded implementation of checkMetastore.
+   * @throws Exception ex
    */
   @Test
-  public void testSingleThreadedCheckMetastore()
-    throws HiveException, AlreadyExistsException, IOException, MetastoreException {
+  public void testSingleThreadedCheckMetastore() throws Exception {
     // set num of threads to 0 so that single-threaded checkMetastore is called
-    hive.getConf().setIntVar(HiveConf.ConfVars.METASTORE_FS_HANDLER_THREADS_COUNT, 0);
+    hive.getConf().set(MetastoreConf.ConfVars.FS_HANDLER_THREADS_COUNT.getVarname(), "0");
     Table testTable = createPartitionedTestTable(dbName, tableName, 2, 0);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 10);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null,  result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String> emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String> emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
@@ -417,22 +493,18 @@ public class TestHiveMetaStoreChecker {
   /**
    * Tests single threaded implementation for deeply nested partitioned tables
    *
-   * @throws HiveException
-   * @throws AlreadyExistsException
-   * @throws IOException
+   * @throws Exception ex
    */
   @Test
-  public void testSingleThreadedDeeplyNestedTables()
-    throws HiveException, AlreadyExistsException, IOException, MetastoreException {
+  public void testSingleThreadedDeeplyNestedTables() throws Exception {
     // set num of threads to 0 so that single-threaded checkMetastore is called
-    hive.getConf().setIntVar(HiveConf.ConfVars.METASTORE_FS_HANDLER_THREADS_COUNT, 0);
+    hive.getConf().set(MetastoreConf.ConfVars.FS_HANDLER_THREADS_COUNT.getVarname(), "0");
     int poolSize = 2;
     // create a deeply nested table which has more partition keys than the pool size
     Table testTable = createPartitionedTestTable(dbName, tableName, poolSize + 2, 0);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 10);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String> emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String> emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
@@ -442,21 +514,17 @@ public class TestHiveMetaStoreChecker {
   /**
    * Tests the case when the number of partition keys are more than the threadpool size.
    *
-   * @throws HiveException
-   * @throws AlreadyExistsException
-   * @throws IOException
+   * @throws Exception ex
    */
   @Test
-  public void testDeeplyNestedPartitionedTables()
-    throws HiveException, AlreadyExistsException, IOException, MetastoreException {
-    hive.getConf().setIntVar(HiveConf.ConfVars.METASTORE_FS_HANDLER_THREADS_COUNT, 2);
+  public void testDeeplyNestedPartitionedTables() throws Exception {
+    hive.getConf().set(MetastoreConf.ConfVars.FS_HANDLER_THREADS_COUNT.getVarname(), "2");
     int poolSize = 2;
     // create a deeply nested table which has more partition keys than the pool size
     Table testTable = createPartitionedTestTable(dbName, tableName, poolSize + 2, 0);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 10);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String> emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String> emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
@@ -466,12 +534,10 @@ public class TestHiveMetaStoreChecker {
   /**
    * Test if checker throws HiveException when the there is a dummy directory present in the nested level
    * of sub-directories
-   * @throws AlreadyExistsException
-   * @throws IOException
-   * @throws HiveException
+   * @throws Exception ex
    */
   @Test
-  public void testErrorForMissingPartitionColumn() throws AlreadyExistsException, IOException, HiveException {
+  public void testErrorForMissingPartitionColumn() throws Exception {
     Table testTable = createPartitionedTestTable(dbName, tableName, 2, 0);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 10);
@@ -481,59 +547,51 @@ public class TestHiveMetaStoreChecker {
     sb.append("dummyPart=error");
     createDirectory(sb.toString());
     //check result now
-    CheckResult result = new CheckResult();
     Exception exception = null;
     try {
-      checker.checkMetastore(catName, dbName, tableName, null, null, result);
+      checker.checkMetastore(catName, dbName, tableName, null, null);
     } catch (Exception e) {
       exception = e;
     }
-    assertTrue("Expected MetastoreException", exception!=null && exception instanceof MetastoreException);
+    assertTrue("Expected MetastoreException", exception instanceof MetastoreException);
     createFile(sb.toString(), "dummyFile");
-    result = new CheckResult();
     exception = null;
     try {
-      checker.checkMetastore(catName, dbName, tableName, null, null, result);
+      checker.checkMetastore(catName, dbName, tableName, null, null);
     } catch (Exception e) {
       exception = e;
     }
-    assertTrue("Expected MetastoreException", exception!=null && exception instanceof MetastoreException);
+    assertTrue("Expected MetastoreException", exception instanceof MetastoreException);
   }
 
   /**
    * Tests if there exists a unknown partition directory on the FS with in-valid order of partition
    * keys than what is specified in table specification.
    *
-   * @throws AlreadyExistsException
-   * @throws HiveException
-   * @throws IOException
+   * @throws Exception ex
    */
   @Test(expected = MetastoreException.class)
-  public void testInvalidOrderForPartitionKeysOnFS()
-    throws AlreadyExistsException, HiveException, IOException, MetastoreException {
+  public void testInvalidOrderForPartitionKeysOnFS() throws Exception {
     Table testTable = createPartitionedTestTable(dbName, tableName, 2, 0);
     // add 10 partitions on the filesystem
     createInvalidPartitionDirsOnFS(testTable, 10);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    checker.checkMetastore(catName, dbName, tableName, null, null);
   }
 
-  /*
-   * In skip mode msck should ignore invalid partitions instead of
-   * throwing exception
+  /**
+   * In skip mode msck should ignore invalid partitions instead of throwing exception.
+   * @throws Exception ex
    */
   @Test
-  public void testSkipInvalidOrderForPartitionKeysOnFS()
-    throws AlreadyExistsException, HiveException, IOException, MetastoreException {
-    hive.getConf().set(HiveConf.ConfVars.HIVE_MSCK_PATH_VALIDATION.varname, "skip");
+  public void testSkipInvalidOrderForPartitionKeysOnFS() throws Exception{
+    hive.getConf().set(MetastoreConf.ConfVars.MSCK_PATH_VALIDATION.getVarname(), "skip");
     checker = new HiveMetaStoreChecker(msc, hive.getConf());
     Table testTable = createPartitionedTestTable(dbName, tableName, 2, 0);
     // add 10 partitions on the filesystem
     createInvalidPartitionDirsOnFS(testTable, 2);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 2);
-    CheckResult result = new CheckResult();
-    checker.checkMetastore(catName, dbName, tableName, null, null, result);
+    CheckResult result = checker.checkMetastore(catName, dbName, tableName, null, null);
     assertEquals(Collections.<String> emptySet(), result.getTablesNotInMs());
     assertEquals(Collections.<String> emptySet(), result.getTablesNotOnFs());
     assertEquals(Collections.<CheckResult.PartitionResult> emptySet(), result.getPartitionsNotOnFs());
@@ -541,15 +599,15 @@ public class TestHiveMetaStoreChecker {
     assertEquals(2, result.getPartitionsNotInMs().size());
   }
 
-  /*
+  /**
    * Test if single-threaded implementation checker throws HiveException when the there is a dummy
-   * directory present in the nested level
+   * directory present in the nested level.
+   * @throws Exception ex
    */
   @Test
-  public void testErrorForMissingPartitionsSingleThreaded()
-      throws AlreadyExistsException, HiveException, IOException {
+  public void testErrorForMissingPartitionsSingleThreaded() throws Exception {
     // set num of threads to 0 so that single-threaded checkMetastore is called
-    hive.getConf().setIntVar(HiveConf.ConfVars.METASTORE_FS_HANDLER_THREADS_COUNT, 0);
+    hive.getConf().set(MetastoreConf.ConfVars.FS_HANDLER_THREADS_COUNT.getVarname(), "0");
     Table testTable = createPartitionedTestTable(dbName, tableName, 2, 0);
     // add 10 partitions on the filesystem
     createPartitionsDirectoriesOnFS(testTable, 10);
@@ -559,23 +617,21 @@ public class TestHiveMetaStoreChecker {
     sb.append("dummyPart=error");
     createDirectory(sb.toString());
     // check result now
-    CheckResult result = new CheckResult();
     Exception exception = null;
     try {
-      checker.checkMetastore(catName, dbName, tableName, null, null, result);
+      checker.checkMetastore(catName, dbName, tableName, null, null);
     } catch (Exception e) {
       exception = e;
     }
-    assertTrue("Expected MetastoreException", exception!=null && exception instanceof MetastoreException);
+    assertTrue("Expected MetastoreException", exception instanceof MetastoreException);
     createFile(sb.toString(), "dummyFile");
-    result = new CheckResult();
     exception = null;
     try {
-      checker.checkMetastore(catName, dbName, tableName, null, null, result);
+      checker.checkMetastore(catName, dbName, tableName, null, null);
     } catch (Exception e) {
       exception = e;
     }
-    assertTrue("Expected MetastoreException", exception!=null && exception instanceof MetastoreException);
+    assertTrue("Expected MetastoreException", exception instanceof MetastoreException);
   }
   /**
    * Creates a test partitioned table with the required level of nested partitions and number of
@@ -586,12 +642,11 @@ public class TestHiveMetaStoreChecker {
    * @param numOfPartKeys - Number of partition keys (nested levels of sub-directories in base table
    *          path)
    * @param valuesPerPartition - If greater than 0 creates valuesPerPartition dummy partitions
-   * @return
-   * @throws AlreadyExistsException
-   * @throws HiveException
+   * @return The new table
+   * @throws Exception ex
    */
-  private Table createPartitionedTestTable(String dbName, String tableName, int numOfPartKeys,
-      int valuesPerPartition) throws AlreadyExistsException, HiveException {
+  private Table createPartitionedTestTable(String dbName, String tableName, int numOfPartKeys, int valuesPerPartition)
+      throws Exception {
     Database db = new Database();
     db.setName(dbName);
     hive.createDatabase(db, true);
@@ -601,9 +656,9 @@ public class TestHiveMetaStoreChecker {
     table.setInputFormatClass(TextInputFormat.class);
     table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
     // create partition key schema
-    ArrayList<FieldSchema> partKeys = new ArrayList<FieldSchema>();
+    ArrayList<FieldSchema> partKeys = new ArrayList<>();
     for (int i = 1; i <= numOfPartKeys; i++) {
-      String partName = "part" + String.valueOf(i);
+      String partName = "part" + i;
       partKeys.add(new FieldSchema(partName, serdeConstants.STRING_TYPE_NAME, ""));
     }
     table.setPartCols(partKeys);
@@ -614,7 +669,7 @@ public class TestHiveMetaStoreChecker {
       return table;
     }
     // create partition specs
-    ArrayList<Map<String, String>> partitionSpecs = new ArrayList<Map<String, String>>();
+    ArrayList<Map<String, String>> partitionSpecs = new ArrayList<>();
     for (int partKeyIndex = 0; partKeyIndex < numOfPartKeys; partKeyIndex++) {
       String partName = partKeys.get(partKeyIndex).getName();
       Map<String, String> partMap = new HashMap<>();
@@ -643,7 +698,7 @@ public class TestHiveMetaStoreChecker {
    * @param numPartitions - Number of partitions to be created
    * @param reverseOrder - If set to true creates the partition sub-directories in the reverse order
    *          of specified by partition keys defined for the table
-   * @throws IOException
+   * @throws IOException ex
    */
   private void createPartitionsDirectoriesOnFS(Table table, int numPartitions, boolean reverseOrder) throws IOException {
     String path = table.getDataLocation().toString();
@@ -687,7 +742,8 @@ public class TestHiveMetaStoreChecker {
    * of sub-directories compared to the partition keys defined in the table. Eg. if the
    * partition keys defined in table are (a int, b int, c int) this method will create
    * an invalid directory c=val_1/b=val_1/a=val_1
-   * @param table
+   * @param table table
+   * @param numPartitions Number of partitions to create
    * @throws IOException
    */
   private void createInvalidPartitionDirsOnFS(Table table, int numPartitions) throws IOException {
@@ -709,4 +765,58 @@ public class TestHiveMetaStoreChecker {
     fs.createNewFile(new Path(partPath + Path.SEPARATOR + "dummydata2"));
     fs.deleteOnExit(part);
   }
+
+  private Path addFolderToPath(FileSystem fs, String rootPath, String folder) throws IOException{
+    Path folderParth = new Path(rootPath, folder);
+    fs.mkdirs(folderParth);
+    fs.deleteOnExit(folderParth);
+    return folderParth;
+  }
+
+  private Table createTestTable(boolean transactional) throws HiveException, AlreadyExistsException {
+    Database db = new Database();
+    db.setName(dbName);
+    hive.createDatabase(db, true);
+
+    Table table = new Table(dbName, tableName);
+    table.setDbName(dbName);
+    if (transactional) {
+      table.setInputFormatClass(OrcInputFormat.class);
+      table.setOutputFormatClass(OrcOutputFormat.class);
+    } else {
+      table.setInputFormatClass(TextInputFormat.class);
+      table.setOutputFormatClass(HiveIgnoreKeyTextOutputFormat.class);
+    }
+    table.setPartCols(partCols);
+    if (transactional) {
+      table.setProperty("transactional", "true");
+    }
+
+    hive.createTable(table);
+    table = hive.getTable(dbName, tableName);
+    Assert.assertTrue(table.getTTable().isSetId());
+    table.getTTable().unsetId();
+
+    for (Map<String, String> partSpec : parts) {
+      hive.createPartition(table, partSpec);
+    }
+    return table;
+  }
+  private Table createNonPartitionedTable() throws Exception {
+    Database db = new Database();
+    db.setName(dbName);
+    hive.createDatabase(db, true);
+
+    Table table = new Table(dbName, tableName);
+    table.setDbName(dbName);
+    table.setInputFormatClass(OrcInputFormat.class);
+    table.setOutputFormatClass(OrcOutputFormat.class);
+    table.setProperty("transactional", "true");
+
+    hive.createTable(table);
+    table = hive.getTable(dbName, tableName);
+    Assert.assertTrue(table.getTTable().isSetId());
+    table.getTTable().unsetId();
+    return table;
+  }
 }
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMSCKRepairOnAcid.java b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMSCKRepairOnAcid.java
new file mode 100644
index 0000000..d1254ab
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestMSCKRepairOnAcid.java
@@ -0,0 +1,509 @@
+/*
+ * 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.metadata;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
+import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
+import org.apache.hadoop.hive.ql.TxnCommandsBaseForTests;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.List;
+
+/**
+ * Tests for the MSCK REPAIR TABLE operation on transactional tables.
+ */
+public class TestMSCKRepairOnAcid extends TxnCommandsBaseForTests {
+
+  private static final String TEST_DATA_DIR = new File(
+      System.getProperty("java.io.tmpdir") + File.separator + TestMSCKRepairOnAcid.class.getCanonicalName()
+          + "-" + System.currentTimeMillis()).getPath().replaceAll("\\\\", "/");
+
+  private final String acidTblPartMsck = "acidtblpartmsck";
+
+  @Override
+  protected String getTestDataDir() {
+    return TEST_DATA_DIR;
+  }
+
+  /**
+   * A new partition copied under a table containing only deltas.
+   * @throws Exception ex
+   */
+  @Test
+  public void testAddPartitionDeltas() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p1'),(2,3,'p1'),(3,4,'p1')");
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " where p='p1' order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck + "/p=p1"));
+    Assert.assertEquals(2, fileStatuses.length);
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " where p='p1' order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+
+  /**
+   * More partition copied under a table containing only deltas.
+   * @throws Exception ex
+   */
+  @Test
+  public void testAddMultiPartitionDeltas() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(4,4,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 }, { 4, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p2"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+
+  /**
+   * A new partition copied under a table containing only deltas, but the table already contains allocated writes,
+   * and the restored partition has higher writeId.
+   * @throws Exception ex
+   */
+  @Test
+  public void testAddPartitionHighWriteIdException() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // Insert data in p1 so we allocate writeId in the msck table
+    runStatementOnDriver("insert into " + acidTblPartMsck + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p2"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, false, hiveConf);
+
+    // One partition written, one copied
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck));
+    Assert.assertEquals(2, fileStatuses.length);
+
+    // call msk repair, it should fail, since it will find a delta folder with writeId 2
+    // that is higher than the allocated max in the table
+    CommandProcessorException e = runStatementOnDriverNegative("msck repair table " + acidTblPartMsck);
+    Assert.assertEquals(-1, e.getErrorCode());
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+  /**
+   * A new partition copied under a table containing only deltas, but the table already contains allocated writes,
+   * and the restored partition has lower writeId.
+   * @throws Exception ex
+   */
+  @Test
+  public void testAddPartitionLowerWriteId() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // Insert data in p2 so we allocate writeId in the msck table
+    runStatementOnDriver("insert into " + acidTblPartMsck + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+    runStatementOnDriver("insert into " + acidTblPartMsck + " partition(p) values(4,5,'p2')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, false, hiveConf);
+
+    // One partition written, one copied
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck));
+    Assert.assertEquals(2, fileStatuses.length);
+
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    // The allocated writeId should be 2 and the copied partition has 1 as maximum, so we are on the safe side
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " order by a, b");
+    int[][] expected2 = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 }, { 4, 5 } };
+    Assert.assertEquals(stringifyValues(expected2), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+  /**
+   * A new partition copied under a table containing compactd delta.
+   * @throws Exception ex
+   */
+  @Test
+  public void testAddPartitionMinorCompacted() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(4,4,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    //run Compaction
+    runStatementOnDriver("alter table " + Table.ACIDTBLPART + " partition (p='p1') compact 'minor'");
+    runWorker(hiveConf);
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 }, { 4, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p2"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck + "/p=p1"));
+    // two + one delta
+    Assert.assertEquals(3, fileStatuses.length);
+
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+  /**
+   * A new partition copied under a table containing compacted base.
+   * @throws Exception ex
+   */
+  @Test
+  public void testAddPartitionMajorCompacted() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(4,4,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    //run Compaction
+    runStatementOnDriver("alter table " + Table.ACIDTBLPART + " partition (p='p1') compact 'major'");
+    runWorker(hiveConf);
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 }, { 4, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p2"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck + "/p=p1"));
+    // two delta + a base
+    Assert.assertEquals(3, fileStatuses.length);
+
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+
+  /**
+   * Delete one partition from a table, drop the partition from the HMS, then restore the partition
+   * and repair the table.
+   * @throws Exception ex
+   */
+  @Test
+  public void testBackUpAndRestorePartition() throws Exception {
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(4,4,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 }, { 4, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    fs.mkdirs(new Path(getWarehouseDir(), "mybackup"));
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), "mybackup"), true, hiveConf);
+
+    // call msk repair to remove partition p1
+    runStatementOnDriver("msck repair table " + Table.ACIDTBLPART + " SYNC PARTITIONS");
+
+    r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected2 = { { 1, 2 }, { 2, 3 }, { 3, 4 }};
+    Assert.assertEquals(stringifyValues(expected2), r);
+
+    // copy the data back
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + "mybackup" + "/p=p1"), fs,
+        new Path(getWarehouseDir(), Table.ACIDTBLPART.toString().toLowerCase()), true, hiveConf);
+
+    // call msk repair to add the partition back
+    runStatementOnDriver("msck repair table " + Table.ACIDTBLPART);
+
+    r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+  }
+
+  /**
+   * Add a new partition to the table that contains a compacted folder with the visibilityTxnId
+   * higher than the HighWaterMark in the HMS. This could happen if the HMS data was lost and we try to
+   * repair the metadata. The system will set the txnId forward in the HMS.
+   * @throws Exception
+   */
+  @Test
+  public void testAddPartitionHighVisibilityId() throws Exception {
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(4,4,'p1')");
+    runStatementOnDriver("insert into " + Table.ACIDTBLPART + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    //run Compaction
+    runStatementOnDriver("alter table " + Table.ACIDTBLPART + " partition (p='p1') compact 'minor'");
+    runWorker(hiveConf);
+    runCleaner(hiveConf);
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBLPART + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 1, 2 }, { 2, 2 }, { 2, 3 }, { 3, 3 }, { 3, 4 }, { 4, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + Table.ACIDTBLPART.toString().toLowerCase() + "/p=p2"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck + "/p=p1"));
+    // one compacted delta
+    Assert.assertEquals(1, fileStatuses.length);
+    // Rename the deltaDir to add a higher visibility transactionId
+    Path deltaDir = fileStatuses[0].getPath();
+    AcidUtils.ParsedDelta parsedDelta = AcidUtils.parsedDelta(deltaDir, fs);
+    long oldTxnId = parsedDelta.getVisibilityTxnId();
+    String newDeltaDir = AcidUtils.addVisibilitySuffix(deltaDir.toString().substring(0, deltaDir.toString().length() - 9), oldTxnId + 100);
+    fs.rename(deltaDir, new Path(newDeltaDir));
+
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    Long nextTxnId = txnHandler.openTxns(new OpenTxnRequest(1, "localhost", "me")).getTxn_ids().get(0);
+    Assert.assertTrue("TxnId should be incremented", nextTxnId > (oldTxnId + 100));
+    txnHandler.abortTxn(new AbortTxnRequest(nextTxnId));
+    // We have to wait for the MetastoreConf.TXN_OPENTXN_TIMEOUT otherwise our hypothetical txnId in the visibilityTxnId
+    // will be considered open and we will not read the folder
+    Thread.sleep(1000);
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+  }
+
+  /**
+   * Add one partition to an insert-only table that contains an insert overwrite base folder.
+   * @throws Exception
+   */
+  @Test
+  public void testAddPartitionMMInsertOverwrite() throws Exception{
+    final String mmTable = "mmtblpartmsck";
+    final String sourceTable = "nonacidpartmsck";
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+    runStatementOnDriver("drop table if exists " + mmTable);
+    runStatementOnDriver("drop table if exists " + sourceTable);
+    runStatementOnDriver("create table " + mmTable + "(a int, b int) partitioned by (p string) stored as orc"
+        + " TBLPROPERTIES ('transactional'='true', 'transactional_properties'='insert_only')");
+    runStatementOnDriver("create table " + sourceTable + "(a int, b int) partitioned by (p string) stored as orc"
+        + " TBLPROPERTIES ('transactional'='false')");
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + mmTable + " partition(p) values(1,1,'p1'),(2,2,'p1'),(3,3,'p1')");
+    runStatementOnDriver("insert into " + mmTable + " partition(p) values(4,4,'p1')");
+    runStatementOnDriver("insert into " + mmTable + " partition(p) values(1,2,'p2'),(2,3,'p2'),(3,4,'p2')");
+
+    // Insert some to the source
+    runStatementOnDriver("insert into " + sourceTable + " partition(p) values(10,10,'p1'),(20,20,'p1'),(30,30,'p1')");
+    runStatementOnDriver("insert into " + sourceTable + " partition(p) values(40,40,'p1')");
+
+    // Overwrite a partition
+    runStatementOnDriver("insert overwrite table " + mmTable + " PARTITION(p='p1') " +
+        " select a,b from " + sourceTable + " where " + sourceTable + ".p='p1'");
+
+    List<String> r = runStatementOnDriver("select a, b from " + mmTable+ " order by a, b");
+    int[][] expected = { { 1, 2 }, { 2, 3 }, { 3, 4 },{ 10, 10 }, { 20, 20 }, { 30, 30 }, {40, 40} };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+    runStatementOnDriver("create table " + acidTblPartMsck
+        + " (a int, b int) partitioned by (p string) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true', 'transactional_properties'='insert_only')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + mmTable + "/p=p1"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+    FileUtil.copy(fs, new Path(getWarehouseDir() + "/" + mmTable + "/p=p2"), fs,
+        new Path(getWarehouseDir(), acidTblPartMsck), false, hiveConf);
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblPartMsck + "/p=p1"));
+    // Two delta and one insert overwrite base
+    Assert.assertEquals(3, fileStatuses.length);
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblPartMsck);
+
+    r = runStatementOnDriver("select a, b from " + acidTblPartMsck + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblPartMsck);
+    runStatementOnDriver("drop table if exists " + mmTable);
+    runStatementOnDriver("drop table if exists " + sourceTable);
+  }
+
+  /**
+   * For non partitioned acid table the writeId and visibility txnId repair should work the same.
+   * @throws Exception ex
+   */
+  @Test
+  public void testNonPartitionedTable() throws Exception {
+    String acidTblMsck = "acidtblmsck";
+    runStatementOnDriver("drop table if exists " + acidTblMsck);
+
+    // Insert few rows
+    runStatementOnDriver("insert into " + Table.ACIDTBL + " values(1,1),(2,2),(3,3)");
+    runStatementOnDriver("insert into " + Table.ACIDTBL + " values(4,4)");
+
+    //run Compaction
+    runStatementOnDriver("alter table " + Table.ACIDTBL + " compact 'minor'");
+    runWorker(hiveConf);
+
+    List<String> r = runStatementOnDriver("select a, b from " + Table.ACIDTBL + " order by a, b");
+    int[][] expected = { { 1, 1 }, { 2, 2 }, { 3, 3 }, { 4, 4 } };
+    Assert.assertEquals(stringifyValues(expected), r);
+    // Create target table
+
+    runStatementOnDriver("create table " + acidTblMsck
+        + " (a int, b int) clustered by (a) into 2 buckets"
+        + " stored as orc TBLPROPERTIES ('transactional'='true')");
+
+    // copy files on fs
+    FileSystem fs = FileSystem.get(hiveConf);
+    for (FileStatus status : fs.listStatus(new Path(getWarehouseDir(), Table.ACIDTBL.toString().toLowerCase()))) {
+      FileUtil.copy(fs, status.getPath(), fs,
+          new Path(getWarehouseDir(), acidTblMsck), false, hiveConf);
+    }
+
+    FileStatus[] fileStatuses = fs.listStatus(new Path(getWarehouseDir(), acidTblMsck));
+    // two + one delta
+    Assert.assertEquals(3, fileStatuses.length);
+
+    // call msk repair
+    runStatementOnDriver("msck repair table " + acidTblMsck);
+
+    r = runStatementOnDriver("select a, b from " + acidTblMsck + " order by a, b");
+    Assert.assertEquals(stringifyValues(expected), r);
+
+    runStatementOnDriver("drop table if exists " + acidTblMsck);
+  }
+}
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MaxAllocatedTableWriteIdRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MaxAllocatedTableWriteIdRequest.java
new file mode 100644
index 0000000..0732d16
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MaxAllocatedTableWriteIdRequest.java
@@ -0,0 +1,490 @@
+/**
+ * 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 MaxAllocatedTableWriteIdRequest implements org.apache.thrift.TBase<MaxAllocatedTableWriteIdRequest, MaxAllocatedTableWriteIdRequest._Fields>, java.io.Serializable, Cloneable, Comparable<MaxAllocatedTableWriteIdRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MaxAllocatedTableWriteIdRequest");
+
+  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 Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new MaxAllocatedTableWriteIdRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new MaxAllocatedTableWriteIdRequestTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tableName; // required
+
+  /** 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");
+
+    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;
+        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
+  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)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MaxAllocatedTableWriteIdRequest.class, metaDataMap);
+  }
+
+  public MaxAllocatedTableWriteIdRequest() {
+  }
+
+  public MaxAllocatedTableWriteIdRequest(
+    String dbName,
+    String tableName)
+  {
+    this();
+    this.dbName = dbName;
+    this.tableName = tableName;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public MaxAllocatedTableWriteIdRequest(MaxAllocatedTableWriteIdRequest other) {
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTableName()) {
+      this.tableName = other.tableName;
+    }
+  }
+
+  public MaxAllocatedTableWriteIdRequest deepCopy() {
+    return new MaxAllocatedTableWriteIdRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tableName = 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 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;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TABLE_NAME:
+      return getTableName();
+
+    }
+    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();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof MaxAllocatedTableWriteIdRequest)
+      return this.equals((MaxAllocatedTableWriteIdRequest)that);
+    return false;
+  }
+
+  public boolean equals(MaxAllocatedTableWriteIdRequest 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;
+    }
+
+    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);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(MaxAllocatedTableWriteIdRequest 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;
+      }
+    }
+    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("MaxAllocatedTableWriteIdRequest(");
+    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;
+    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());
+    }
+
+    // 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 MaxAllocatedTableWriteIdRequestStandardSchemeFactory implements SchemeFactory {
+    public MaxAllocatedTableWriteIdRequestStandardScheme getScheme() {
+      return new MaxAllocatedTableWriteIdRequestStandardScheme();
+    }
+  }
+
+  private static class MaxAllocatedTableWriteIdRequestStandardScheme extends StandardScheme<MaxAllocatedTableWriteIdRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, MaxAllocatedTableWriteIdRequest 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;
+          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, MaxAllocatedTableWriteIdRequest 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();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class MaxAllocatedTableWriteIdRequestTupleSchemeFactory implements SchemeFactory {
+    public MaxAllocatedTableWriteIdRequestTupleScheme getScheme() {
+      return new MaxAllocatedTableWriteIdRequestTupleScheme();
+    }
+  }
+
+  private static class MaxAllocatedTableWriteIdRequestTupleScheme extends TupleScheme<MaxAllocatedTableWriteIdRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, MaxAllocatedTableWriteIdRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tableName);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, MaxAllocatedTableWriteIdRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tableName = iprot.readString();
+      struct.setTableNameIsSet(true);
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MaxAllocatedTableWriteIdResponse.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MaxAllocatedTableWriteIdResponse.java
new file mode 100644
index 0000000..32eba83
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/MaxAllocatedTableWriteIdResponse.java
@@ -0,0 +1,387 @@
+/**
+ * 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 MaxAllocatedTableWriteIdResponse implements org.apache.thrift.TBase<MaxAllocatedTableWriteIdResponse, MaxAllocatedTableWriteIdResponse._Fields>, java.io.Serializable, Cloneable, Comparable<MaxAllocatedTableWriteIdResponse> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("MaxAllocatedTableWriteIdResponse");
+
+  private static final org.apache.thrift.protocol.TField MAX_WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("maxWriteId", org.apache.thrift.protocol.TType.I64, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new MaxAllocatedTableWriteIdResponseStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new MaxAllocatedTableWriteIdResponseTupleSchemeFactory());
+  }
+
+  private long maxWriteId; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    MAX_WRITE_ID((short)1, "maxWriteId");
+
+    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: // MAX_WRITE_ID
+          return MAX_WRITE_ID;
+        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 __MAXWRITEID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  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.MAX_WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("maxWriteId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(MaxAllocatedTableWriteIdResponse.class, metaDataMap);
+  }
+
+  public MaxAllocatedTableWriteIdResponse() {
+  }
+
+  public MaxAllocatedTableWriteIdResponse(
+    long maxWriteId)
+  {
+    this();
+    this.maxWriteId = maxWriteId;
+    setMaxWriteIdIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public MaxAllocatedTableWriteIdResponse(MaxAllocatedTableWriteIdResponse other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.maxWriteId = other.maxWriteId;
+  }
+
+  public MaxAllocatedTableWriteIdResponse deepCopy() {
+    return new MaxAllocatedTableWriteIdResponse(this);
+  }
+
+  @Override
+  public void clear() {
+    setMaxWriteIdIsSet(false);
+    this.maxWriteId = 0;
+  }
+
+  public long getMaxWriteId() {
+    return this.maxWriteId;
+  }
+
+  public void setMaxWriteId(long maxWriteId) {
+    this.maxWriteId = maxWriteId;
+    setMaxWriteIdIsSet(true);
+  }
+
+  public void unsetMaxWriteId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXWRITEID_ISSET_ID);
+  }
+
+  /** Returns true if field maxWriteId is set (has been assigned a value) and false otherwise */
+  public boolean isSetMaxWriteId() {
+    return EncodingUtils.testBit(__isset_bitfield, __MAXWRITEID_ISSET_ID);
+  }
+
+  public void setMaxWriteIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXWRITEID_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case MAX_WRITE_ID:
+      if (value == null) {
+        unsetMaxWriteId();
+      } else {
+        setMaxWriteId((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case MAX_WRITE_ID:
+      return getMaxWriteId();
+
+    }
+    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 MAX_WRITE_ID:
+      return isSetMaxWriteId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof MaxAllocatedTableWriteIdResponse)
+      return this.equals((MaxAllocatedTableWriteIdResponse)that);
+    return false;
+  }
+
+  public boolean equals(MaxAllocatedTableWriteIdResponse that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_maxWriteId = true;
+    boolean that_present_maxWriteId = true;
+    if (this_present_maxWriteId || that_present_maxWriteId) {
+      if (!(this_present_maxWriteId && that_present_maxWriteId))
+        return false;
+      if (this.maxWriteId != that.maxWriteId)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_maxWriteId = true;
+    list.add(present_maxWriteId);
+    if (present_maxWriteId)
+      list.add(maxWriteId);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(MaxAllocatedTableWriteIdResponse other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetMaxWriteId()).compareTo(other.isSetMaxWriteId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetMaxWriteId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxWriteId, other.maxWriteId);
+      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("MaxAllocatedTableWriteIdResponse(");
+    boolean first = true;
+
+    sb.append("maxWriteId:");
+    sb.append(this.maxWriteId);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetMaxWriteId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'maxWriteId' 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 MaxAllocatedTableWriteIdResponseStandardSchemeFactory implements SchemeFactory {
+    public MaxAllocatedTableWriteIdResponseStandardScheme getScheme() {
+      return new MaxAllocatedTableWriteIdResponseStandardScheme();
+    }
+  }
+
+  private static class MaxAllocatedTableWriteIdResponseStandardScheme extends StandardScheme<MaxAllocatedTableWriteIdResponse> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, MaxAllocatedTableWriteIdResponse 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: // MAX_WRITE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.maxWriteId = iprot.readI64();
+              struct.setMaxWriteIdIsSet(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, MaxAllocatedTableWriteIdResponse struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(MAX_WRITE_ID_FIELD_DESC);
+      oprot.writeI64(struct.maxWriteId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class MaxAllocatedTableWriteIdResponseTupleSchemeFactory implements SchemeFactory {
+    public MaxAllocatedTableWriteIdResponseTupleScheme getScheme() {
+      return new MaxAllocatedTableWriteIdResponseTupleScheme();
+    }
+  }
+
+  private static class MaxAllocatedTableWriteIdResponseTupleScheme extends TupleScheme<MaxAllocatedTableWriteIdResponse> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, MaxAllocatedTableWriteIdResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.maxWriteId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, MaxAllocatedTableWriteIdResponse struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.maxWriteId = iprot.readI64();
+      struct.setMaxWriteIdIsSet(true);
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SeedTableWriteIdsRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SeedTableWriteIdsRequest.java
new file mode 100644
index 0000000..69d0016
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SeedTableWriteIdsRequest.java
@@ -0,0 +1,589 @@
+/**
+ * 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 SeedTableWriteIdsRequest implements org.apache.thrift.TBase<SeedTableWriteIdsRequest, SeedTableWriteIdsRequest._Fields>, java.io.Serializable, Cloneable, Comparable<SeedTableWriteIdsRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SeedTableWriteIdsRequest");
+
+  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 SEED_WRITE_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("seedWriteId", org.apache.thrift.protocol.TType.I64, (short)3);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SeedTableWriteIdsRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SeedTableWriteIdsRequestTupleSchemeFactory());
+  }
+
+  private String dbName; // required
+  private String tableName; // required
+  private long seedWriteId; // required
+
+  /** 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"),
+    SEED_WRITE_ID((short)3, "seedWriteId");
+
+    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: // SEED_WRITE_ID
+          return SEED_WRITE_ID;
+        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 __SEEDWRITEID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  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.SEED_WRITE_ID, new org.apache.thrift.meta_data.FieldMetaData("seedWriteId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SeedTableWriteIdsRequest.class, metaDataMap);
+  }
+
+  public SeedTableWriteIdsRequest() {
+  }
+
+  public SeedTableWriteIdsRequest(
+    String dbName,
+    String tableName,
+    long seedWriteId)
+  {
+    this();
+    this.dbName = dbName;
+    this.tableName = tableName;
+    this.seedWriteId = seedWriteId;
+    setSeedWriteIdIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SeedTableWriteIdsRequest(SeedTableWriteIdsRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    if (other.isSetDbName()) {
+      this.dbName = other.dbName;
+    }
+    if (other.isSetTableName()) {
+      this.tableName = other.tableName;
+    }
+    this.seedWriteId = other.seedWriteId;
+  }
+
+  public SeedTableWriteIdsRequest deepCopy() {
+    return new SeedTableWriteIdsRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    this.dbName = null;
+    this.tableName = null;
+    setSeedWriteIdIsSet(false);
+    this.seedWriteId = 0;
+  }
+
+  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 long getSeedWriteId() {
+    return this.seedWriteId;
+  }
+
+  public void setSeedWriteId(long seedWriteId) {
+    this.seedWriteId = seedWriteId;
+    setSeedWriteIdIsSet(true);
+  }
+
+  public void unsetSeedWriteId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SEEDWRITEID_ISSET_ID);
+  }
+
+  /** Returns true if field seedWriteId is set (has been assigned a value) and false otherwise */
+  public boolean isSetSeedWriteId() {
+    return EncodingUtils.testBit(__isset_bitfield, __SEEDWRITEID_ISSET_ID);
+  }
+
+  public void setSeedWriteIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SEEDWRITEID_ISSET_ID, value);
+  }
+
+  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 SEED_WRITE_ID:
+      if (value == null) {
+        unsetSeedWriteId();
+      } else {
+        setSeedWriteId((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case DB_NAME:
+      return getDbName();
+
+    case TABLE_NAME:
+      return getTableName();
+
+    case SEED_WRITE_ID:
+      return getSeedWriteId();
+
+    }
+    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 SEED_WRITE_ID:
+      return isSetSeedWriteId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SeedTableWriteIdsRequest)
+      return this.equals((SeedTableWriteIdsRequest)that);
+    return false;
+  }
+
+  public boolean equals(SeedTableWriteIdsRequest 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_seedWriteId = true;
+    boolean that_present_seedWriteId = true;
+    if (this_present_seedWriteId || that_present_seedWriteId) {
+      if (!(this_present_seedWriteId && that_present_seedWriteId))
+        return false;
+      if (this.seedWriteId != that.seedWriteId)
+        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_seedWriteId = true;
+    list.add(present_seedWriteId);
+    if (present_seedWriteId)
+      list.add(seedWriteId);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SeedTableWriteIdsRequest 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(isSetSeedWriteId()).compareTo(other.isSetSeedWriteId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSeedWriteId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.seedWriteId, other.seedWriteId);
+      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("SeedTableWriteIdsRequest(");
+    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("seedWriteId:");
+    sb.append(this.seedWriteId);
+    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 (!isSetSeedWriteId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'seedWriteId' 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 SeedTableWriteIdsRequestStandardSchemeFactory implements SchemeFactory {
+    public SeedTableWriteIdsRequestStandardScheme getScheme() {
+      return new SeedTableWriteIdsRequestStandardScheme();
+    }
+  }
+
+  private static class SeedTableWriteIdsRequestStandardScheme extends StandardScheme<SeedTableWriteIdsRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SeedTableWriteIdsRequest 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: // SEED_WRITE_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.seedWriteId = iprot.readI64();
+              struct.setSeedWriteIdIsSet(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, SeedTableWriteIdsRequest 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();
+      }
+      oprot.writeFieldBegin(SEED_WRITE_ID_FIELD_DESC);
+      oprot.writeI64(struct.seedWriteId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SeedTableWriteIdsRequestTupleSchemeFactory implements SchemeFactory {
+    public SeedTableWriteIdsRequestTupleScheme getScheme() {
+      return new SeedTableWriteIdsRequestTupleScheme();
+    }
+  }
+
+  private static class SeedTableWriteIdsRequestTupleScheme extends TupleScheme<SeedTableWriteIdsRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SeedTableWriteIdsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.dbName);
+      oprot.writeString(struct.tableName);
+      oprot.writeI64(struct.seedWriteId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SeedTableWriteIdsRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.dbName = iprot.readString();
+      struct.setDbNameIsSet(true);
+      struct.tableName = iprot.readString();
+      struct.setTableNameIsSet(true);
+      struct.seedWriteId = iprot.readI64();
+      struct.setSeedWriteIdIsSet(true);
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SeedTxnIdRequest.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SeedTxnIdRequest.java
new file mode 100644
index 0000000..d327065
--- /dev/null
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/SeedTxnIdRequest.java
@@ -0,0 +1,387 @@
+/**
+ * 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 SeedTxnIdRequest implements org.apache.thrift.TBase<SeedTxnIdRequest, SeedTxnIdRequest._Fields>, java.io.Serializable, Cloneable, Comparable<SeedTxnIdRequest> {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("SeedTxnIdRequest");
+
+  private static final org.apache.thrift.protocol.TField SEED_TXN_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("seedTxnId", org.apache.thrift.protocol.TType.I64, (short)1);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new SeedTxnIdRequestStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new SeedTxnIdRequestTupleSchemeFactory());
+  }
+
+  private long seedTxnId; // required
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SEED_TXN_ID((short)1, "seedTxnId");
+
+    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: // SEED_TXN_ID
+          return SEED_TXN_ID;
+        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 __SEEDTXNID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  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.SEED_TXN_ID, new org.apache.thrift.meta_data.FieldMetaData("seedTxnId", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(SeedTxnIdRequest.class, metaDataMap);
+  }
+
+  public SeedTxnIdRequest() {
+  }
+
+  public SeedTxnIdRequest(
+    long seedTxnId)
+  {
+    this();
+    this.seedTxnId = seedTxnId;
+    setSeedTxnIdIsSet(true);
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public SeedTxnIdRequest(SeedTxnIdRequest other) {
+    __isset_bitfield = other.__isset_bitfield;
+    this.seedTxnId = other.seedTxnId;
+  }
+
+  public SeedTxnIdRequest deepCopy() {
+    return new SeedTxnIdRequest(this);
+  }
+
+  @Override
+  public void clear() {
+    setSeedTxnIdIsSet(false);
+    this.seedTxnId = 0;
+  }
+
+  public long getSeedTxnId() {
+    return this.seedTxnId;
+  }
+
+  public void setSeedTxnId(long seedTxnId) {
+    this.seedTxnId = seedTxnId;
+    setSeedTxnIdIsSet(true);
+  }
+
+  public void unsetSeedTxnId() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SEEDTXNID_ISSET_ID);
+  }
+
+  /** Returns true if field seedTxnId is set (has been assigned a value) and false otherwise */
+  public boolean isSetSeedTxnId() {
+    return EncodingUtils.testBit(__isset_bitfield, __SEEDTXNID_ISSET_ID);
+  }
+
+  public void setSeedTxnIdIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SEEDTXNID_ISSET_ID, value);
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SEED_TXN_ID:
+      if (value == null) {
+        unsetSeedTxnId();
+      } else {
+        setSeedTxnId((Long)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SEED_TXN_ID:
+      return getSeedTxnId();
+
+    }
+    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 SEED_TXN_ID:
+      return isSetSeedTxnId();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof SeedTxnIdRequest)
+      return this.equals((SeedTxnIdRequest)that);
+    return false;
+  }
+
+  public boolean equals(SeedTxnIdRequest that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_seedTxnId = true;
+    boolean that_present_seedTxnId = true;
+    if (this_present_seedTxnId || that_present_seedTxnId) {
+      if (!(this_present_seedTxnId && that_present_seedTxnId))
+        return false;
+      if (this.seedTxnId != that.seedTxnId)
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    List<Object> list = new ArrayList<Object>();
+
+    boolean present_seedTxnId = true;
+    list.add(present_seedTxnId);
+    if (present_seedTxnId)
+      list.add(seedTxnId);
+
+    return list.hashCode();
+  }
+
+  @Override
+  public int compareTo(SeedTxnIdRequest other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+
+    lastComparison = Boolean.valueOf(isSetSeedTxnId()).compareTo(other.isSetSeedTxnId());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetSeedTxnId()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.seedTxnId, other.seedTxnId);
+      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("SeedTxnIdRequest(");
+    boolean first = true;
+
+    sb.append("seedTxnId:");
+    sb.append(this.seedTxnId);
+    first = false;
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetSeedTxnId()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'seedTxnId' 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 SeedTxnIdRequestStandardSchemeFactory implements SchemeFactory {
+    public SeedTxnIdRequestStandardScheme getScheme() {
+      return new SeedTxnIdRequestStandardScheme();
+    }
+  }
+
+  private static class SeedTxnIdRequestStandardScheme extends StandardScheme<SeedTxnIdRequest> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, SeedTxnIdRequest 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: // SEED_TXN_ID
+            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
+              struct.seedTxnId = iprot.readI64();
+              struct.setSeedTxnIdIsSet(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, SeedTxnIdRequest struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      oprot.writeFieldBegin(SEED_TXN_ID_FIELD_DESC);
+      oprot.writeI64(struct.seedTxnId);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class SeedTxnIdRequestTupleSchemeFactory implements SchemeFactory {
+    public SeedTxnIdRequestTupleScheme getScheme() {
+      return new SeedTxnIdRequestTupleScheme();
+    }
+  }
+
+  private static class SeedTxnIdRequestTupleScheme extends TupleScheme<SeedTxnIdRequest> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, SeedTxnIdRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeI64(struct.seedTxnId);
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, SeedTxnIdRequest struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.seedTxnId = iprot.readI64();
+      struct.setSeedTxnIdIsSet(true);
+    }
+  }
+
+}
+
diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
index e89c155..4863551 100644
--- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
+++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ThriftHiveMetastore.java
@@ -380,6 +380,12 @@ import org.slf4j.LoggerFactory;
 
     public AllocateTableWriteIdsResponse allocate_table_write_ids(AllocateTableWriteIdsRequest rqst) throws NoSuchTxnException, TxnAbortedException, MetaException, org.apache.thrift.TException;
 
+    public MaxAllocatedTableWriteIdResponse get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst) throws MetaException, org.apache.thrift.TException;
+
+    public void seed_write_id(SeedTableWriteIdsRequest rqst) throws MetaException, org.apache.thrift.TException;
+
+    public void seed_txn_id(SeedTxnIdRequest rqst) throws MetaException, org.apache.thrift.TException;
+
     public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException;
 
     public LockResponse check_lock(CheckLockRequest rqst) throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, org.apache.thrift.TException;
@@ -866,6 +872,12 @@ import org.slf4j.LoggerFactory;
 
     public void allocate_table_write_ids(AllocateTableWriteIdsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void seed_write_id(SeedTableWriteIdsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
+    public void seed_txn_id(SeedTxnIdRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
     public void check_lock(CheckLockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
@@ -5980,6 +5992,78 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "allocate_table_write_ids failed: unknown result");
     }
 
+    public MaxAllocatedTableWriteIdResponse get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst) throws MetaException, org.apache.thrift.TException
+    {
+      send_get_max_allocated_table_write_id(rqst);
+      return recv_get_max_allocated_table_write_id();
+    }
+
+    public void send_get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst) throws org.apache.thrift.TException
+    {
+      get_max_allocated_table_write_id_args args = new get_max_allocated_table_write_id_args();
+      args.setRqst(rqst);
+      sendBase("get_max_allocated_table_write_id", args);
+    }
+
+    public MaxAllocatedTableWriteIdResponse recv_get_max_allocated_table_write_id() throws MetaException, org.apache.thrift.TException
+    {
+      get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
+      receiveBase(result, "get_max_allocated_table_write_id");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "get_max_allocated_table_write_id failed: unknown result");
+    }
+
+    public void seed_write_id(SeedTableWriteIdsRequest rqst) throws MetaException, org.apache.thrift.TException
+    {
+      send_seed_write_id(rqst);
+      recv_seed_write_id();
+    }
+
+    public void send_seed_write_id(SeedTableWriteIdsRequest rqst) throws org.apache.thrift.TException
+    {
+      seed_write_id_args args = new seed_write_id_args();
+      args.setRqst(rqst);
+      sendBase("seed_write_id", args);
+    }
+
+    public void recv_seed_write_id() throws MetaException, org.apache.thrift.TException
+    {
+      seed_write_id_result result = new seed_write_id_result();
+      receiveBase(result, "seed_write_id");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      return;
+    }
+
+    public void seed_txn_id(SeedTxnIdRequest rqst) throws MetaException, org.apache.thrift.TException
+    {
+      send_seed_txn_id(rqst);
+      recv_seed_txn_id();
+    }
+
+    public void send_seed_txn_id(SeedTxnIdRequest rqst) throws org.apache.thrift.TException
+    {
+      seed_txn_id_args args = new seed_txn_id_args();
+      args.setRqst(rqst);
+      sendBase("seed_txn_id", args);
+    }
+
+    public void recv_seed_txn_id() throws MetaException, org.apache.thrift.TException
+    {
+      seed_txn_id_result result = new seed_txn_id_result();
+      receiveBase(result, "seed_txn_id");
+      if (result.o1 != null) {
+        throw result.o1;
+      }
+      return;
+    }
+
     public LockResponse lock(LockRequest rqst) throws NoSuchTxnException, TxnAbortedException, org.apache.thrift.TException
     {
       send_lock(rqst);
@@ -13827,6 +13911,102 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void get_max_allocated_table_write_id(MaxAllocatedTableWriteIdRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      get_max_allocated_table_write_id_call method_call = new get_max_allocated_table_write_id_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 get_max_allocated_table_write_id_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private MaxAllocatedTableWriteIdRequest rqst;
+      public get_max_allocated_table_write_id_call(MaxAllocatedTableWriteIdRequest 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("get_max_allocated_table_write_id", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        get_max_allocated_table_write_id_args args = new get_max_allocated_table_write_id_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public MaxAllocatedTableWriteIdResponse getResult() throws MetaException, 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_get_max_allocated_table_write_id();
+      }
+    }
+
+    public void seed_write_id(SeedTableWriteIdsRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      seed_write_id_call method_call = new seed_write_id_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 seed_write_id_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SeedTableWriteIdsRequest rqst;
+      public seed_write_id_call(SeedTableWriteIdsRequest 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("seed_write_id", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        seed_write_id_args args = new seed_write_id_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws MetaException, 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);
+        (new Client(prot)).recv_seed_write_id();
+      }
+    }
+
+    public void seed_txn_id(SeedTxnIdRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      seed_txn_id_call method_call = new seed_txn_id_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 seed_txn_id_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private SeedTxnIdRequest rqst;
+      public seed_txn_id_call(SeedTxnIdRequest 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("seed_txn_id", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        seed_txn_id_args args = new seed_txn_id_args();
+        args.setRqst(rqst);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws MetaException, 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);
+        (new Client(prot)).recv_seed_txn_id();
+      }
+    }
+
     public void lock(LockRequest rqst, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       lock_call method_call = new lock_call(rqst, resultHandler, this, ___protocolFactory, ___transport);
@@ -16322,6 +16502,9 @@ import org.slf4j.LoggerFactory;
       processMap.put("repl_tbl_writeid_state", new repl_tbl_writeid_state());
       processMap.put("get_valid_write_ids", new get_valid_write_ids());
       processMap.put("allocate_table_write_ids", new allocate_table_write_ids());
+      processMap.put("get_max_allocated_table_write_id", new get_max_allocated_table_write_id());
+      processMap.put("seed_write_id", new seed_write_id());
+      processMap.put("seed_txn_id", new seed_txn_id());
       processMap.put("lock", new lock());
       processMap.put("check_lock", new check_lock());
       processMap.put("unlock", new unlock());
@@ -20798,6 +20981,78 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_max_allocated_table_write_id<I extends Iface> extends org.apache.thrift.ProcessFunction<I, get_max_allocated_table_write_id_args> {
+      public get_max_allocated_table_write_id() {
+        super("get_max_allocated_table_write_id");
+      }
+
+      public get_max_allocated_table_write_id_args getEmptyArgsInstance() {
+        return new get_max_allocated_table_write_id_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public get_max_allocated_table_write_id_result getResult(I iface, get_max_allocated_table_write_id_args args) throws org.apache.thrift.TException {
+        get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
+        try {
+          result.success = iface.get_max_allocated_table_write_id(args.rqst);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_write_id<I extends Iface> extends org.apache.thrift.ProcessFunction<I, seed_write_id_args> {
+      public seed_write_id() {
+        super("seed_write_id");
+      }
+
+      public seed_write_id_args getEmptyArgsInstance() {
+        return new seed_write_id_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public seed_write_id_result getResult(I iface, seed_write_id_args args) throws org.apache.thrift.TException {
+        seed_write_id_result result = new seed_write_id_result();
+        try {
+          iface.seed_write_id(args.rqst);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_txn_id<I extends Iface> extends org.apache.thrift.ProcessFunction<I, seed_txn_id_args> {
+      public seed_txn_id() {
+        super("seed_txn_id");
+      }
+
+      public seed_txn_id_args getEmptyArgsInstance() {
+        return new seed_txn_id_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public seed_txn_id_result getResult(I iface, seed_txn_id_args args) throws org.apache.thrift.TException {
+        seed_txn_id_result result = new seed_txn_id_result();
+        try {
+          iface.seed_txn_id(args.rqst);
+        } catch (MetaException o1) {
+          result.o1 = o1;
+        }
+        return result;
+      }
+    }
+
     @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class lock<I extends Iface> extends org.apache.thrift.ProcessFunction<I, lock_args> {
       public lock() {
         super("lock");
@@ -22747,6 +23002,9 @@ import org.slf4j.LoggerFactory;
       processMap.put("repl_tbl_writeid_state", new repl_tbl_writeid_state());
       processMap.put("get_valid_write_ids", new get_valid_write_ids());
       processMap.put("allocate_table_write_ids", new allocate_table_write_ids());
+      processMap.put("get_max_allocated_table_write_id", new get_max_allocated_table_write_id());
+      processMap.put("seed_write_id", new seed_write_id());
+      processMap.put("seed_txn_id", new seed_txn_id());
       processMap.put("lock", new lock());
       processMap.put("check_lock", new check_lock());
       processMap.put("unlock", new unlock());
@@ -33281,20 +33539,20 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, lock_args, LockResponse> {
-      public lock() {
-        super("lock");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_max_allocated_table_write_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, get_max_allocated_table_write_id_args, MaxAllocatedTableWriteIdResponse> {
+      public get_max_allocated_table_write_id() {
+        super("get_max_allocated_table_write_id");
       }
 
-      public lock_args getEmptyArgsInstance() {
-        return new lock_args();
+      public get_max_allocated_table_write_id_args getEmptyArgsInstance() {
+        return new get_max_allocated_table_write_id_args();
       }
 
-      public AsyncMethodCallback<LockResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<LockResponse>() { 
-          public void onComplete(LockResponse o) {
-            lock_result result = new lock_result();
+        return new AsyncMethodCallback<MaxAllocatedTableWriteIdResponse>() { 
+          public void onComplete(MaxAllocatedTableWriteIdResponse o) {
+            get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
             result.success = o;
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
@@ -33307,17 +33565,12 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            lock_result result = new lock_result();
-            if (e instanceof NoSuchTxnException) {
-                        result.o1 = (NoSuchTxnException) e;
+            get_max_allocated_table_write_id_result result = new get_max_allocated_table_write_id_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof TxnAbortedException) {
-                        result.o2 = (TxnAbortedException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
-            }
              else 
             {
               msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION;
@@ -33338,26 +33591,25 @@ import org.slf4j.LoggerFactory;
         return false;
       }
 
-      public void start(I iface, lock_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws TException {
-        iface.lock(args.rqst,resultHandler);
+      public void start(I iface, get_max_allocated_table_write_id_args args, org.apache.thrift.async.AsyncMethodCallback<MaxAllocatedTableWriteIdResponse> resultHandler) throws TException {
+        iface.get_max_allocated_table_write_id(args.rqst,resultHandler);
       }
     }
 
-    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class check_lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, check_lock_args, LockResponse> {
-      public check_lock() {
-        super("check_lock");
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_write_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, seed_write_id_args, Void> {
+      public seed_write_id() {
+        super("seed_write_id");
       }
 
-      public check_lock_args getEmptyArgsInstance() {
-        return new check_lock_args();
+      public seed_write_id_args getEmptyArgsInstance() {
+        return new seed_write_id_args();
       }
 
-      public AsyncMethodCallback<LockResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
         final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<LockResponse>() { 
-          public void onComplete(LockResponse o) {
-            check_lock_result result = new check_lock_result();
-            result.success = o;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            seed_write_id_result result = new seed_write_id_result();
             try {
               fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid);
               return;
@@ -33369,20 +33621,195 @@ import org.slf4j.LoggerFactory;
           public void onError(Exception e) {
             byte msgType = org.apache.thrift.protocol.TMessageType.REPLY;
             org.apache.thrift.TBase msg;
-            check_lock_result result = new check_lock_result();
-            if (e instanceof NoSuchTxnException) {
-                        result.o1 = (NoSuchTxnException) e;
+            seed_write_id_result result = new seed_write_id_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
                         result.setO1IsSet(true);
                         msg = result;
             }
-            else             if (e instanceof TxnAbortedException) {
-                        result.o2 = (TxnAbortedException) e;
-                        result.setO2IsSet(true);
-                        msg = result;
+             else 
+            {
+              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());
             }
-            else             if (e instanceof NoSuchLockException) {
-                        result.o3 = (NoSuchLockException) e;
-                        result.setO3IsSet(true);
+            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, seed_write_id_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.seed_write_id(args.rqst,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class seed_txn_id<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, seed_txn_id_args, Void> {
+      public seed_txn_id() {
+        super("seed_txn_id");
+      }
+
+      public seed_txn_id_args getEmptyArgsInstance() {
+        return new seed_txn_id_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+            seed_txn_id_result result = new seed_txn_id_result();
+            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;
+            seed_txn_id_result result = new seed_txn_id_result();
+            if (e instanceof MetaException) {
+                        result.o1 = (MetaException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              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, seed_txn_id_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.seed_txn_id(args.rqst,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, lock_args, LockResponse> {
+      public lock() {
+        super("lock");
+      }
+
+      public lock_args getEmptyArgsInstance() {
+        return new lock_args();
+      }
+
+      public AsyncMethodCallback<LockResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<LockResponse>() { 
+          public void onComplete(LockResponse o) {
+            lock_result result = new lock_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;
+            lock_result result = new lock_result();
+            if (e instanceof NoSuchTxnException) {
+                        result.o1 = (NoSuchTxnException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof TxnAbortedException) {
+                        result.o2 = (TxnAbortedException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+             else 
+            {
+              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, lock_args args, org.apache.thrift.async.AsyncMethodCallback<LockResponse> resultHandler) throws TException {
+        iface.lock(args.rqst,resultHandler);
+      }
+    }
+
+    @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class check_lock<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, check_lock_args, LockResponse> {
+      public check_lock() {
+        super("check_lock");
+      }
+
+      public check_lock_args getEmptyArgsInstance() {
+        return new check_lock_args();
+      }
+
+      public AsyncMethodCallback<LockResponse> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<LockResponse>() { 
+          public void onComplete(LockResponse o) {
+            check_lock_result result = new check_lock_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;
+            check_lock_result result = new check_lock_result();
+            if (e instanceof NoSuchTxnException) {
+                        result.o1 = (NoSuchTxnException) e;
+                        result.setO1IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof TxnAbortedException) {
+                        result.o2 = (TxnAbortedException) e;
+                        result.setO2IsSet(true);
+                        msg = result;
+            }
+            else             if (e instanceof NoSuchLockException) {
+                        result.o3 = (NoSuchLockException) e;
+                        result.setO3IsSet(true);
                         msg = result;
             }
              else 
@@ -204208,11 +204635,1879 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("remove_master_key_result(");
+      StringBuilder sb = new StringBuilder("remove_master_key_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      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 {
+        // 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 remove_master_key_resultStandardSchemeFactory implements SchemeFactory {
+      public remove_master_key_resultStandardScheme getScheme() {
+        return new remove_master_key_resultStandardScheme();
+      }
+    }
+
+    private static class remove_master_key_resultStandardScheme extends StandardScheme<remove_master_key_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, remove_master_key_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+                struct.success = iprot.readBool();
+                struct.setSuccessIsSet(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, remove_master_key_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.isSetSuccess()) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          oprot.writeBool(struct.success);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class remove_master_key_resultTupleSchemeFactory implements SchemeFactory {
+      public remove_master_key_resultTupleScheme getScheme() {
+        return new remove_master_key_resultTupleScheme();
+      }
+    }
+
+    private static class remove_master_key_resultTupleScheme extends TupleScheme<remove_master_key_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, remove_master_key_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          oprot.writeBool(struct.success);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, remove_master_key_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = iprot.readBool();
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_master_keys_args implements org.apache.thrift.TBase<get_master_keys_args, get_master_keys_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_master_keys_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_master_keys_args");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_master_keys_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_master_keys_argsTupleSchemeFactory());
+    }
+
+
+    /** 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(get_master_keys_args.class, metaDataMap);
+    }
+
+    public get_master_keys_args() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_master_keys_args(get_master_keys_args other) {
+    }
+
+    public get_master_keys_args deepCopy() {
+      return new get_master_keys_args(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 get_master_keys_args)
+        return this.equals((get_master_keys_args)that);
+      return false;
+    }
+
+    public boolean equals(get_master_keys_args 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(get_master_keys_args 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("get_master_keys_args(");
+      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 get_master_keys_argsStandardSchemeFactory implements SchemeFactory {
+      public get_master_keys_argsStandardScheme getScheme() {
+        return new get_master_keys_argsStandardScheme();
+      }
+    }
+
+    private static class get_master_keys_argsStandardScheme extends StandardScheme<get_master_keys_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_master_keys_args 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, get_master_keys_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_master_keys_argsTupleSchemeFactory implements SchemeFactory {
+      public get_master_keys_argsTupleScheme getScheme() {
+        return new get_master_keys_argsTupleScheme();
+      }
+    }
+
+    private static class get_master_keys_argsTupleScheme extends TupleScheme<get_master_keys_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_master_keys_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_master_keys_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_master_keys_result implements org.apache.thrift.TBase<get_master_keys_result, get_master_keys_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_master_keys_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_master_keys_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_master_keys_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_master_keys_resultTupleSchemeFactory());
+    }
+
+    private List<String> success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          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
+    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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          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(get_master_keys_result.class, metaDataMap);
+    }
+
+    public get_master_keys_result() {
+    }
+
+    public get_master_keys_result(
+      List<String> success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_master_keys_result(get_master_keys_result other) {
+      if (other.isSetSuccess()) {
+        List<String> __this__success = new ArrayList<String>(other.success);
+        this.success = __this__success;
+      }
+    }
+
+    public get_master_keys_result deepCopy() {
+      return new get_master_keys_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public int getSuccessSize() {
+      return (this.success == null) ? 0 : this.success.size();
+    }
+
+    public java.util.Iterator<String> getSuccessIterator() {
+      return (this.success == null) ? null : this.success.iterator();
+    }
+
+    public void addToSuccess(String elem) {
+      if (this.success == null) {
+        this.success = new ArrayList<String>();
+      }
+      this.success.add(elem);
+    }
+
+    public List<String> getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(List<String> success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((List<String>)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      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 SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_master_keys_result)
+        return this.equals((get_master_keys_result)that);
+      return false;
+    }
+
+    public boolean equals(get_master_keys_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_master_keys_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        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("get_master_keys_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      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 get_master_keys_resultStandardSchemeFactory implements SchemeFactory {
+      public get_master_keys_resultStandardScheme getScheme() {
+        return new get_master_keys_resultStandardScheme();
+      }
+    }
+
+    private static class get_master_keys_resultStandardScheme extends StandardScheme<get_master_keys_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_master_keys_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list1860 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list1860.size);
+                  String _elem1861;
+                  for (int _i1862 = 0; _i1862 < _list1860.size; ++_i1862)
+                  {
+                    _elem1861 = iprot.readString();
+                    struct.success.add(_elem1861);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setSuccessIsSet(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, get_master_keys_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
+            for (String _iter1863 : struct.success)
+            {
+              oprot.writeString(_iter1863);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_master_keys_resultTupleSchemeFactory implements SchemeFactory {
+      public get_master_keys_resultTupleScheme getScheme() {
+        return new get_master_keys_resultTupleScheme();
+      }
+    }
+
+    private static class get_master_keys_resultTupleScheme extends TupleScheme<get_master_keys_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_master_keys_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          {
+            oprot.writeI32(struct.success.size());
+            for (String _iter1864 : struct.success)
+            {
+              oprot.writeString(_iter1864);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_master_keys_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          {
+            org.apache.thrift.protocol.TList _list1865 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list1865.size);
+            String _elem1866;
+            for (int _i1867 = 0; _i1867 < _list1865.size; ++_i1867)
+            {
+              _elem1866 = iprot.readString();
+              struct.success.add(_elem1866);
+            }
+          }
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_args implements org.apache.thrift.TBase<get_open_txns_args, get_open_txns_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_args");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_open_txns_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_open_txns_argsTupleSchemeFactory());
+    }
+
+
+    /** 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(get_open_txns_args.class, metaDataMap);
+    }
+
+    public get_open_txns_args() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_open_txns_args(get_open_txns_args other) {
+    }
+
+    public get_open_txns_args deepCopy() {
+      return new get_open_txns_args(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 get_open_txns_args)
+        return this.equals((get_open_txns_args)that);
+      return false;
+    }
+
+    public boolean equals(get_open_txns_args 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(get_open_txns_args 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("get_open_txns_args(");
+      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 get_open_txns_argsStandardSchemeFactory implements SchemeFactory {
+      public get_open_txns_argsStandardScheme getScheme() {
+        return new get_open_txns_argsStandardScheme();
+      }
+    }
+
+    private static class get_open_txns_argsStandardScheme extends StandardScheme<get_open_txns_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_args 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, get_open_txns_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_open_txns_argsTupleSchemeFactory implements SchemeFactory {
+      public get_open_txns_argsTupleScheme getScheme() {
+        return new get_open_txns_argsTupleScheme();
+      }
+    }
+
+    private static class get_open_txns_argsTupleScheme extends TupleScheme<get_open_txns_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_result implements org.apache.thrift.TBase<get_open_txns_result, get_open_txns_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_open_txns_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_open_txns_resultTupleSchemeFactory());
+    }
+
+    private GetOpenTxnsResponse success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          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
+    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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_result.class, metaDataMap);
+    }
+
+    public get_open_txns_result() {
+    }
+
+    public get_open_txns_result(
+      GetOpenTxnsResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_open_txns_result(get_open_txns_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new GetOpenTxnsResponse(other.success);
+      }
+    }
+
+    public get_open_txns_result deepCopy() {
+      return new get_open_txns_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public GetOpenTxnsResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(GetOpenTxnsResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((GetOpenTxnsResponse)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      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 SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_open_txns_result)
+        return this.equals((get_open_txns_result)that);
+      return false;
+    }
+
+    public boolean equals(get_open_txns_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_open_txns_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        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("get_open_txns_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.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 {
+        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 get_open_txns_resultStandardSchemeFactory implements SchemeFactory {
+      public get_open_txns_resultStandardScheme getScheme() {
+        return new get_open_txns_resultStandardScheme();
+      }
+    }
+
+    private static class get_open_txns_resultStandardScheme extends StandardScheme<get_open_txns_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new GetOpenTxnsResponse();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(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, get_open_txns_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_open_txns_resultTupleSchemeFactory implements SchemeFactory {
+      public get_open_txns_resultTupleScheme getScheme() {
+        return new get_open_txns_resultTupleScheme();
+      }
+    }
+
+    private static class get_open_txns_resultTupleScheme extends TupleScheme<get_open_txns_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new GetOpenTxnsResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_info_args implements org.apache.thrift.TBase<get_open_txns_info_args, get_open_txns_info_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_info_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_info_args");
+
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_open_txns_info_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_open_txns_info_argsTupleSchemeFactory());
+    }
+
+
+    /** 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(get_open_txns_info_args.class, metaDataMap);
+    }
+
+    public get_open_txns_info_args() {
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_open_txns_info_args(get_open_txns_info_args other) {
+    }
+
+    public get_open_txns_info_args deepCopy() {
+      return new get_open_txns_info_args(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 get_open_txns_info_args)
+        return this.equals((get_open_txns_info_args)that);
+      return false;
+    }
+
+    public boolean equals(get_open_txns_info_args 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(get_open_txns_info_args 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("get_open_txns_info_args(");
+      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 get_open_txns_info_argsStandardSchemeFactory implements SchemeFactory {
+      public get_open_txns_info_argsStandardScheme getScheme() {
+        return new get_open_txns_info_argsStandardScheme();
+      }
+    }
+
+    private static class get_open_txns_info_argsStandardScheme extends StandardScheme<get_open_txns_info_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_info_args 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, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class get_open_txns_info_argsTupleSchemeFactory implements SchemeFactory {
+      public get_open_txns_info_argsTupleScheme getScheme() {
+        return new get_open_txns_info_argsTupleScheme();
+      }
+    }
+
+    private static class get_open_txns_info_argsTupleScheme extends TupleScheme<get_open_txns_info_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+      }
+    }
+
+  }
+
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_info_result implements org.apache.thrift.TBase<get_open_txns_info_result, get_open_txns_info_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_info_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_info_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new get_open_txns_info_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_open_txns_info_resultTupleSchemeFactory());
+    }
+
+    private GetOpenTxnsInfoResponse success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          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
+    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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsInfoResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_info_result.class, metaDataMap);
+    }
+
+    public get_open_txns_info_result() {
+    }
+
+    public get_open_txns_info_result(
+      GetOpenTxnsInfoResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public get_open_txns_info_result(get_open_txns_info_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new GetOpenTxnsInfoResponse(other.success);
+      }
+    }
+
+    public get_open_txns_info_result deepCopy() {
+      return new get_open_txns_info_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public GetOpenTxnsInfoResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(GetOpenTxnsInfoResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((GetOpenTxnsInfoResponse)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      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 SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof get_open_txns_info_result)
+        return this.equals((get_open_txns_info_result)that);
+      return false;
+    }
+
+    public boolean equals(get_open_txns_info_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      List<Object> list = new ArrayList<Object>();
+
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
+      return list.hashCode();
+    }
+
+    @Override
+    public int compareTo(get_open_txns_info_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        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("get_open_txns_info_result(");
       boolean first = true;
 
       sb.append("success:");
-      sb.append(this.success);
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
       first = false;
       sb.append(")");
       return sb.toString();
@@ -204221,6 +206516,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -204233,23 +206531,21 @@ import org.slf4j.LoggerFactory;
 
     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 remove_master_key_resultStandardSchemeFactory implements SchemeFactory {
-      public remove_master_key_resultStandardScheme getScheme() {
-        return new remove_master_key_resultStandardScheme();
+    private static class get_open_txns_info_resultStandardSchemeFactory implements SchemeFactory {
+      public get_open_txns_info_resultStandardScheme getScheme() {
+        return new get_open_txns_info_resultStandardScheme();
       }
     }
 
-    private static class remove_master_key_resultStandardScheme extends StandardScheme<remove_master_key_result> {
+    private static class get_open_txns_info_resultStandardScheme extends StandardScheme<get_open_txns_info_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, remove_master_key_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -204260,8 +206556,9 @@ import org.slf4j.LoggerFactory;
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-                struct.success = iprot.readBool();
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new GetOpenTxnsInfoResponse();
+                struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -204276,13 +206573,13 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, remove_master_key_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.isSetSuccess()) {
+        if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          oprot.writeBool(struct.success);
+          struct.success.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -204291,16 +206588,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class remove_master_key_resultTupleSchemeFactory implements SchemeFactory {
-      public remove_master_key_resultTupleScheme getScheme() {
-        return new remove_master_key_resultTupleScheme();
+    private static class get_open_txns_info_resultTupleSchemeFactory implements SchemeFactory {
+      public get_open_txns_info_resultTupleScheme getScheme() {
+        return new get_open_txns_info_resultTupleScheme();
       }
     }
 
-    private static class remove_master_key_resultTupleScheme extends TupleScheme<remove_master_key_result> {
+    private static class get_open_txns_info_resultTupleScheme extends TupleScheme<get_open_txns_info_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, remove_master_key_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -204308,16 +206605,17 @@ import org.slf4j.LoggerFactory;
         }
         oprot.writeBitSet(optionals, 1);
         if (struct.isSetSuccess()) {
-          oprot.writeBool(struct.success);
+          struct.success.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, remove_master_key_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = iprot.readBool();
+          struct.success = new GetOpenTxnsInfoResponse();
+          struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
       }
@@ -204325,20 +206623,22 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_master_keys_args implements org.apache.thrift.TBase<get_master_keys_args, get_master_keys_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_master_keys_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_master_keys_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class open_txns_args implements org.apache.thrift.TBase<open_txns_args, open_txns_args._Fields>, java.io.Serializable, Cloneable, Comparable<open_txns_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("open_txns_args");
 
+    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_master_keys_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_master_keys_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new open_txns_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new open_txns_argsTupleSchemeFactory());
     }
 
+    private OpenTxnRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
+      RQST((short)1, "rqst");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -204353,6 +206653,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // RQST
+            return RQST;
           default:
             return null;
         }
@@ -204391,37 +206693,86 @@ import org.slf4j.LoggerFactory;
         return _fieldName;
       }
     }
+
+    // isset id assignments
     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.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OpenTxnRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_master_keys_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(open_txns_args.class, metaDataMap);
     }
 
-    public get_master_keys_args() {
+    public open_txns_args() {
+    }
+
+    public open_txns_args(
+      OpenTxnRequest rqst)
+    {
+      this();
+      this.rqst = rqst;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_master_keys_args(get_master_keys_args other) {
+    public open_txns_args(open_txns_args other) {
+      if (other.isSetRqst()) {
+        this.rqst = new OpenTxnRequest(other.rqst);
+      }
     }
 
-    public get_master_keys_args deepCopy() {
-      return new get_master_keys_args(this);
+    public open_txns_args deepCopy() {
+      return new open_txns_args(this);
     }
 
     @Override
     public void clear() {
+      this.rqst = null;
+    }
+
+    public OpenTxnRequest getRqst() {
+      return this.rqst;
+    }
+
+    public void setRqst(OpenTxnRequest rqst) {
+      this.rqst = rqst;
+    }
+
+    public void unsetRqst() {
+      this.rqst = null;
+    }
+
+    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
+    public boolean isSetRqst() {
+      return this.rqst != null;
+    }
+
+    public void setRqstIsSet(boolean value) {
+      if (!value) {
+        this.rqst = null;
+      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case RQST:
+        if (value == null) {
+          unsetRqst();
+        } else {
+          setRqst((OpenTxnRequest)value);
+        }
+        break;
+
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case RQST:
+        return getRqst();
+
       }
       throw new IllegalStateException();
     }
@@ -204433,6 +206784,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
+      case RQST:
+        return isSetRqst();
       }
       throw new IllegalStateException();
     }
@@ -204441,15 +206794,24 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_master_keys_args)
-        return this.equals((get_master_keys_args)that);
+      if (that instanceof open_txns_args)
+        return this.equals((open_txns_args)that);
       return false;
     }
 
-    public boolean equals(get_master_keys_args that) {
+    public boolean equals(open_txns_args that) {
       if (that == null)
         return false;
 
+      boolean this_present_rqst = true && this.isSetRqst();
+      boolean that_present_rqst = true && that.isSetRqst();
+      if (this_present_rqst || that_present_rqst) {
+        if (!(this_present_rqst && that_present_rqst))
+          return false;
+        if (!this.rqst.equals(that.rqst))
+          return false;
+      }
+
       return true;
     }
 
@@ -204457,17 +206819,32 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_rqst = true && (isSetRqst());
+      list.add(present_rqst);
+      if (present_rqst)
+        list.add(rqst);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_master_keys_args other) {
+    public int compareTo(open_txns_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRqst()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -204485,9 +206862,16 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_master_keys_args(");
+      StringBuilder sb = new StringBuilder("open_txns_args(");
       boolean first = true;
 
+      sb.append("rqst:");
+      if (this.rqst == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.rqst);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -204495,6 +206879,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (rqst != null) {
+        rqst.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -204513,15 +206900,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_master_keys_argsStandardSchemeFactory implements SchemeFactory {
-      public get_master_keys_argsStandardScheme getScheme() {
-        return new get_master_keys_argsStandardScheme();
+    private static class open_txns_argsStandardSchemeFactory implements SchemeFactory {
+      public open_txns_argsStandardScheme getScheme() {
+        return new open_txns_argsStandardScheme();
       }
     }
 
-    private static class get_master_keys_argsStandardScheme extends StandardScheme<get_master_keys_args> {
+    private static class open_txns_argsStandardScheme extends StandardScheme<open_txns_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_master_keys_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, open_txns_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -204531,6 +206918,15 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
+            case 1: // RQST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.rqst = new OpenTxnRequest();
+                struct.rqst.read(iprot);
+                struct.setRqstIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -204540,49 +206936,68 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_master_keys_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, open_txns_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.rqst != null) {
+          oprot.writeFieldBegin(RQST_FIELD_DESC);
+          struct.rqst.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class get_master_keys_argsTupleSchemeFactory implements SchemeFactory {
-      public get_master_keys_argsTupleScheme getScheme() {
-        return new get_master_keys_argsTupleScheme();
+    private static class open_txns_argsTupleSchemeFactory implements SchemeFactory {
+      public open_txns_argsTupleScheme getScheme() {
+        return new open_txns_argsTupleScheme();
       }
     }
 
-    private static class get_master_keys_argsTupleScheme extends TupleScheme<get_master_keys_args> {
+    private static class open_txns_argsTupleScheme extends TupleScheme<open_txns_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_master_keys_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, open_txns_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRqst()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRqst()) {
+          struct.rqst.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_master_keys_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, open_txns_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.rqst = new OpenTxnRequest();
+          struct.rqst.read(iprot);
+          struct.setRqstIsSet(true);
+        }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_master_keys_result implements org.apache.thrift.TBase<get_master_keys_result, get_master_keys_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_master_keys_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_master_keys_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class open_txns_result implements org.apache.thrift.TBase<open_txns_result, open_txns_result._Fields>, java.io.Serializable, Cloneable, Comparable<open_txns_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("open_txns_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.LIST, (short)0);
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_master_keys_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_master_keys_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new open_txns_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new open_txns_resultTupleSchemeFactory());
     }
 
-    private List<String> success; // required
+    private OpenTxnsResponse success; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -204647,17 +207062,16 @@ import org.slf4j.LoggerFactory;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          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))));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OpenTxnsResponse.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_master_keys_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(open_txns_result.class, metaDataMap);
     }
 
-    public get_master_keys_result() {
+    public open_txns_result() {
     }
 
-    public get_master_keys_result(
-      List<String> success)
+    public open_txns_result(
+      OpenTxnsResponse success)
     {
       this();
       this.success = success;
@@ -204666,15 +207080,14 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_master_keys_result(get_master_keys_result other) {
+    public open_txns_result(open_txns_result other) {
       if (other.isSetSuccess()) {
-        List<String> __this__success = new ArrayList<String>(other.success);
-        this.success = __this__success;
+        this.success = new OpenTxnsResponse(other.success);
       }
     }
 
-    public get_master_keys_result deepCopy() {
-      return new get_master_keys_result(this);
+    public open_txns_result deepCopy() {
+      return new open_txns_result(this);
     }
 
     @Override
@@ -204682,26 +207095,11 @@ import org.slf4j.LoggerFactory;
       this.success = null;
     }
 
-    public int getSuccessSize() {
-      return (this.success == null) ? 0 : this.success.size();
-    }
-
-    public java.util.Iterator<String> getSuccessIterator() {
-      return (this.success == null) ? null : this.success.iterator();
-    }
-
-    public void addToSuccess(String elem) {
-      if (this.success == null) {
-        this.success = new ArrayList<String>();
-      }
-      this.success.add(elem);
-    }
-
-    public List<String> getSuccess() {
+    public OpenTxnsResponse getSuccess() {
       return this.success;
     }
 
-    public void setSuccess(List<String> success) {
+    public void setSuccess(OpenTxnsResponse success) {
       this.success = success;
     }
 
@@ -204726,7 +207124,7 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetSuccess();
         } else {
-          setSuccess((List<String>)value);
+          setSuccess((OpenTxnsResponse)value);
         }
         break;
 
@@ -204759,12 +207157,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_master_keys_result)
-        return this.equals((get_master_keys_result)that);
+      if (that instanceof open_txns_result)
+        return this.equals((open_txns_result)that);
       return false;
     }
 
-    public boolean equals(get_master_keys_result that) {
+    public boolean equals(open_txns_result that) {
       if (that == null)
         return false;
 
@@ -204793,7 +207191,7 @@ import org.slf4j.LoggerFactory;
     }
 
     @Override
-    public int compareTo(get_master_keys_result other) {
+    public int compareTo(open_txns_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -204827,7 +207225,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_master_keys_result(");
+      StringBuilder sb = new StringBuilder("open_txns_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -204844,6 +207242,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -204862,15 +207263,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_master_keys_resultStandardSchemeFactory implements SchemeFactory {
-      public get_master_keys_resultStandardScheme getScheme() {
-        return new get_master_keys_resultStandardScheme();
+    private static class open_txns_resultStandardSchemeFactory implements SchemeFactory {
+      public open_txns_resultStandardScheme getScheme() {
+        return new open_txns_resultStandardScheme();
       }
     }
 
-    private static class get_master_keys_resultStandardScheme extends StandardScheme<get_master_keys_result> {
+    private static class open_txns_resultStandardScheme extends StandardScheme<open_txns_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_master_keys_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, open_txns_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -204881,18 +207282,9 @@ import org.slf4j.LoggerFactory;
           }
           switch (schemeField.id) {
             case 0: // SUCCESS
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list1860 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list1860.size);
-                  String _elem1861;
-                  for (int _i1862 = 0; _i1862 < _list1860.size; ++_i1862)
-                  {
-                    _elem1861 = iprot.readString();
-                    struct.success.add(_elem1861);
-                  }
-                  iprot.readListEnd();
-                }
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new OpenTxnsResponse();
+                struct.success.read(iprot);
                 struct.setSuccessIsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
@@ -204907,20 +207299,13 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_master_keys_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, open_txns_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
         if (struct.success != null) {
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          {
-            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter1863 : struct.success)
-            {
-              oprot.writeString(_iter1863);
-            }
-            oprot.writeListEnd();
-          }
+          struct.success.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -204929,16 +207314,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class get_master_keys_resultTupleSchemeFactory implements SchemeFactory {
-      public get_master_keys_resultTupleScheme getScheme() {
-        return new get_master_keys_resultTupleScheme();
+    private static class open_txns_resultTupleSchemeFactory implements SchemeFactory {
+      public open_txns_resultTupleScheme getScheme() {
+        return new open_txns_resultTupleScheme();
       }
     }
 
-    private static class get_master_keys_resultTupleScheme extends TupleScheme<get_master_keys_result> {
+    private static class open_txns_resultTupleScheme extends TupleScheme<open_txns_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_master_keys_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, open_txns_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetSuccess()) {
@@ -204946,31 +207331,17 @@ import org.slf4j.LoggerFactory;
         }
         oprot.writeBitSet(optionals, 1);
         if (struct.isSetSuccess()) {
-          {
-            oprot.writeI32(struct.success.size());
-            for (String _iter1864 : struct.success)
-            {
-              oprot.writeString(_iter1864);
-            }
-          }
+          struct.success.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_master_keys_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, open_txns_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          {
-            org.apache.thrift.protocol.TList _list1865 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list1865.size);
-            String _elem1866;
-            for (int _i1867 = 0; _i1867 < _list1865.size; ++_i1867)
-            {
-              _elem1866 = iprot.readString();
-              struct.success.add(_elem1866);
-            }
-          }
+          struct.success = new OpenTxnsResponse();
+          struct.success.read(iprot);
           struct.setSuccessIsSet(true);
         }
       }
@@ -204978,20 +207349,22 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_args implements org.apache.thrift.TBase<get_open_txns_args, get_open_txns_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txn_args implements org.apache.thrift.TBase<abort_txn_args, abort_txn_args._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txn_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txn_args");
 
+    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_open_txns_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_open_txns_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new abort_txn_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new abort_txn_argsTupleSchemeFactory());
     }
 
+    private AbortTxnRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
+      RQST((short)1, "rqst");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -205006,6 +207379,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // RQST
+            return RQST;
           default:
             return null;
         }
@@ -205044,37 +207419,86 @@ import org.slf4j.LoggerFactory;
         return _fieldName;
       }
     }
+
+    // isset id assignments
     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.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AbortTxnRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txn_args.class, metaDataMap);
     }
 
-    public get_open_txns_args() {
+    public abort_txn_args() {
+    }
+
+    public abort_txn_args(
+      AbortTxnRequest rqst)
+    {
+      this();
+      this.rqst = rqst;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_open_txns_args(get_open_txns_args other) {
+    public abort_txn_args(abort_txn_args other) {
+      if (other.isSetRqst()) {
+        this.rqst = new AbortTxnRequest(other.rqst);
+      }
     }
 
-    public get_open_txns_args deepCopy() {
-      return new get_open_txns_args(this);
+    public abort_txn_args deepCopy() {
+      return new abort_txn_args(this);
     }
 
     @Override
     public void clear() {
+      this.rqst = null;
+    }
+
+    public AbortTxnRequest getRqst() {
+      return this.rqst;
+    }
+
+    public void setRqst(AbortTxnRequest rqst) {
+      this.rqst = rqst;
+    }
+
+    public void unsetRqst() {
+      this.rqst = null;
+    }
+
+    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
+    public boolean isSetRqst() {
+      return this.rqst != null;
+    }
+
+    public void setRqstIsSet(boolean value) {
+      if (!value) {
+        this.rqst = null;
+      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case RQST:
+        if (value == null) {
+          unsetRqst();
+        } else {
+          setRqst((AbortTxnRequest)value);
+        }
+        break;
+
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case RQST:
+        return getRqst();
+
       }
       throw new IllegalStateException();
     }
@@ -205086,6 +207510,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
+      case RQST:
+        return isSetRqst();
       }
       throw new IllegalStateException();
     }
@@ -205094,15 +207520,24 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_open_txns_args)
-        return this.equals((get_open_txns_args)that);
+      if (that instanceof abort_txn_args)
+        return this.equals((abort_txn_args)that);
       return false;
     }
 
-    public boolean equals(get_open_txns_args that) {
+    public boolean equals(abort_txn_args that) {
       if (that == null)
         return false;
 
+      boolean this_present_rqst = true && this.isSetRqst();
+      boolean that_present_rqst = true && that.isSetRqst();
+      if (this_present_rqst || that_present_rqst) {
+        if (!(this_present_rqst && that_present_rqst))
+          return false;
+        if (!this.rqst.equals(that.rqst))
+          return false;
+      }
+
       return true;
     }
 
@@ -205110,17 +207545,32 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_rqst = true && (isSetRqst());
+      list.add(present_rqst);
+      if (present_rqst)
+        list.add(rqst);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_open_txns_args other) {
+    public int compareTo(abort_txn_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRqst()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -205138,9 +207588,16 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_open_txns_args(");
+      StringBuilder sb = new StringBuilder("abort_txn_args(");
       boolean first = true;
 
+      sb.append("rqst:");
+      if (this.rqst == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.rqst);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -205148,6 +207605,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (rqst != null) {
+        rqst.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -205166,15 +207626,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_open_txns_argsStandardSchemeFactory implements SchemeFactory {
-      public get_open_txns_argsStandardScheme getScheme() {
-        return new get_open_txns_argsStandardScheme();
+    private static class abort_txn_argsStandardSchemeFactory implements SchemeFactory {
+      public abort_txn_argsStandardScheme getScheme() {
+        return new abort_txn_argsStandardScheme();
       }
     }
 
-    private static class get_open_txns_argsStandardScheme extends StandardScheme<get_open_txns_args> {
+    private static class abort_txn_argsStandardScheme extends StandardScheme<abort_txn_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txn_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -205184,6 +207644,15 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
+            case 1: // RQST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.rqst = new AbortTxnRequest();
+                struct.rqst.read(iprot);
+                struct.setRqstIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -205193,53 +207662,72 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txn_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.rqst != null) {
+          oprot.writeFieldBegin(RQST_FIELD_DESC);
+          struct.rqst.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class get_open_txns_argsTupleSchemeFactory implements SchemeFactory {
-      public get_open_txns_argsTupleScheme getScheme() {
-        return new get_open_txns_argsTupleScheme();
+    private static class abort_txn_argsTupleSchemeFactory implements SchemeFactory {
+      public abort_txn_argsTupleScheme getScheme() {
+        return new abort_txn_argsTupleScheme();
       }
     }
 
-    private static class get_open_txns_argsTupleScheme extends TupleScheme<get_open_txns_args> {
+    private static class abort_txn_argsTupleScheme extends TupleScheme<abort_txn_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txn_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRqst()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRqst()) {
+          struct.rqst.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txn_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.rqst = new AbortTxnRequest();
+          struct.rqst.read(iprot);
+          struct.setRqstIsSet(true);
+        }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_result implements org.apache.thrift.TBase<get_open_txns_result, get_open_txns_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txn_result implements org.apache.thrift.TBase<abort_txn_result, abort_txn_result._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txn_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txn_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_open_txns_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_open_txns_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new abort_txn_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new abort_txn_resultTupleSchemeFactory());
     }
 
-    private GetOpenTxnsResponse success; // required
+    private NoSuchTxnException o1; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
+      O1((short)1, "o1");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -205254,8 +207742,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
+          case 1: // O1
+            return O1;
           default:
             return null;
         }
@@ -205299,70 +207787,70 @@ import org.slf4j.LoggerFactory;
     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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsResponse.class)));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txn_result.class, metaDataMap);
     }
 
-    public get_open_txns_result() {
+    public abort_txn_result() {
     }
 
-    public get_open_txns_result(
-      GetOpenTxnsResponse success)
+    public abort_txn_result(
+      NoSuchTxnException o1)
     {
       this();
-      this.success = success;
+      this.o1 = o1;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_open_txns_result(get_open_txns_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new GetOpenTxnsResponse(other.success);
+    public abort_txn_result(abort_txn_result other) {
+      if (other.isSetO1()) {
+        this.o1 = new NoSuchTxnException(other.o1);
       }
     }
 
-    public get_open_txns_result deepCopy() {
-      return new get_open_txns_result(this);
+    public abort_txn_result deepCopy() {
+      return new abort_txn_result(this);
     }
 
     @Override
     public void clear() {
-      this.success = null;
+      this.o1 = null;
     }
 
-    public GetOpenTxnsResponse getSuccess() {
-      return this.success;
+    public NoSuchTxnException getO1() {
+      return this.o1;
     }
 
-    public void setSuccess(GetOpenTxnsResponse success) {
-      this.success = success;
+    public void setO1(NoSuchTxnException o1) {
+      this.o1 = o1;
     }
 
-    public void unsetSuccess() {
-      this.success = null;
+    public void unsetO1() {
+      this.o1 = null;
     }
 
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
     }
 
-    public void setSuccessIsSet(boolean value) {
+    public void setO1IsSet(boolean value) {
       if (!value) {
-        this.success = null;
+        this.o1 = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case SUCCESS:
+      case O1:
         if (value == null) {
-          unsetSuccess();
+          unsetO1();
         } else {
-          setSuccess((GetOpenTxnsResponse)value);
+          setO1((NoSuchTxnException)value);
         }
         break;
 
@@ -205371,8 +207859,8 @@ import org.slf4j.LoggerFactory;
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case SUCCESS:
-        return getSuccess();
+      case O1:
+        return getO1();
 
       }
       throw new IllegalStateException();
@@ -205385,8 +207873,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
+      case O1:
+        return isSetO1();
       }
       throw new IllegalStateException();
     }
@@ -205395,21 +207883,21 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_open_txns_result)
-        return this.equals((get_open_txns_result)that);
+      if (that instanceof abort_txn_result)
+        return this.equals((abort_txn_result)that);
       return false;
     }
 
-    public boolean equals(get_open_txns_result that) {
+    public boolean equals(abort_txn_result that) {
       if (that == null)
         return false;
 
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
           return false;
-        if (!this.success.equals(that.success))
+        if (!this.o1.equals(that.o1))
           return false;
       }
 
@@ -205420,28 +207908,28 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_success = true && (isSetSuccess());
-      list.add(present_success);
-      if (present_success)
-        list.add(success);
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_open_txns_result other) {
+    public int compareTo(abort_txn_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -205463,14 +207951,14 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_open_txns_result(");
+      StringBuilder sb = new StringBuilder("abort_txn_result(");
       boolean first = true;
 
-      sb.append("success:");
-      if (this.success == null) {
+      sb.append("o1:");
+      if (this.o1 == null) {
         sb.append("null");
       } else {
-        sb.append(this.success);
+        sb.append(this.o1);
       }
       first = false;
       sb.append(")");
@@ -205480,9 +207968,6 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -205501,15 +207986,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_open_txns_resultStandardSchemeFactory implements SchemeFactory {
-      public get_open_txns_resultStandardScheme getScheme() {
-        return new get_open_txns_resultStandardScheme();
+    private static class abort_txn_resultStandardSchemeFactory implements SchemeFactory {
+      public abort_txn_resultStandardScheme getScheme() {
+        return new abort_txn_resultStandardScheme();
       }
     }
 
-    private static class get_open_txns_resultStandardScheme extends StandardScheme<get_open_txns_result> {
+    private static class abort_txn_resultStandardScheme extends StandardScheme<abort_txn_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txn_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -205519,11 +208004,11 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
+            case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new GetOpenTxnsResponse();
-                struct.success.read(iprot);
-                struct.setSuccessIsSet(true);
+                struct.o1 = new NoSuchTxnException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -205537,13 +208022,13 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txn_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -205552,55 +208037,57 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class get_open_txns_resultTupleSchemeFactory implements SchemeFactory {
-      public get_open_txns_resultTupleScheme getScheme() {
-        return new get_open_txns_resultTupleScheme();
+    private static class abort_txn_resultTupleSchemeFactory implements SchemeFactory {
+      public abort_txn_resultTupleScheme getScheme() {
+        return new abort_txn_resultTupleScheme();
       }
     }
 
-    private static class get_open_txns_resultTupleScheme extends TupleScheme<get_open_txns_result> {
+    private static class abort_txn_resultTupleScheme extends TupleScheme<abort_txn_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txn_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetSuccess()) {
+        if (struct.isSetO1()) {
           optionals.set(0);
         }
         oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txn_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new GetOpenTxnsResponse();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
+          struct.o1 = new NoSuchTxnException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
         }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_info_args implements org.apache.thrift.TBase<get_open_txns_info_args, get_open_txns_info_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_info_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_info_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txns_args implements org.apache.thrift.TBase<abort_txns_args, abort_txns_args._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txns_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txns_args");
 
+    private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_open_txns_info_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_open_txns_info_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new abort_txns_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new abort_txns_argsTupleSchemeFactory());
     }
 
+    private AbortTxnsRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-;
+      RQST((short)1, "rqst");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -205615,6 +208102,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 1: // RQST
+            return RQST;
           default:
             return null;
         }
@@ -205653,37 +208142,86 @@ import org.slf4j.LoggerFactory;
         return _fieldName;
       }
     }
+
+    // isset id assignments
     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.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AbortTxnsRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_info_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txns_args.class, metaDataMap);
     }
 
-    public get_open_txns_info_args() {
+    public abort_txns_args() {
+    }
+
+    public abort_txns_args(
+      AbortTxnsRequest rqst)
+    {
+      this();
+      this.rqst = rqst;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_open_txns_info_args(get_open_txns_info_args other) {
+    public abort_txns_args(abort_txns_args other) {
+      if (other.isSetRqst()) {
+        this.rqst = new AbortTxnsRequest(other.rqst);
+      }
     }
 
-    public get_open_txns_info_args deepCopy() {
-      return new get_open_txns_info_args(this);
+    public abort_txns_args deepCopy() {
+      return new abort_txns_args(this);
     }
 
     @Override
     public void clear() {
+      this.rqst = null;
+    }
+
+    public AbortTxnsRequest getRqst() {
+      return this.rqst;
+    }
+
+    public void setRqst(AbortTxnsRequest rqst) {
+      this.rqst = rqst;
+    }
+
+    public void unsetRqst() {
+      this.rqst = null;
+    }
+
+    /** Returns true if field rqst is set (has been assigned a value) and false otherwise */
+    public boolean isSetRqst() {
+      return this.rqst != null;
+    }
+
+    public void setRqstIsSet(boolean value) {
+      if (!value) {
+        this.rqst = null;
+      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case RQST:
+        if (value == null) {
+          unsetRqst();
+        } else {
+          setRqst((AbortTxnsRequest)value);
+        }
+        break;
+
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case RQST:
+        return getRqst();
+
       }
       throw new IllegalStateException();
     }
@@ -205695,6 +208233,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
+      case RQST:
+        return isSetRqst();
       }
       throw new IllegalStateException();
     }
@@ -205703,15 +208243,24 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_open_txns_info_args)
-        return this.equals((get_open_txns_info_args)that);
+      if (that instanceof abort_txns_args)
+        return this.equals((abort_txns_args)that);
       return false;
     }
 
-    public boolean equals(get_open_txns_info_args that) {
+    public boolean equals(abort_txns_args that) {
       if (that == null)
         return false;
 
+      boolean this_present_rqst = true && this.isSetRqst();
+      boolean that_present_rqst = true && that.isSetRqst();
+      if (this_present_rqst || that_present_rqst) {
+        if (!(this_present_rqst && that_present_rqst))
+          return false;
+        if (!this.rqst.equals(that.rqst))
+          return false;
+      }
+
       return true;
     }
 
@@ -205719,17 +208268,32 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_rqst = true && (isSetRqst());
+      list.add(present_rqst);
+      if (present_rqst)
+        list.add(rqst);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_open_txns_info_args other) {
+    public int compareTo(abort_txns_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetRqst()).compareTo(other.isSetRqst());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRqst()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rqst, other.rqst);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -205747,9 +208311,16 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_open_txns_info_args(");
+      StringBuilder sb = new StringBuilder("abort_txns_args(");
       boolean first = true;
 
+      sb.append("rqst:");
+      if (this.rqst == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.rqst);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -205757,6 +208328,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (rqst != null) {
+        rqst.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -205775,15 +208349,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_open_txns_info_argsStandardSchemeFactory implements SchemeFactory {
-      public get_open_txns_info_argsStandardScheme getScheme() {
-        return new get_open_txns_info_argsStandardScheme();
+    private static class abort_txns_argsStandardSchemeFactory implements SchemeFactory {
+      public abort_txns_argsStandardScheme getScheme() {
+        return new abort_txns_argsStandardScheme();
       }
     }
 
-    private static class get_open_txns_info_argsStandardScheme extends StandardScheme<get_open_txns_info_args> {
+    private static class abort_txns_argsStandardScheme extends StandardScheme<abort_txns_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txns_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -205793,6 +208367,15 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
+            case 1: // RQST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.rqst = new AbortTxnsRequest();
+                struct.rqst.read(iprot);
+                struct.setRqstIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -205802,53 +208385,72 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txns_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.rqst != null) {
+          oprot.writeFieldBegin(RQST_FIELD_DESC);
+          struct.rqst.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class get_open_txns_info_argsTupleSchemeFactory implements SchemeFactory {
-      public get_open_txns_info_argsTupleScheme getScheme() {
-        return new get_open_txns_info_argsTupleScheme();
+    private static class abort_txns_argsTupleSchemeFactory implements SchemeFactory {
+      public abort_txns_argsTupleScheme getScheme() {
+        return new abort_txns_argsTupleScheme();
       }
     }
 
-    private static class get_open_txns_info_argsTupleScheme extends TupleScheme<get_open_txns_info_args> {
+    private static class abort_txns_argsTupleScheme extends TupleScheme<abort_txns_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txns_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRqst()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRqst()) {
+          struct.rqst.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txns_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.rqst = new AbortTxnsRequest();
+          struct.rqst.read(iprot);
+          struct.setRqstIsSet(true);
+        }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_open_txns_info_result implements org.apache.thrift.TBase<get_open_txns_info_result, get_open_txns_info_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_open_txns_info_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_open_txns_info_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txns_result implements org.apache.thrift.TBase<abort_txns_result, abort_txns_result._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txns_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txns_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new get_open_txns_info_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new get_open_txns_info_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new abort_txns_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new abort_txns_resultTupleSchemeFactory());
     }
 
-    private GetOpenTxnsInfoResponse success; // required
+    private NoSuchTxnException o1; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
+      O1((short)1, "o1");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -205863,8 +208465,8 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
+          case 1: // O1
+            return O1;
           default:
             return null;
         }
@@ -205908,70 +208510,70 @@ import org.slf4j.LoggerFactory;
     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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetOpenTxnsInfoResponse.class)));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_open_txns_info_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txns_result.class, metaDataMap);
     }
 
-    public get_open_txns_info_result() {
+    public abort_txns_result() {
     }
 
-    public get_open_txns_info_result(
-      GetOpenTxnsInfoResponse success)
+    public abort_txns_result(
+      NoSuchTxnException o1)
     {
       this();
-      this.success = success;
+      this.o1 = o1;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public get_open_txns_info_result(get_open_txns_info_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new GetOpenTxnsInfoResponse(other.success);
+    public abort_txns_result(abort_txns_result other) {
+      if (other.isSetO1()) {
+        this.o1 = new NoSuchTxnException(other.o1);
       }
     }
 
-    public get_open_txns_info_result deepCopy() {
-      return new get_open_txns_info_result(this);
+    public abort_txns_result deepCopy() {
+      return new abort_txns_result(this);
     }
 
     @Override
     public void clear() {
-      this.success = null;
+      this.o1 = null;
     }
 
-    public GetOpenTxnsInfoResponse getSuccess() {
-      return this.success;
+    public NoSuchTxnException getO1() {
+      return this.o1;
     }
 
-    public void setSuccess(GetOpenTxnsInfoResponse success) {
-      this.success = success;
+    public void setO1(NoSuchTxnException o1) {
+      this.o1 = o1;
     }
 
-    public void unsetSuccess() {
-      this.success = null;
+    public void unsetO1() {
+      this.o1 = null;
     }
 
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
     }
 
-    public void setSuccessIsSet(boolean value) {
+    public void setO1IsSet(boolean value) {
       if (!value) {
-        this.success = null;
+        this.o1 = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case SUCCESS:
+      case O1:
         if (value == null) {
-          unsetSuccess();
+          unsetO1();
         } else {
-          setSuccess((GetOpenTxnsInfoResponse)value);
+          setO1((NoSuchTxnException)value);
         }
         break;
 
@@ -205980,8 +208582,8 @@ import org.slf4j.LoggerFactory;
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case SUCCESS:
-        return getSuccess();
+      case O1:
+        return getO1();
 
       }
       throw new IllegalStateException();
@@ -205994,8 +208596,8 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
+      case O1:
+        return isSetO1();
       }
       throw new IllegalStateException();
     }
@@ -206004,21 +208606,21 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof get_open_txns_info_result)
-        return this.equals((get_open_txns_info_result)that);
+      if (that instanceof abort_txns_result)
+        return this.equals((abort_txns_result)that);
       return false;
     }
 
-    public boolean equals(get_open_txns_info_result that) {
+    public boolean equals(abort_txns_result that) {
       if (that == null)
         return false;
 
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
           return false;
-        if (!this.success.equals(that.success))
+        if (!this.o1.equals(that.o1))
           return false;
       }
 
@@ -206029,28 +208631,28 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_success = true && (isSetSuccess());
-      list.add(present_success);
-      if (present_success)
-        list.add(success);
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(get_open_txns_info_result other) {
+    public int compareTo(abort_txns_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -206072,14 +208674,14 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("get_open_txns_info_result(");
+      StringBuilder sb = new StringBuilder("abort_txns_result(");
       boolean first = true;
 
-      sb.append("success:");
-      if (this.success == null) {
+      sb.append("o1:");
+      if (this.o1 == null) {
         sb.append("null");
       } else {
-        sb.append(this.success);
+        sb.append(this.o1);
       }
       first = false;
       sb.append(")");
@@ -206089,9 +208691,6 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -206110,15 +208709,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class get_open_txns_info_resultStandardSchemeFactory implements SchemeFactory {
-      public get_open_txns_info_resultStandardScheme getScheme() {
-        return new get_open_txns_info_resultStandardScheme();
+    private static class abort_txns_resultStandardSchemeFactory implements SchemeFactory {
+      public abort_txns_resultStandardScheme getScheme() {
+        return new abort_txns_resultStandardScheme();
       }
     }
 
-    private static class get_open_txns_info_resultStandardScheme extends StandardScheme<get_open_txns_info_result> {
+    private static class abort_txns_resultStandardScheme extends StandardScheme<abort_txns_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txns_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -206128,11 +208727,11 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
+            case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new GetOpenTxnsInfoResponse();
-                struct.success.read(iprot);
-                struct.setSuccessIsSet(true);
+                struct.o1 = new NoSuchTxnException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -206146,13 +208745,13 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txns_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -206161,53 +208760,53 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class get_open_txns_info_resultTupleSchemeFactory implements SchemeFactory {
-      public get_open_txns_info_resultTupleScheme getScheme() {
-        return new get_open_txns_info_resultTupleScheme();
+    private static class abort_txns_resultTupleSchemeFactory implements SchemeFactory {
+      public abort_txns_resultTupleScheme getScheme() {
+        return new abort_txns_resultTupleScheme();
       }
     }
 
-    private static class get_open_txns_info_resultTupleScheme extends TupleScheme<get_open_txns_info_result> {
+    private static class abort_txns_resultTupleScheme extends TupleScheme<abort_txns_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txns_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetSuccess()) {
+        if (struct.isSetO1()) {
           optionals.set(0);
         }
         oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, get_open_txns_info_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txns_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.success = new GetOpenTxnsInfoResponse();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
+          struct.o1 = new NoSuchTxnException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
         }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class open_txns_args implements org.apache.thrift.TBase<open_txns_args, open_txns_args._Fields>, java.io.Serializable, Cloneable, Comparable<open_txns_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("open_txns_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class commit_txn_args implements org.apache.thrift.TBase<commit_txn_args, commit_txn_args._Fields>, java.io.Serializable, Cloneable, Comparable<commit_txn_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("commit_txn_args");
 
     private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new open_txns_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new open_txns_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new commit_txn_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new commit_txn_argsTupleSchemeFactory());
     }
 
-    private OpenTxnRequest rqst; // required
+    private CommitTxnRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -206272,16 +208871,16 @@ import org.slf4j.LoggerFactory;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OpenTxnRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CommitTxnRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(open_txns_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(commit_txn_args.class, metaDataMap);
     }
 
-    public open_txns_args() {
+    public commit_txn_args() {
     }
 
-    public open_txns_args(
-      OpenTxnRequest rqst)
+    public commit_txn_args(
+      CommitTxnRequest rqst)
     {
       this();
       this.rqst = rqst;
@@ -206290,14 +208889,14 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public open_txns_args(open_txns_args other) {
+    public commit_txn_args(commit_txn_args other) {
       if (other.isSetRqst()) {
-        this.rqst = new OpenTxnRequest(other.rqst);
+        this.rqst = new CommitTxnRequest(other.rqst);
       }
     }
 
-    public open_txns_args deepCopy() {
-      return new open_txns_args(this);
+    public commit_txn_args deepCopy() {
+      return new commit_txn_args(this);
     }
 
     @Override
@@ -206305,11 +208904,11 @@ import org.slf4j.LoggerFactory;
       this.rqst = null;
     }
 
-    public OpenTxnRequest getRqst() {
+    public CommitTxnRequest getRqst() {
       return this.rqst;
     }
 
-    public void setRqst(OpenTxnRequest rqst) {
+    public void setRqst(CommitTxnRequest rqst) {
       this.rqst = rqst;
     }
 
@@ -206334,7 +208933,7 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetRqst();
         } else {
-          setRqst((OpenTxnRequest)value);
+          setRqst((CommitTxnRequest)value);
         }
         break;
 
@@ -206367,12 +208966,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof open_txns_args)
-        return this.equals((open_txns_args)that);
+      if (that instanceof commit_txn_args)
+        return this.equals((commit_txn_args)that);
       return false;
     }
 
-    public boolean equals(open_txns_args that) {
+    public boolean equals(commit_txn_args that) {
       if (that == null)
         return false;
 
@@ -206401,7 +209000,7 @@ import org.slf4j.LoggerFactory;
     }
 
     @Override
-    public int compareTo(open_txns_args other) {
+    public int compareTo(commit_txn_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -206435,7 +209034,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("open_txns_args(");
+      StringBuilder sb = new StringBuilder("commit_txn_args(");
       boolean first = true;
 
       sb.append("rqst:");
@@ -206473,15 +209072,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class open_txns_argsStandardSchemeFactory implements SchemeFactory {
-      public open_txns_argsStandardScheme getScheme() {
-        return new open_txns_argsStandardScheme();
+    private static class commit_txn_argsStandardSchemeFactory implements SchemeFactory {
+      public commit_txn_argsStandardScheme getScheme() {
+        return new commit_txn_argsStandardScheme();
       }
     }
 
-    private static class open_txns_argsStandardScheme extends StandardScheme<open_txns_args> {
+    private static class commit_txn_argsStandardScheme extends StandardScheme<commit_txn_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, open_txns_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, commit_txn_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -206493,7 +209092,7 @@ import org.slf4j.LoggerFactory;
           switch (schemeField.id) {
             case 1: // RQST
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.rqst = new OpenTxnRequest();
+                struct.rqst = new CommitTxnRequest();
                 struct.rqst.read(iprot);
                 struct.setRqstIsSet(true);
               } else { 
@@ -206509,7 +209108,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, open_txns_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, commit_txn_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -206524,16 +209123,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class open_txns_argsTupleSchemeFactory implements SchemeFactory {
-      public open_txns_argsTupleScheme getScheme() {
-        return new open_txns_argsTupleScheme();
+    private static class commit_txn_argsTupleSchemeFactory implements SchemeFactory {
+      public commit_txn_argsTupleScheme getScheme() {
+        return new commit_txn_argsTupleScheme();
       }
     }
 
-    private static class open_txns_argsTupleScheme extends TupleScheme<open_txns_args> {
+    private static class commit_txn_argsTupleScheme extends TupleScheme<commit_txn_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, open_txns_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, commit_txn_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetRqst()) {
@@ -206546,11 +209145,11 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, open_txns_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, commit_txn_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.rqst = new OpenTxnRequest();
+          struct.rqst = new CommitTxnRequest();
           struct.rqst.read(iprot);
           struct.setRqstIsSet(true);
         }
@@ -206559,22 +209158,25 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class open_txns_result implements org.apache.thrift.TBase<open_txns_result, open_txns_result._Fields>, java.io.Serializable, Cloneable, Comparable<open_txns_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("open_txns_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class commit_txn_result implements org.apache.thrift.TBase<commit_txn_result, commit_txn_result._Fields>, java.io.Serializable, Cloneable, Comparable<commit_txn_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("commit_txn_result");
 
-    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new open_txns_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new open_txns_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new commit_txn_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new commit_txn_resultTupleSchemeFactory());
     }
 
-    private OpenTxnsResponse success; // required
+    private NoSuchTxnException o1; // required
+    private TxnAbortedException o2; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      SUCCESS((short)0, "success");
+      O1((short)1, "o1"),
+      O2((short)2, "o2");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -206589,8 +209191,10 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 0: // SUCCESS
-            return SUCCESS;
+          case 1: // O1
+            return O1;
+          case 2: // O2
+            return O2;
           default:
             return null;
         }
@@ -206634,70 +209238,109 @@ import org.slf4j.LoggerFactory;
     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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, OpenTxnsResponse.class)));
+      tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(open_txns_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(commit_txn_result.class, metaDataMap);
     }
 
-    public open_txns_result() {
+    public commit_txn_result() {
     }
 
-    public open_txns_result(
-      OpenTxnsResponse success)
+    public commit_txn_result(
+      NoSuchTxnException o1,
+      TxnAbortedException o2)
     {
       this();
-      this.success = success;
+      this.o1 = o1;
+      this.o2 = o2;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public open_txns_result(open_txns_result other) {
-      if (other.isSetSuccess()) {
-        this.success = new OpenTxnsResponse(other.success);
+    public commit_txn_result(commit_txn_result other) {
+      if (other.isSetO1()) {
+        this.o1 = new NoSuchTxnException(other.o1);
+      }
+      if (other.isSetO2()) {
+        this.o2 = new TxnAbortedException(other.o2);
       }
     }
 
-    public open_txns_result deepCopy() {
-      return new open_txns_result(this);
+    public commit_txn_result deepCopy() {
+      return new commit_txn_result(this);
     }
 
     @Override
     public void clear() {
-      this.success = null;
+      this.o1 = null;
+      this.o2 = null;
     }
 
-    public OpenTxnsResponse getSuccess() {
-      return this.success;
+    public NoSuchTxnException getO1() {
+      return this.o1;
     }
 
-    public void setSuccess(OpenTxnsResponse success) {
-      this.success = success;
+    public void setO1(NoSuchTxnException o1) {
+      this.o1 = o1;
     }
 
-    public void unsetSuccess() {
-      this.success = null;
+    public void unsetO1() {
+      this.o1 = null;
     }
 
-    /** Returns true if field success is set (has been assigned a value) and false otherwise */
-    public boolean isSetSuccess() {
-      return this.success != null;
+    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO1() {
+      return this.o1 != null;
     }
 
-    public void setSuccessIsSet(boolean value) {
+    public void setO1IsSet(boolean value) {
       if (!value) {
-        this.success = null;
+        this.o1 = null;
+      }
+    }
+
+    public TxnAbortedException getO2() {
+      return this.o2;
+    }
+
+    public void setO2(TxnAbortedException o2) {
+      this.o2 = o2;
+    }
+
+    public void unsetO2() {
+      this.o2 = null;
+    }
+
+    /** Returns true if field o2 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO2() {
+      return this.o2 != null;
+    }
+
+    public void setO2IsSet(boolean value) {
+      if (!value) {
+        this.o2 = null;
       }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case SUCCESS:
+      case O1:
         if (value == null) {
-          unsetSuccess();
+          unsetO1();
         } else {
-          setSuccess((OpenTxnsResponse)value);
+          setO1((NoSuchTxnException)value);
+        }
+        break;
+
+      case O2:
+        if (value == null) {
+          unsetO2();
+        } else {
+          setO2((TxnAbortedException)value);
         }
         break;
 
@@ -206706,8 +209349,11 @@ import org.slf4j.LoggerFactory;
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case SUCCESS:
-        return getSuccess();
+      case O1:
+        return getO1();
+
+      case O2:
+        return getO2();
 
       }
       throw new IllegalStateException();
@@ -206720,8 +209366,10 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case SUCCESS:
-        return isSetSuccess();
+      case O1:
+        return isSetO1();
+      case O2:
+        return isSetO2();
       }
       throw new IllegalStateException();
     }
@@ -206730,21 +209378,30 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof open_txns_result)
-        return this.equals((open_txns_result)that);
+      if (that instanceof commit_txn_result)
+        return this.equals((commit_txn_result)that);
       return false;
     }
 
-    public boolean equals(open_txns_result that) {
+    public boolean equals(commit_txn_result that) {
       if (that == null)
         return false;
 
-      boolean this_present_success = true && this.isSetSuccess();
-      boolean that_present_success = true && that.isSetSuccess();
-      if (this_present_success || that_present_success) {
-        if (!(this_present_success && that_present_success))
+      boolean this_present_o1 = true && this.isSetO1();
+      boolean that_present_o1 = true && that.isSetO1();
+      if (this_present_o1 || that_present_o1) {
+        if (!(this_present_o1 && that_present_o1))
           return false;
-        if (!this.success.equals(that.success))
+        if (!this.o1.equals(that.o1))
+          return false;
+      }
+
+      boolean this_present_o2 = true && this.isSetO2();
+      boolean that_present_o2 = true && that.isSetO2();
+      if (this_present_o2 || that_present_o2) {
+        if (!(this_present_o2 && that_present_o2))
+          return false;
+        if (!this.o2.equals(that.o2))
           return false;
       }
 
@@ -206755,28 +209412,43 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_success = true && (isSetSuccess());
-      list.add(present_success);
-      if (present_success)
-        list.add(success);
+      boolean present_o1 = true && (isSetO1());
+      list.add(present_o1);
+      if (present_o1)
+        list.add(o1);
+
+      boolean present_o2 = true && (isSetO2());
+      list.add(present_o2);
+      if (present_o2)
+        list.add(o2);
 
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(open_txns_result other) {
+    public int compareTo(commit_txn_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
       }
-      if (isSetSuccess()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+      if (isSetO1()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetO2()).compareTo(other.isSetO2());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO2()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, other.o2);
         if (lastComparison != 0) {
           return lastComparison;
         }
@@ -206798,14 +209470,22 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("open_txns_result(");
+      StringBuilder sb = new StringBuilder("commit_txn_result(");
       boolean first = true;
 
-      sb.append("success:");
-      if (this.success == null) {
+      sb.append("o1:");
+      if (this.o1 == null) {
         sb.append("null");
       } else {
-        sb.append(this.success);
+        sb.append(this.o1);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("o2:");
+      if (this.o2 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o2);
       }
       first = false;
       sb.append(")");
@@ -206815,9 +209495,6 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
-      if (success != null) {
-        success.validate();
-      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -206836,15 +209513,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class open_txns_resultStandardSchemeFactory implements SchemeFactory {
-      public open_txns_resultStandardScheme getScheme() {
-        return new open_txns_resultStandardScheme();
+    private static class commit_txn_resultStandardSchemeFactory implements SchemeFactory {
+      public commit_txn_resultStandardScheme getScheme() {
+        return new commit_txn_resultStandardScheme();
       }
     }
 
-    private static class open_txns_resultStandardScheme extends StandardScheme<open_txns_result> {
+    private static class commit_txn_resultStandardScheme extends StandardScheme<commit_txn_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, open_txns_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, commit_txn_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -206854,11 +209531,20 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 0: // SUCCESS
+            case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.success = new OpenTxnsResponse();
-                struct.success.read(iprot);
-                struct.setSuccessIsSet(true);
+                struct.o1 = new NoSuchTxnException();
+                struct.o1.read(iprot);
+                struct.setO1IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // O2
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o2 = new TxnAbortedException();
+                struct.o2.read(iprot);
+                struct.setO2IsSet(true);
               } else { 
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
@@ -206872,13 +209558,18 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, open_txns_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, commit_txn_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.success != null) {
-          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
-          struct.success.write(oprot);
+        if (struct.o1 != null) {
+          oprot.writeFieldBegin(O1_FIELD_DESC);
+          struct.o1.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.o2 != null) {
+          oprot.writeFieldBegin(O2_FIELD_DESC);
+          struct.o2.write(oprot);
           oprot.writeFieldEnd();
         }
         oprot.writeFieldStop();
@@ -206887,53 +209578,64 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class open_txns_resultTupleSchemeFactory implements SchemeFactory {
-      public open_txns_resultTupleScheme getScheme() {
-        return new open_txns_resultTupleScheme();
+    private static class commit_txn_resultTupleSchemeFactory implements SchemeFactory {
+      public commit_txn_resultTupleScheme getScheme() {
+        return new commit_txn_resultTupleScheme();
       }
     }
 
-    private static class open_txns_resultTupleScheme extends TupleScheme<open_txns_result> {
+    private static class commit_txn_resultTupleScheme extends TupleScheme<commit_txn_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, open_txns_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, commit_txn_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetSuccess()) {
+        if (struct.isSetO1()) {
           optionals.set(0);
         }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetSuccess()) {
-          struct.success.write(oprot);
+        if (struct.isSetO2()) {
+          optionals.set(1);
+        }
+        oprot.writeBitSet(optionals, 2);
+        if (struct.isSetO1()) {
+          struct.o1.write(oprot);
+        }
+        if (struct.isSetO2()) {
+          struct.o2.write(oprot);
         }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, open_txns_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, commit_txn_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
+        BitSet incoming = iprot.readBitSet(2);
         if (incoming.get(0)) {
-          struct.success = new OpenTxnsResponse();
-          struct.success.read(iprot);
-          struct.setSuccessIsSet(true);
+          struct.o1 = new NoSuchTxnException();
+          struct.o1.read(iprot);
+          struct.setO1IsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.o2 = new TxnAbortedException();
+          struct.o2.read(iprot);
+          struct.setO2IsSet(true);
         }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txn_args implements org.apache.thrift.TBase<abort_txn_args, abort_txn_args._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txn_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txn_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class repl_tbl_writeid_state_args implements org.apache.thrift.TBase<repl_tbl_writeid_state_args, repl_tbl_writeid_state_args._Fields>, java.io.Serializable, Cloneable, Comparable<repl_tbl_writeid_state_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("repl_tbl_writeid_state_args");
 
     private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new abort_txn_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new abort_txn_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new repl_tbl_writeid_state_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new repl_tbl_writeid_state_argsTupleSchemeFactory());
     }
 
-    private AbortTxnRequest rqst; // required
+    private ReplTblWriteIdStateRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -206998,16 +209700,16 @@ import org.slf4j.LoggerFactory;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AbortTxnRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, ReplTblWriteIdStateRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txn_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(repl_tbl_writeid_state_args.class, metaDataMap);
     }
 
-    public abort_txn_args() {
+    public repl_tbl_writeid_state_args() {
     }
 
-    public abort_txn_args(
-      AbortTxnRequest rqst)
+    public repl_tbl_writeid_state_args(
+      ReplTblWriteIdStateRequest rqst)
     {
       this();
       this.rqst = rqst;
@@ -207016,14 +209718,14 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public abort_txn_args(abort_txn_args other) {
+    public repl_tbl_writeid_state_args(repl_tbl_writeid_state_args other) {
       if (other.isSetRqst()) {
-        this.rqst = new AbortTxnRequest(other.rqst);
+        this.rqst = new ReplTblWriteIdStateRequest(other.rqst);
       }
     }
 
-    public abort_txn_args deepCopy() {
-      return new abort_txn_args(this);
+    public repl_tbl_writeid_state_args deepCopy() {
+      return new repl_tbl_writeid_state_args(this);
     }
 
     @Override
@@ -207031,11 +209733,11 @@ import org.slf4j.LoggerFactory;
       this.rqst = null;
     }
 
-    public AbortTxnRequest getRqst() {
+    public ReplTblWriteIdStateRequest getRqst() {
       return this.rqst;
     }
 
-    public void setRqst(AbortTxnRequest rqst) {
+    public void setRqst(ReplTblWriteIdStateRequest rqst) {
       this.rqst = rqst;
     }
 
@@ -207060,7 +209762,7 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetRqst();
         } else {
-          setRqst((AbortTxnRequest)value);
+          setRqst((ReplTblWriteIdStateRequest)value);
         }
         break;
 
@@ -207093,12 +209795,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof abort_txn_args)
-        return this.equals((abort_txn_args)that);
+      if (that instanceof repl_tbl_writeid_state_args)
+        return this.equals((repl_tbl_writeid_state_args)that);
       return false;
     }
 
-    public boolean equals(abort_txn_args that) {
+    public boolean equals(repl_tbl_writeid_state_args that) {
       if (that == null)
         return false;
 
@@ -207127,7 +209829,7 @@ import org.slf4j.LoggerFactory;
     }
 
     @Override
-    public int compareTo(abort_txn_args other) {
+    public int compareTo(repl_tbl_writeid_state_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -207161,7 +209863,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("abort_txn_args(");
+      StringBuilder sb = new StringBuilder("repl_tbl_writeid_state_args(");
       boolean first = true;
 
       sb.append("rqst:");
@@ -207199,15 +209901,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class abort_txn_argsStandardSchemeFactory implements SchemeFactory {
-      public abort_txn_argsStandardScheme getScheme() {
-        return new abort_txn_argsStandardScheme();
+    private static class repl_tbl_writeid_state_argsStandardSchemeFactory implements SchemeFactory {
+      public repl_tbl_writeid_state_argsStandardScheme getScheme() {
+        return new repl_tbl_writeid_state_argsStandardScheme();
       }
     }
 
-    private static class abort_txn_argsStandardScheme extends StandardScheme<abort_txn_args> {
+    private static class repl_tbl_writeid_state_argsStandardScheme extends StandardScheme<repl_tbl_writeid_state_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txn_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, repl_tbl_writeid_state_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -207219,7 +209921,7 @@ import org.slf4j.LoggerFactory;
           switch (schemeField.id) {
             case 1: // RQST
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.rqst = new AbortTxnRequest();
+                struct.rqst = new ReplTblWriteIdStateRequest();
                 struct.rqst.read(iprot);
                 struct.setRqstIsSet(true);
               } else { 
@@ -207235,7 +209937,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txn_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, repl_tbl_writeid_state_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -207250,16 +209952,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class abort_txn_argsTupleSchemeFactory implements SchemeFactory {
-      public abort_txn_argsTupleScheme getScheme() {
-        return new abort_txn_argsTupleScheme();
+    private static class repl_tbl_writeid_state_argsTupleSchemeFactory implements SchemeFactory {
+      public repl_tbl_writeid_state_argsTupleScheme getScheme() {
+        return new repl_tbl_writeid_state_argsTupleScheme();
       }
     }
 
-    private static class abort_txn_argsTupleScheme extends TupleScheme<abort_txn_args> {
+    private static class repl_tbl_writeid_state_argsTupleScheme extends TupleScheme<repl_tbl_writeid_state_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txn_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, repl_tbl_writeid_state_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetRqst()) {
@@ -207272,11 +209974,11 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txn_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, repl_tbl_writeid_state_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.rqst = new AbortTxnRequest();
+          struct.rqst = new ReplTblWriteIdStateRequest();
           struct.rqst.read(iprot);
           struct.setRqstIsSet(true);
         }
@@ -207285,22 +209987,20 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txn_result implements org.apache.thrift.TBase<abort_txn_result, abort_txn_result._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txn_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txn_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class repl_tbl_writeid_state_result implements org.apache.thrift.TBase<repl_tbl_writeid_state_result, repl_tbl_writeid_state_result._Fields>, java.io.Serializable, Cloneable, Comparable<repl_tbl_writeid_state_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("repl_tbl_writeid_state_result");
 
-    private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new abort_txn_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new abort_txn_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new repl_tbl_writeid_state_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new repl_tbl_writeid_state_resultTupleSchemeFactory());
     }
 
-    private NoSuchTxnException o1; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      O1((short)1, "o1");
+;
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -207315,8 +210015,6 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
-          case 1: // O1
-            return O1;
           default:
             return null;
         }
@@ -207355,86 +210053,37 @@ import org.slf4j.LoggerFactory;
         return _fieldName;
       }
     }
-
-    // isset id assignments
     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.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txn_result.class, metaDataMap);
-    }
-
-    public abort_txn_result() {
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(repl_tbl_writeid_state_result.class, metaDataMap);
     }
 
-    public abort_txn_result(
-      NoSuchTxnException o1)
-    {
-      this();
-      this.o1 = o1;
+    public repl_tbl_writeid_state_result() {
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public abort_txn_result(abort_txn_result other) {
-      if (other.isSetO1()) {
-        this.o1 = new NoSuchTxnException(other.o1);
-      }
+    public repl_tbl_writeid_state_result(repl_tbl_writeid_state_result other) {
     }
 
-    public abort_txn_result deepCopy() {
-      return new abort_txn_result(this);
+    public repl_tbl_writeid_state_result deepCopy() {
+      return new repl_tbl_writeid_state_result(this);
     }
 
     @Override
     public void clear() {
-      this.o1 = null;
-    }
-
-    public NoSuchTxnException getO1() {
-      return this.o1;
-    }
-
-    public void setO1(NoSuchTxnException o1) {
-      this.o1 = o1;
-    }
-
-    public void unsetO1() {
-      this.o1 = null;
-    }
-
-    /** Returns true if field o1 is set (has been assigned a value) and false otherwise */
-    public boolean isSetO1() {
-      return this.o1 != null;
-    }
-
-    public void setO1IsSet(boolean value) {
-      if (!value) {
-        this.o1 = null;
-      }
     }
 
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
-      case O1:
-        if (value == null) {
-          unsetO1();
-        } else {
-          setO1((NoSuchTxnException)value);
-        }
-        break;
-
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
-      case O1:
-        return getO1();
-
       }
       throw new IllegalStateException();
     }
@@ -207446,8 +210095,6 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
-      case O1:
-        return isSetO1();
       }
       throw new IllegalStateException();
     }
@@ -207456,24 +210103,15 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof abort_txn_result)
-        return this.equals((abort_txn_result)that);
+      if (that instanceof repl_tbl_writeid_state_result)
+        return this.equals((repl_tbl_writeid_state_result)that);
       return false;
     }
 
-    public boolean equals(abort_txn_result that) {
+    public boolean equals(repl_tbl_writeid_state_result that) {
       if (that == null)
         return false;
 
-      boolean this_present_o1 = true && this.isSetO1();
-      boolean that_present_o1 = true && that.isSetO1();
-      if (this_present_o1 || that_present_o1) {
-        if (!(this_present_o1 && that_present_o1))
-          return false;
-        if (!this.o1.equals(that.o1))
-          return false;
-      }
-
       return true;
     }
 
@@ -207481,32 +210119,17 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
-      boolean present_o1 = true && (isSetO1());
-      list.add(present_o1);
-      if (present_o1)
-        list.add(o1);
-
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(abort_txn_result other) {
+    public int compareTo(repl_tbl_writeid_state_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
-      lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetO1()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o1, other.o1);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
       return 0;
     }
 
@@ -207524,16 +210147,9 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("abort_txn_result(");
+      StringBuilder sb = new StringBuilder("repl_tbl_writeid_state_result(");
       boolean first = true;
 
-      sb.append("o1:");
-      if (this.o1 == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.o1);
-      }
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -207559,15 +210175,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class abort_txn_resultStandardSchemeFactory implements SchemeFactory {
-      public abort_txn_resultStandardScheme getScheme() {
-        return new abort_txn_resultStandardScheme();
+    private static class repl_tbl_writeid_state_resultStandardSchemeFactory implements SchemeFactory {
+      public repl_tbl_writeid_state_resultStandardScheme getScheme() {
+        return new repl_tbl_writeid_state_resultStandardScheme();
       }
     }
 
-    private static class abort_txn_resultStandardScheme extends StandardScheme<abort_txn_result> {
+    private static class repl_tbl_writeid_state_resultStandardScheme extends StandardScheme<repl_tbl_writeid_state_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txn_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, repl_tbl_writeid_state_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -207577,15 +210193,6 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
-            case 1: // O1
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.o1 = new NoSuchTxnException();
-                struct.o1.read(iprot);
-                struct.setO1IsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -207595,68 +210202,49 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txn_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, repl_tbl_writeid_state_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.o1 != null) {
-          oprot.writeFieldBegin(O1_FIELD_DESC);
-          struct.o1.write(oprot);
-          oprot.writeFieldEnd();
-        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class abort_txn_resultTupleSchemeFactory implements SchemeFactory {
-      public abort_txn_resultTupleScheme getScheme() {
-        return new abort_txn_resultTupleScheme();
+    private static class repl_tbl_writeid_state_resultTupleSchemeFactory implements SchemeFactory {
+      public repl_tbl_writeid_state_resultTupleScheme getScheme() {
+        return new repl_tbl_writeid_state_resultTupleScheme();
       }
     }
 
-    private static class abort_txn_resultTupleScheme extends TupleScheme<abort_txn_result> {
+    private static class repl_tbl_writeid_state_resultTupleScheme extends TupleScheme<repl_tbl_writeid_state_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txn_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, repl_tbl_writeid_state_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetO1()) {
-          optionals.set(0);
-        }
-        oprot.writeBitSet(optionals, 1);
-        if (struct.isSetO1()) {
-          struct.o1.write(oprot);
-        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txn_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, repl_tbl_writeid_state_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
-        if (incoming.get(0)) {
-          struct.o1 = new NoSuchTxnException();
-          struct.o1.read(iprot);
-          struct.setO1IsSet(true);
-        }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txns_args implements org.apache.thrift.TBase<abort_txns_args, abort_txns_args._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txns_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txns_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_valid_write_ids_args implements org.apache.thrift.TBase<get_valid_write_ids_args, get_valid_write_ids_args._Fields>, java.io.Serializable, Cloneable, Comparable<get_valid_write_ids_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_valid_write_ids_args");
 
     private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new abort_txns_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new abort_txns_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_valid_write_ids_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_valid_write_ids_argsTupleSchemeFactory());
     }
 
-    private AbortTxnsRequest rqst; // required
+    private GetValidWriteIdsRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -207721,16 +210309,16 @@ import org.slf4j.LoggerFactory;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AbortTxnsRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetValidWriteIdsRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txns_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_valid_write_ids_args.class, metaDataMap);
     }
 
-    public abort_txns_args() {
+    public get_valid_write_ids_args() {
     }
 
-    public abort_txns_args(
-      AbortTxnsRequest rqst)
+    public get_valid_write_ids_args(
+      GetValidWriteIdsRequest rqst)
     {
       this();
       this.rqst = rqst;
@@ -207739,14 +210327,14 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public abort_txns_args(abort_txns_args other) {
+    public get_valid_write_ids_args(get_valid_write_ids_args other) {
       if (other.isSetRqst()) {
-        this.rqst = new AbortTxnsRequest(other.rqst);
+        this.rqst = new GetValidWriteIdsRequest(other.rqst);
       }
     }
 
-    public abort_txns_args deepCopy() {
-      return new abort_txns_args(this);
+    public get_valid_write_ids_args deepCopy() {
+      return new get_valid_write_ids_args(this);
     }
 
     @Override
@@ -207754,11 +210342,11 @@ import org.slf4j.LoggerFactory;
       this.rqst = null;
     }
 
-    public AbortTxnsRequest getRqst() {
+    public GetValidWriteIdsRequest getRqst() {
       return this.rqst;
     }
 
-    public void setRqst(AbortTxnsRequest rqst) {
+    public void setRqst(GetValidWriteIdsRequest rqst) {
       this.rqst = rqst;
     }
 
@@ -207783,7 +210371,7 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetRqst();
         } else {
-          setRqst((AbortTxnsRequest)value);
+          setRqst((GetValidWriteIdsRequest)value);
         }
         break;
 
@@ -207816,12 +210404,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof abort_txns_args)
-        return this.equals((abort_txns_args)that);
+      if (that instanceof get_valid_write_ids_args)
+        return this.equals((get_valid_write_ids_args)that);
       return false;
     }
 
-    public boolean equals(abort_txns_args that) {
+    public boolean equals(get_valid_write_ids_args that) {
       if (that == null)
         return false;
 
@@ -207850,7 +210438,7 @@ import org.slf4j.LoggerFactory;
     }
 
     @Override
-    public int compareTo(abort_txns_args other) {
+    public int compareTo(get_valid_write_ids_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -207884,7 +210472,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("abort_txns_args(");
+      StringBuilder sb = new StringBuilder("get_valid_write_ids_args(");
       boolean first = true;
 
       sb.append("rqst:");
@@ -207922,15 +210510,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class abort_txns_argsStandardSchemeFactory implements SchemeFactory {
-      public abort_txns_argsStandardScheme getScheme() {
-        return new abort_txns_argsStandardScheme();
+    private static class get_valid_write_ids_argsStandardSchemeFactory implements SchemeFactory {
+      public get_valid_write_ids_argsStandardScheme getScheme() {
+        return new get_valid_write_ids_argsStandardScheme();
       }
     }
 
-    private static class abort_txns_argsStandardScheme extends StandardScheme<abort_txns_args> {
+    private static class get_valid_write_ids_argsStandardScheme extends StandardScheme<get_valid_write_ids_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txns_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_valid_write_ids_args struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -207942,7 +210530,7 @@ import org.slf4j.LoggerFactory;
           switch (schemeField.id) {
             case 1: // RQST
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.rqst = new AbortTxnsRequest();
+                struct.rqst = new GetValidWriteIdsRequest();
                 struct.rqst.read(iprot);
                 struct.setRqstIsSet(true);
               } else { 
@@ -207958,7 +210546,7 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txns_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_valid_write_ids_args struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
@@ -207973,16 +210561,16 @@ import org.slf4j.LoggerFactory;
 
     }
 
-    private static class abort_txns_argsTupleSchemeFactory implements SchemeFactory {
-      public abort_txns_argsTupleScheme getScheme() {
-        return new abort_txns_argsTupleScheme();
+    private static class get_valid_write_ids_argsTupleSchemeFactory implements SchemeFactory {
+      public get_valid_write_ids_argsTupleScheme getScheme() {
+        return new get_valid_write_ids_argsTupleScheme();
       }
     }
 
-    private static class abort_txns_argsTupleScheme extends TupleScheme<abort_txns_args> {
+    private static class get_valid_write_ids_argsTupleScheme extends TupleScheme<get_valid_write_ids_args> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txns_args struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_valid_write_ids_args struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
         if (struct.isSetRqst()) {
@@ -207995,11 +210583,11 @@ import org.slf4j.LoggerFactory;
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txns_args struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_valid_write_ids_args struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
-          struct.rqst = new AbortTxnsRequest();
+          struct.rqst = new GetValidWriteIdsRequest();
           struct.rqst.read(iprot);
           struct.setRqstIsSet(true);
         }
@@ -208008,22 +210596,28 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class abort_txns_result implements org.apache.thrift.TBase<abort_txns_result, abort_txns_result._Fields>, java.io.Serializable, Cloneable, Comparable<abort_txns_result>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("abort_txns_result");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class get_valid_write_ids_result implements org.apache.thrift.TBase<get_valid_write_ids_result, get_valid_write_ids_result._Fields>, java.io.Serializable, Cloneable, Comparable<get_valid_write_ids_result>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("get_valid_write_ids_result");
 
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
     private static final org.apache.thrift.protocol.TField O1_FIELD_DESC = new org.apache.thrift.protocol.TField("o1", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField O2_FIELD_DESC = new org.apache.thrift.protocol.TField("o2", org.apache.thrift.protocol.TType.STRUCT, (short)2);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new abort_txns_resultStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new abort_txns_resultTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new get_valid_write_ids_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new get_valid_write_ids_resultTupleSchemeFactory());
     }
 
+    private GetValidWriteIdsResponse success; // required
     private NoSuchTxnException o1; // required
+    private MetaException o2; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      O1((short)1, "o1");
+      SUCCESS((short)0, "success"),
+      O1((short)1, "o1"),
+      O2((short)2, "o2");
 
       private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -208038,8 +210632,12 @@ import org.slf4j.LoggerFactory;
        */
       public static _Fields findByThriftId(int fieldId) {
         switch(fieldId) {
+          case 0: // SUCCESS
+            return SUCCESS;
           case 1: // O1
             return O1;
+          case 2: // O2
+            return O2;
           default:
             return null;
         }
@@ -208083,38 +210681,77 @@ import org.slf4j.LoggerFactory;
     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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, GetValidWriteIdsResponse.class)));
       tmpMap.put(_Fields.O1, new org.apache.thrift.meta_data.FieldMetaData("o1", org.apache.thrift.TFieldRequirementType.DEFAULT, 
           new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
+      tmpMap.put(_Fields.O2, new org.apache.thrift.meta_data.FieldMetaData("o2", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(abort_txns_result.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(get_valid_write_ids_result.class, metaDataMap);
     }
 
-    public abort_txns_result() {
+    public get_valid_write_ids_result() {
     }
 
-    public abort_txns_result(
-      NoSuchTxnException o1)
+    public get_valid_write_ids_result(
+      GetValidWriteIdsResponse success,
+      NoSuchTxnException o1,
+      MetaException o2)
     {
       this();
+      this.success = success;
       this.o1 = o1;
+      this.o2 = o2;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public abort_txns_result(abort_txns_result other) {
+    public get_valid_write_ids_result(get_valid_write_ids_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new GetValidWriteIdsResponse(other.success);
+      }
       if (other.isSetO1()) {
         this.o1 = new NoSuchTxnException(other.o1);
       }
+      if (other.isSetO2()) {
+        this.o2 = new MetaException(other.o2);
+      }
     }
 
-    public abort_txns_result deepCopy() {
-      return new abort_txns_result(this);
+    public get_valid_write_ids_result deepCopy() {
+      return new get_valid_write_ids_result(this);
     }
 
     @Override
     public void clear() {
+      this.success = null;
       this.o1 = null;
+      this.o2 = null;
+    }
+
+    public GetValidWriteIdsResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(GetValidWriteIdsResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
     }
 
     public NoSuchTxnException getO1() {
@@ -208140,8 +210777,39 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public MetaException getO2() {
+      return this.o2;
+    }
+
+    public void setO2(MetaException o2) {
+      this.o2 = o2;
+    }
+
+    public void unsetO2() {
+      this.o2 = null;
+    }
+
+    /** Returns true if field o2 is set (has been assigned a value) and false otherwise */
+    public boolean isSetO2() {
+      return this.o2 != null;
+    }
+
+    public void setO2IsSet(boolean value) {
+      if (!value) {
+        this.o2 = null;
+      }
+    }
+
     public void setFieldValue(_Fields field, Object value) {
       switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((GetValidWriteIdsResponse)value);
+        }
+        break;
+
       case O1:
         if (value == null) {
           unsetO1();
@@ -208150,14 +210818,28 @@ import org.slf4j.LoggerFactory;
         }
         break;
 
+      case O2:
+        if (value == null) {
+          unsetO2();
+        } else {
+          setO2((MetaException)value);
+        }
+        break;
+
       }
     }
 
     public Object getFieldValue(_Fields field) {
       switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
       case O1:
         return getO1();
 
+      case O2:
+        return getO2();
+
       }
       throw new IllegalStateException();
     }
@@ -208169,8 +210851,12 @@ import org.slf4j.LoggerFactory;
       }
 
       switch (field) {
+      case SUCCESS:
+        return isSetSuccess();
       case O1:
         return isSetO1();
+      case O2:
+        return isSetO2();
       }
       throw new IllegalStateException();
     }
@@ -208179,15 +210865,24 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof abort_txns_result)
-        return this.equals((abort_txns_result)that);
+      if (that instanceof get_valid_write_ids_result)
+        return this.equals((get_valid_write_ids_result)that);
       return false;
     }
 
-    public boolean equals(abort_txns_result that) {
+    public boolean equals(get_valid_write_ids_result that) {
       if (that == null)
         return false;
 
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
       boolean this_present_o1 = true && this.isSetO1();
       boolean that_present_o1 = true && that.isSetO1();
       if (this_present_o1 || that_present_o1) {
@@ -208197,6 +210892,15 @@ import org.slf4j.LoggerFactory;
           return false;
       }
 
+      boolean this_present_o2 = true && this.isSetO2();
+      boolean that_present_o2 = true && that.isSetO2();
+      if (this_present_o2 || that_present_o2) {
+        if (!(this_present_o2 && that_present_o2))
+          return false;
+        if (!this.o2.equals(that.o2))
+          return false;
+      }
+
       return true;
     }
 
@@ -208204,22 +210908,42 @@ import org.slf4j.LoggerFactory;
     public int hashCode() {
       List<Object> list = new ArrayList<Object>();
 
+      boolean present_success = true && (isSetSuccess());
+      list.add(present_success);
+      if (present_success)
+        list.add(success);
+
       boolean present_o1 = true && (isSetO1());
       list.add(present_o1);
       if (present_o1)
         list.add(o1);
 
+      boolean present_o2 = true && (isSetO2());
+      list.add(present_o2);
+      if (present_o2)
+        list.add(o2);
+
       return list.hashCode();
     }
 
     @Override
-    public int compareTo(abort_txns_result other) {
+    public int compareTo(get_valid_write_ids_result other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
 
       int lastComparison = 0;
 
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       lastComparison = Boolean.valueOf(isSetO1()).compareTo(other.isSetO1());
       if (lastComparison != 0) {
         return lastComparison;
@@ -208230,6 +210954,16 @@ import org.slf4j.LoggerFactory;
           return lastComparison;
         }
       }
+      lastComparison = Boolean.valueOf(isSetO2()).compareTo(other.isSetO2());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetO2()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.o2, other.o2);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
       return 0;
     }
 
@@ -208247,9 +210981,17 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("abort_txns_result(");
+      StringBuilder sb = new StringBuilder("get_valid_write_ids_result(");
       boolean first = true;
 
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      if (!first) sb.append(", ");
       sb.append("o1:");
       if (this.o1 == null) {
         sb.append("null");
@@ -208257,6 +210999,14 @@ import org.slf4j.LoggerFactory;
         sb.append(this.o1);
       }
       first = false;
+      if (!first) sb.append(", ");
+      sb.append("o2:");
+      if (this.o2 == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.o2);
+      }
+      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -208264,6 +211014,9 @@ import org.slf4j.LoggerFactory;
     public void validate() throws org.apache.thrift.TException {
       // check for required fields
       // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
     }
 
     private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -208282,15 +211035,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class abort_txns_resultStandardSchemeFactory implements SchemeFactory {
-      public abort_txns_resultStandardScheme getScheme() {
-        return new abort_txns_resultStandardScheme();
+    private static class get_valid_write_ids_resultStandardSchemeFactory implements SchemeFactory {
+      public get_valid_write_ids_resultStandardScheme getScheme() {
+        return new get_valid_write_ids_resultStandardScheme();
       }
     }
 
-    private static class abort_txns_resultStandardScheme extends StandardScheme<abort_txns_result> {
+    private static class get_valid_write_ids_resultStandardScheme extends StandardScheme<get_valid_write_ids_result> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, abort_txns_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol iprot, get_valid_write_ids_result struct) throws org.apache.thrift.TException {
         org.apache.thrift.protocol.TField schemeField;
         iprot.readStructBegin();
         while (true)
@@ -208300,6 +211053,15 @@ import org.slf4j.LoggerFactory;
             break;
           }
           switch (schemeField.id) {
+            case 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new GetValidWriteIdsResponse();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             case 1: // O1
               if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
                 struct.o1 = new NoSuchTxnException();
@@ -208309,6 +211071,15 @@ import org.slf4j.LoggerFactory;
                 org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
               }
               break;
+            case 2: // O2
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.o2 = new MetaException();
+                struct.o2.read(iprot);
+                struct.setO2IsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
             default:
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
           }
@@ -208318,68 +211089,100 @@ import org.slf4j.LoggerFactory;
         struct.validate();
       }
 
-      public void write(org.apache.thrift.protocol.TProtocol oprot, abort_txns_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol oprot, get_valid_write_ids_result struct) throws org.apache.thrift.TException {
         struct.validate();
 
         oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
         if (struct.o1 != null) {
           oprot.writeFieldBegin(O1_FIELD_DESC);
           struct.o1.write(oprot);
           oprot.writeFieldEnd();
         }
+        if (struct.o2 != null) {
+          oprot.writeFieldBegin(O2_FIELD_DESC);
+          struct.o2.write(oprot);
+          oprot.writeFieldEnd();
+        }
         oprot.writeFieldStop();
         oprot.writeStructEnd();
       }
 
     }
 
-    private static class abort_txns_resultTupleSchemeFactory implements SchemeFactory {
-      public abort_txns_resultTupleScheme getScheme() {
-        return new abort_txns_resultTupleScheme();
+    private static class get_valid_write_ids_resultTupleSchemeFactory implements SchemeFactory {
+      public get_valid_write_ids_resultTupleScheme getScheme() {
+        return new get_valid_write_ids_resultTupleScheme();
       }
     }
 
-    private static class abort_txns_resultTupleScheme extends TupleScheme<abort_txns_result> {
+    private static class get_valid_write_ids_resultTupleScheme extends TupleScheme<get_valid_write_ids_result> {
 
       @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, abort_txns_result struct) throws org.apache.thrift.TException {
+      public void write(org.apache.thrift.protocol.TProtocol prot, get_valid_write_ids_result struct) throws org.apache.thrift.TException {
         TTupleProtocol oprot = (TTupleProtocol) prot;
         BitSet optionals = new BitSet();
-        if (struct.isSetO1()) {
+        if (struct.isSetSuccess()) {
           optionals.set(0);
         }
-        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetO1()) {
+          optionals.set(1);
+        }
+        if (struct.isSetO2()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
         if (struct.isSetO1()) {
           struct.o1.write(oprot);
         }
+        if (struct.isSetO2()) {
+          struct.o2.write(oprot);
+        }
       }
 
       @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, abort_txns_result struct) throws org.apache.thrift.TException {
+      public void read(org.apache.thrift.protocol.TProtocol prot, get_valid_write_ids_result struct) throws org.apache.thrift.TException {
         TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(1);
+        BitSet incoming = iprot.readBitSet(3);
         if (incoming.get(0)) {
+          struct.success = new GetValidWriteIdsResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+        if (incoming.get(1)) {
           struct.o1 = new NoSuchTxnException();
           struct.o1.read(iprot);
           struct.setO1IsSet(true);
         }
+        if (incoming.get(2)) {
+          struct.o2 = new MetaException();
+          struct.o2.read(iprot);
+          struct.setO2IsSet(true);
+        }
       }
     }
 
   }
 
-  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class commit_txn_args implements org.apache.thrift.TBase<commit_txn_args, commit_txn_args._Fields>, java.io.Serializable, Cloneable, Comparable<commit_txn_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("commit_txn_args");
+  @org.apache.hadoop.classification.InterfaceAudience.Public @org.apache.hadoop.classification.InterfaceStability.Stable public static class allocate_table_write_ids_args implements org.apache.thrift.TBase<allocate_table_write_ids_args, allocate_table_write_ids_args._Fields>, java.io.Serializable, Cloneable, Comparable<allocate_table_write_ids_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("allocate_table_write_ids_args");
 
     private static final org.apache.thrift.protocol.TField RQST_FIELD_DESC = new org.apache.thrift.protocol.TField("rqst", org.apache.thrift.protocol.TType.STRUCT, (short)1);
 
     private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
     static {
-      schemes.put(StandardScheme.class, new commit_txn_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new commit_txn_argsTupleSchemeFactory());
+      schemes.put(StandardScheme.class, new allocate_table_write_ids_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new allocate_table_write_ids_argsTupleSchemeFactory());
     }
 
-    private CommitTxnRequest rqst; // required
+    private AllocateTableWriteIdsRequest rqst; // required
 
     /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
     public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -208444,16 +211247,16 @@ import org.slf4j.LoggerFactory;
     static {
       Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
       tmpMap.put(_Fields.RQST, new org.apache.thrift.meta_data.FieldMetaData("rqst", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, CommitTxnRequest.class)));
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, AllocateTableWriteIdsRequest.class)));
       metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(commit_txn_args.class, metaDataMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(allocate_table_write_ids_args.class, metaDataMap);
     }
 
-    public commit_txn_args() {
+    public allocate_table_write_ids_args() {
     }
 
-    public commit_txn_args(
-      CommitTxnRequest rqst)
+    public allocate_table_write_ids_args(
+      AllocateTableWriteIdsRequest rqst)
     {
       this();
       this.rqst = rqst;
@@ -208462,14 +211265,14 @@ import org.slf4j.LoggerFactory;
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public commit_txn_args(commit_txn_args other) {
+    public allocate_table_write_ids_args(allocate_table_write_ids_args other) {
       if (other.isSetRqst()) {
-        this.rqst = new CommitTxnRequest(other.rqst);
+        this.rqst = new AllocateTableWriteIdsRequest(other.rqst);
       }
     }
 
-    public commit_txn_args deepCopy() {
-      return new commit_txn_args(this);
+    public allocate_table_write_ids_args deepCopy() {
+      return new allocate_table_write_ids_args(this);
     }
 
     @Override
@@ -208477,11 +211280,11 @@ import org.slf4j.LoggerFactory;
       this.rqst = null;
     }
 
-    public CommitTxnRequest getRqst() {
+    public AllocateTableWriteIdsRequest getRqst() {
       return this.rqst;
     }
 
-    public void setRqst(CommitTxnRequest rqst) {
+    public void setRqst(AllocateTableWriteIdsRequest rqst) {
       this.rqst = rqst;
     }
 
@@ -208506,7 +211309,7 @@ import org.slf4j.LoggerFactory;
         if (value == null) {
           unsetRqst();
         } else {
-          setRqst((CommitTxnRequest)value);
+          setRqst((AllocateTableWriteIdsRequest)value);
         }
         break;
 
@@ -208539,12 +211342,12 @@ import org.slf4j.LoggerFactory;
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof commit_txn_args)
-        return this.equals((commit_txn_args)that);
+      if (that instanceof allocate_table_write_ids_args)
+        return this.equals((allocate_table_write_ids_args)that);
       return false;
     }
 
-    public boolean equals(commit_txn_args that) {
+    public boolean equals(allocate_table_write_ids_args that) {
       if (that == null)
         return false;
 
@@ -208573,7 +211376,7 @@ import org.slf4j.LoggerFactory;
     }
 
     @Override
-    public int compareTo(commit_txn_args other) {
+    public int compareTo(allocate_table_write_ids_args other) {
       if (!getClass().equals(other.getClass())) {
         return getClass().getName().compareTo(other.getClass().getName());
       }
@@ -208607,7 +211410,7 @@ import org.slf4j.LoggerFactory;
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("commit_txn_args(");
+      StringBuilder sb = new StringBuilder("allocate_table_write_ids_args(");
       boolean first = true;
 
       sb.append("rqst:");
@@ -208645,15 +211448,15 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    private static class commit_txn_argsStandardSchemeFactory implements SchemeFactory {
-      public commit_txn_argsStandardScheme getScheme() {
-        return new commit_txn_argsStandardScheme();
+    private static class allocate_table_write_ids_argsStandardSchemeFactory implements SchemeFactory {
+      public allocate_table_write_ids_argsStandardScheme getScheme() {
+        return new allocate_table_write_ids_argsStandardScheme();
       }
     }
 
-    private static class commit_txn_argsStandardScheme extends StandardScheme<commit_txn_args> {
+    private static class allocate_table_write_ids_argsStandardScheme extends StandardScheme<allocate_table_write_ids_args> {
 
-      public void read(org.apache.thrift.protocol.TProtocol iprot, commit_txn_args struct) throws org.apache.thrift.TException {
... 6324 lines suppressed ...