You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ek...@apache.org on 2016/05/19 20:00:30 UTC

[1/5] hive git commit: HIVE-13622 WriteSet tracking optimizations (Eugene Koifman, reviewed by Alan Gates)

Repository: hive
Updated Branches:
  refs/heads/branch-1 6c6583274 -> c0b532fce


http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
index 0770298..584cd45 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hive.ql.lockmgr;
 
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.TestTxnCommands2;
@@ -854,8 +855,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", null, locks.get(1));
     //update stmt has p=blah, thus nothing is actually update and we generate empty dyn part list
     Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(),
-      "default", "tab2", Collections.EMPTY_LIST));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(),
+      "default", "tab2", Collections.EMPTY_LIST);
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();
     //Short Running updated nothing, so we expect 0 rows in WRITE_SET
     Assert.assertEquals( 0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
@@ -869,8 +872,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", null, locks.get(1));//since TAB2 is empty
     //update stmt has p=blah, thus nothing is actually update and we generate empty dyn part list
     Assert.assertEquals(0, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(),
-      "default", "tab2", Collections.singletonList("p=two")));//simulate partition update
+    adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(),
+      "default", "tab2", Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);//simulate partition update
     txnMgr2.commitTxn();
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
       1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET"));
@@ -882,8 +887,10 @@ public class TestDbTxnManager2 {
     checkCmdOnDriver(driver.compileAndRespond("update TAB2 set b = 17 where a = 1"));//no rows match
     txnMgr.acquireLocks(driver.getPlan(), ctx, "Long Running");
     //so generate empty Dyn Part call
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(),
-      "default", "tab2", Collections.EMPTY_LIST));     
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(),
+      "default", "tab2", Collections.EMPTY_LIST);
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);     
     txnMgr.commitTxn();
 
     locks = getLocks(txnMgr);
@@ -984,16 +991,20 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=one", locks.get(1));
     
     //this simulates the completion of txnid:2
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab2",
-      Collections.singletonList("p=two")));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab2",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();//txnid:2
     
     locks = getLocks(txnMgr2);
     Assert.assertEquals("Unexpected lock count", 1, locks.size());
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB2", "p=one", locks.get(0));
     //completion of txnid:3
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab2",
-      Collections.singletonList("p=one")));
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab2",
+      Collections.singletonList("p=one"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr.commitTxn();//txnid:3
     //now both txns concurrently updated TAB2 but different partitions.
     
@@ -1031,8 +1042,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=one", locks.get(3));
 
     //this simulates the completion of txnid:5
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=one")));
+    adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=one"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();//txnid:5
 
     ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
@@ -1041,8 +1054,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
     //completion of txnid:6
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr.commitTxn();//txnid:6
 
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
@@ -1082,8 +1097,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
 
     //this simulates the completion of txnid:2
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=one")));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=one"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();//txnid:2
 
     ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
@@ -1091,8 +1108,10 @@ public class TestDbTxnManager2 {
     Assert.assertEquals("Unexpected lock count", 1, locks.size());
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
     //completion of txnid:3
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr.commitTxn();//txnid:3
 
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
@@ -1131,8 +1150,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
 
     //this simulates the completion of txnid:2
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=one")));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=one"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();//txnid:2
 
     ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
@@ -1140,14 +1161,22 @@ public class TestDbTxnManager2 {
     Assert.assertEquals("Unexpected lock count", 1, locks.size());
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
     //completion of txnid:3
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.DELETE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr.commitTxn();//txnid:3
 
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      2, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_txnid=1  and ctc_table='tab1'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_txnid=2  and ctc_table='tab1' and ctc_partition='p=one'"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_txnid=3  and ctc_table='tab1' and ctc_partition='p=two'"));
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
       1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=one' and ws_operation_type='u' and ws_table='tab1'"));
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
-      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='u' and ws_table='tab1'"));
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1'"));
     Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
       4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
   }
@@ -1180,8 +1209,10 @@ public class TestDbTxnManager2 {
     checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
 
     //this simulates the completion of txnid:2
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.UPDATE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();//txnid:2
 
     ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
@@ -1189,8 +1220,10 @@ public class TestDbTxnManager2 {
     Assert.assertEquals("Unexpected lock count", 1, locks.size());
     checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
     //completion of txnid:3
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.DELETE);
+    txnHandler.addDynamicPartitions(adp);
     LockException exception = null;
     try {
       txnMgr.commitTxn();//txnid:3
@@ -1210,11 +1243,7 @@ public class TestDbTxnManager2 {
   }
   /**
    * Concurrent delte/detele of same partition - should pass
-   * This test doesn't work yet, because we don't yet pass in operation type
-   * 
-   * todo: Concurrent insert/update of same partition - should pass
    */
-  @Ignore("HIVE-13622")
   @Test
   public void testWriteSetTracking11() throws Exception {
     CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
@@ -1232,46 +1261,86 @@ public class TestDbTxnManager2 {
 
     //now start concurrent txn
     txnMgr.openTxn("T3");
+    checkCmdOnDriver(driver.compileAndRespond("select * from tab1 where b=1 and p='one'"));
+    ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
     checkCmdOnDriver(driver.compileAndRespond("delete from tab1 where p='two' and b=2"));
     ((DbTxnManager)txnMgr).acquireLocks(driver.getPlan(), ctx, "T3", false);
     locks = getLocks(txnMgr);
-    Assert.assertEquals("Unexpected lock count", 3, locks.size());
-    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
-    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
-    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(2));
+    Assert.assertEquals("Unexpected lock count", 5, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB1", null, locks.get(0));
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(2));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(3));
+    checkLock(LockType.SHARED_WRITE, LockState.WAITING, "default", "TAB1", "p=two", locks.get(4));
 
     //this simulates the completion of txnid:2
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnMgr2.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.DELETE);
+    txnHandler.addDynamicPartitions(adp);
     txnMgr2.commitTxn();//txnid:2
 
-    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(2).getLockid());//retest WAITING locks (both have same ext id)
+    ((DbLockManager)txnMgr.getLockManager()).checkLock(locks.get(4).getLockid());//retest WAITING locks (both have same ext id)
     locks = getLocks(txnMgr);
-    Assert.assertEquals("Unexpected lock count", 1, locks.size());
-    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(0));
+    Assert.assertEquals("Unexpected lock count", 3, locks.size());
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB1", null, locks.get(0));
+    checkLock(LockType.SHARED_READ, LockState.ACQUIRED, "default", "TAB1", "p=one", locks.get(1));
+    checkLock(LockType.SHARED_WRITE, LockState.ACQUIRED, "default", "TAB1", "p=two", locks.get(2));
     //completion of txnid:3
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
-      Collections.singletonList("p=two")));
-    LockException exception = null;
-    try {
-      txnMgr.commitTxn();//txnid:3
-    }
-    catch(LockException e) {
-      exception = e;
-    }
-    Assert.assertNotEquals("Expected exception", null, exception);
-    Assert.assertEquals("Exception msg doesn't match",
-      "Aborting [txnid:3,3] due to a write conflict on default/tab1/p=two committed by [txnid:2,3]",
-      exception.getCause().getMessage());
+    adp = new AddDynamicPartitions(txnMgr.getCurrentTxnId(), "default", "tab1",
+      Collections.singletonList("p=two"));
+    adp.setOperationType(DataOperationType.DELETE);
+    txnHandler.addDynamicPartitions(adp);
+    txnMgr.commitTxn();//txnid:3
 
-    //todo: this currently fails since we don't yet set operation type properly
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
-      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1'"));
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1' and ws_txnid=2"));
     Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
-      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1'"));
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1' and ws_txnid=3"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1' and ws_txnid=2"));
+    Assert.assertEquals("WRITE_SET mismatch: " + TxnDbUtil.queryToString("select * from WRITE_SET"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from WRITE_SET where ws_partition='p=two' and ws_operation_type='d' and ws_table='tab1' and ws_txnid=3"));
     Assert.assertEquals("COMPLETED_TXN_COMPONENTS mismatch: " + TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
       4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_table='tab1' and ctc_partition is not null"));
   }
+  @Test
+  public void testCompletedTxnComponents() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    cpr = driver.run("create table if not exists tab_not_acid2 (a int, b int)");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab_not_acid2 values(1,1),(2,2)"));
+    //writing both acid and non-acid resources in the same txn
+    checkCmdOnDriver(driver.run("from tab_not_acid2 insert into tab1 partition(p='two')(a,b) select a,b insert into tab_not_acid2(a,b) select a,b "));//txnid:1
+    Assert.assertEquals(TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS"));
+    //only expect transactional components to be in COMPLETED_TXN_COMPONENTS
+    Assert.assertEquals(TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      1, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_txnid=1 and ctc_table='tab1'"));
+  }
+  @Test
+  public void testMultiInsert() throws Exception {
+    CommandProcessorResponse cpr = driver.run("create table if not exists tab1 (a int, b int) partitioned by (p string) " +
+      "clustered by (a) into 2  buckets stored as orc TBLPROPERTIES ('transactional'='true')");
+    checkCmdOnDriver(cpr);
+    cpr = driver.run("create table if not exists tab_not_acid (a int, b int, p string)");
+    checkCmdOnDriver(cpr);
+    checkCmdOnDriver(driver.run("insert into tab_not_acid values(1,1,'one'),(2,2,'two')"));
+    checkCmdOnDriver(driver.run("insert into tab1 partition(p) values(3,3,'one'),(4,4,'two')"));//txinid:1
+    //writing both acid and non-acid resources in the same txn
+    //tab1 write is a dynamic partition insert
+    checkCmdOnDriver(driver.run("from tab_not_acid insert into tab1 partition(p)(a,b,p) select a,b,p insert into tab_not_acid(a,b) select a,b where p='two'"));//txnid:2
+    Assert.assertEquals(TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      4, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS"));
+    //only expect transactional components to be in COMPLETED_TXN_COMPONENTS
+    Assert.assertEquals(TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      2, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_txnid=2"));
+    Assert.assertEquals(TxnDbUtil.queryToString("select * from COMPLETED_TXN_COMPONENTS"),
+      2, TxnDbUtil.countQueryAgent("select count(*) from COMPLETED_TXN_COMPONENTS where ctc_txnid=2 and ctc_table='tab1'"));
+  }
+  //todo: Concurrent insert/update of same partition - should pass
 
   private List<ShowLocksResponseElement> getLocksWithFilterOptions(HiveTxnManager txnMgr,
       String dbName, String tblName, Map<String, String> partSpec) throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
index 8d75ab3..b797b55 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java
@@ -208,6 +208,7 @@ public class TestCleaner extends CompactorTest {
 
     LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default");
     comp.setTablename("bblt");
+    comp.setOperationType(DataOperationType.SELECT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -246,6 +247,7 @@ public class TestCleaner extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("bblp");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -288,6 +290,7 @@ public class TestCleaner extends CompactorTest {
 
     LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default");
     comp.setTablename("bblt");
+    comp.setOperationType(DataOperationType.INSERT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -311,6 +314,7 @@ public class TestCleaner extends CompactorTest {
     // clean request
     LockComponent comp2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "default");
     comp2.setTablename("bblt");
+    comp.setOperationType(DataOperationType.SELECT);
     List<LockComponent> components2 = new ArrayList<LockComponent>(1);
     components2.add(comp2);
     LockRequest req2 = new LockRequest(components, "me", "localhost");
@@ -360,6 +364,7 @@ public class TestCleaner extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "default");
     comp.setTablename("bblt");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.INSERT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -385,6 +390,7 @@ public class TestCleaner extends CompactorTest {
     LockComponent comp2 = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "default");
     comp2.setTablename("bblt");
     comp2.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.SELECT);
     List<LockComponent> components2 = new ArrayList<LockComponent>(1);
     components2.add(comp2);
     LockRequest req2 = new LockRequest(components, "me", "localhost");

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
index f84bd7e..bbd2bf8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
@@ -110,6 +110,7 @@ public class TestInitiator extends CompactorTest {
       long txnid = openTxn();
       LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE,  "default");
       comp.setTablename("mcottma");
+      comp.setOperationType(DataOperationType.UPDATE);
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
       LockRequest req = new LockRequest(components, "me", "localhost");
@@ -140,6 +141,7 @@ public class TestInitiator extends CompactorTest {
       LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
       comp.setTablename("mcoptma");
       comp.setPartitionname("ds=today");
+      comp.setOperationType(DataOperationType.DELETE);
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
       LockRequest req = new LockRequest(components, "me", "localhost");
@@ -173,6 +175,7 @@ public class TestInitiator extends CompactorTest {
       LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE,  "default");
       comp.setTablename("ncomdpa");
       comp.setPartitionname("ds=day-" + i);
+      comp.setOperationType(DataOperationType.UPDATE);
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
       LockRequest req = new LockRequest(components, "me", "localhost");
@@ -197,6 +200,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("ceat");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -229,6 +233,7 @@ public class TestInitiator extends CompactorTest {
       long txnid = openTxn();
       LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
       comp.setTablename("ncwncs");
+      comp.setOperationType(DataOperationType.UPDATE);
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
       LockRequest req = new LockRequest(components, "me", "localhost");
@@ -254,6 +259,7 @@ public class TestInitiator extends CompactorTest {
     for (int i = 0; i < 11; i++) {
       long txnid = openTxn();
       LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
+      comp.setOperationType(DataOperationType.DELETE);
       comp.setTablename("ncwncs");
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
@@ -279,6 +285,7 @@ public class TestInitiator extends CompactorTest {
       long txnid = openTxn();
       LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE,  "default");
       comp.setTablename("ncwcas");
+      comp.setOperationType(DataOperationType.UPDATE);
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
       LockRequest req = new LockRequest(components, "me", "localhost");
@@ -319,6 +326,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("cthdp");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -351,6 +359,7 @@ public class TestInitiator extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("cphdp");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -382,6 +391,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("nctdpnhe");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -417,6 +427,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("cttmd");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -458,6 +469,7 @@ public class TestInitiator extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("cptmd");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -489,6 +501,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("nctned");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -524,6 +537,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("cmomwbv");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -564,6 +578,7 @@ public class TestInitiator extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("ednb");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -597,6 +612,7 @@ public class TestInitiator extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("ttospgocr");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -608,6 +624,7 @@ public class TestInitiator extends CompactorTest {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("ttospgocr");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.UPDATE);
     components = new ArrayList<LockComponent>(1);
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -640,6 +657,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "default");
     comp.setTablename("nctdp");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -667,6 +685,7 @@ public class TestInitiator extends CompactorTest {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("dt");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -698,6 +717,7 @@ public class TestInitiator extends CompactorTest {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.PARTITION, "default");
     comp.setTablename("dp");
     comp.setPartitionname("ds=today");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
----------------------------------------------------------------------
diff --git a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
index af2d93d..cbdbd56 100755
--- a/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
+++ b/service/src/gen/thrift/gen-py/hive_service/ThriftHive-remote
@@ -145,6 +145,7 @@ if len(sys.argv) <= 1 or sys.argv[1] == '--help':
   print('  GetOpenTxnsInfoResponse get_open_txns_info()')
   print('  OpenTxnsResponse open_txns(OpenTxnRequest rqst)')
   print('  void abort_txn(AbortTxnRequest rqst)')
+  print('  void abort_txns(AbortTxnsRequest rqst)')
   print('  void commit_txn(CommitTxnRequest rqst)')
   print('  LockResponse lock(LockRequest rqst)')
   print('  LockResponse check_lock(CheckLockRequest rqst)')
@@ -953,6 +954,12 @@ elif cmd == 'abort_txn':
     sys.exit(1)
   pp.pprint(client.abort_txn(eval(args[0]),))
 
+elif cmd == 'abort_txns':
+  if len(args) != 1:
+    print('abort_txns requires 1 args')
+    sys.exit(1)
+  pp.pprint(client.abort_txns(eval(args[0]),))
+
 elif cmd == 'commit_txn':
   if len(args) != 1:
     print('commit_txn requires 1 args')


[3/5] hive git commit: HIVE-13622 WriteSet tracking optimizations (Eugene Koifman, reviewed by Alan Gates)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 7ea4493..9e2e883 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -119,6 +119,24 @@ const char* _kGrantRevokeTypeNames[] = {
 };
 const std::map<int, const char*> _GrantRevokeType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(2, _kGrantRevokeTypeValues, _kGrantRevokeTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
+int _kDataOperationTypeValues[] = {
+  DataOperationType::SELECT,
+  DataOperationType::INSERT,
+  DataOperationType::UPDATE,
+  DataOperationType::DELETE,
+  DataOperationType::UNSET,
+  DataOperationType::NO_TXN
+};
+const char* _kDataOperationTypeNames[] = {
+  "SELECT",
+  "INSERT",
+  "UPDATE",
+  "DELETE",
+  "UNSET",
+  "NO_TXN"
+};
+const std::map<int, const char*> _DataOperationType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(6, _kDataOperationTypeValues, _kDataOperationTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+
 int _kEventRequestTypeValues[] = {
   EventRequestType::INSERT,
   EventRequestType::UPDATE,
@@ -10999,6 +11017,16 @@ void LockComponent::__set_partitionname(const std::string& val) {
 __isset.partitionname = true;
 }
 
+void LockComponent::__set_operationType(const DataOperationType::type val) {
+  this->operationType = val;
+__isset.operationType = true;
+}
+
+void LockComponent::__set_isAcid(const bool val) {
+  this->isAcid = val;
+__isset.isAcid = true;
+}
+
 uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -11067,6 +11095,24 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast508;
+          xfer += iprot->readI32(ecast508);
+          this->operationType = (DataOperationType::type)ecast508;
+          this->__isset.operationType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_BOOL) {
+          xfer += iprot->readBool(this->isAcid);
+          this->__isset.isAcid = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -11112,6 +11158,16 @@ uint32_t LockComponent::write(::apache::thrift::protocol::TProtocol* oprot) cons
     xfer += oprot->writeString(this->partitionname);
     xfer += oprot->writeFieldEnd();
   }
+  if (this->__isset.operationType) {
+    xfer += oprot->writeFieldBegin("operationType", ::apache::thrift::protocol::T_I32, 6);
+    xfer += oprot->writeI32((int32_t)this->operationType);
+    xfer += oprot->writeFieldEnd();
+  }
+  if (this->__isset.isAcid) {
+    xfer += oprot->writeFieldBegin("isAcid", ::apache::thrift::protocol::T_BOOL, 7);
+    xfer += oprot->writeBool(this->isAcid);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -11124,24 +11180,30 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.dbname, b.dbname);
   swap(a.tablename, b.tablename);
   swap(a.partitionname, b.partitionname);
+  swap(a.operationType, b.operationType);
+  swap(a.isAcid, b.isAcid);
   swap(a.__isset, b.__isset);
 }
 
-LockComponent::LockComponent(const LockComponent& other508) {
-  type = other508.type;
-  level = other508.level;
-  dbname = other508.dbname;
-  tablename = other508.tablename;
-  partitionname = other508.partitionname;
-  __isset = other508.__isset;
-}
-LockComponent& LockComponent::operator=(const LockComponent& other509) {
+LockComponent::LockComponent(const LockComponent& other509) {
   type = other509.type;
   level = other509.level;
   dbname = other509.dbname;
   tablename = other509.tablename;
   partitionname = other509.partitionname;
+  operationType = other509.operationType;
+  isAcid = other509.isAcid;
   __isset = other509.__isset;
+}
+LockComponent& LockComponent::operator=(const LockComponent& other510) {
+  type = other510.type;
+  level = other510.level;
+  dbname = other510.dbname;
+  tablename = other510.tablename;
+  partitionname = other510.partitionname;
+  operationType = other510.operationType;
+  isAcid = other510.isAcid;
+  __isset = other510.__isset;
   return *this;
 }
 void LockComponent::printTo(std::ostream& out) const {
@@ -11152,6 +11214,8 @@ void LockComponent::printTo(std::ostream& out) const {
   out << ", " << "dbname=" << to_string(dbname);
   out << ", " << "tablename="; (__isset.tablename ? (out << to_string(tablename)) : (out << "<null>"));
   out << ", " << "partitionname="; (__isset.partitionname ? (out << to_string(partitionname)) : (out << "<null>"));
+  out << ", " << "operationType="; (__isset.operationType ? (out << to_string(operationType)) : (out << "<null>"));
+  out << ", " << "isAcid="; (__isset.isAcid ? (out << to_string(isAcid)) : (out << "<null>"));
   out << ")";
 }
 
@@ -11210,14 +11274,14 @@ uint32_t LockRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->component.clear();
-            uint32_t _size510;
-            ::apache::thrift::protocol::TType _etype513;
-            xfer += iprot->readListBegin(_etype513, _size510);
-            this->component.resize(_size510);
-            uint32_t _i514;
-            for (_i514 = 0; _i514 < _size510; ++_i514)
+            uint32_t _size511;
+            ::apache::thrift::protocol::TType _etype514;
+            xfer += iprot->readListBegin(_etype514, _size511);
+            this->component.resize(_size511);
+            uint32_t _i515;
+            for (_i515 = 0; _i515 < _size511; ++_i515)
             {
-              xfer += this->component[_i514].read(iprot);
+              xfer += this->component[_i515].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11284,10 +11348,10 @@ uint32_t LockRequest::write(::apache::thrift::protocol::TProtocol* oprot) const
   xfer += oprot->writeFieldBegin("component", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->component.size()));
-    std::vector<LockComponent> ::const_iterator _iter515;
-    for (_iter515 = this->component.begin(); _iter515 != this->component.end(); ++_iter515)
+    std::vector<LockComponent> ::const_iterator _iter516;
+    for (_iter516 = this->component.begin(); _iter516 != this->component.end(); ++_iter516)
     {
-      xfer += (*_iter515).write(oprot);
+      xfer += (*_iter516).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -11326,21 +11390,21 @@ void swap(LockRequest &a, LockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-LockRequest::LockRequest(const LockRequest& other516) {
-  component = other516.component;
-  txnid = other516.txnid;
-  user = other516.user;
-  hostname = other516.hostname;
-  agentInfo = other516.agentInfo;
-  __isset = other516.__isset;
-}
-LockRequest& LockRequest::operator=(const LockRequest& other517) {
+LockRequest::LockRequest(const LockRequest& other517) {
   component = other517.component;
   txnid = other517.txnid;
   user = other517.user;
   hostname = other517.hostname;
   agentInfo = other517.agentInfo;
   __isset = other517.__isset;
+}
+LockRequest& LockRequest::operator=(const LockRequest& other518) {
+  component = other518.component;
+  txnid = other518.txnid;
+  user = other518.user;
+  hostname = other518.hostname;
+  agentInfo = other518.agentInfo;
+  __isset = other518.__isset;
   return *this;
 }
 void LockRequest::printTo(std::ostream& out) const {
@@ -11400,9 +11464,9 @@ uint32_t LockResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast518;
-          xfer += iprot->readI32(ecast518);
-          this->state = (LockState::type)ecast518;
+          int32_t ecast519;
+          xfer += iprot->readI32(ecast519);
+          this->state = (LockState::type)ecast519;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11448,13 +11512,13 @@ void swap(LockResponse &a, LockResponse &b) {
   swap(a.state, b.state);
 }
 
-LockResponse::LockResponse(const LockResponse& other519) {
-  lockid = other519.lockid;
-  state = other519.state;
-}
-LockResponse& LockResponse::operator=(const LockResponse& other520) {
+LockResponse::LockResponse(const LockResponse& other520) {
   lockid = other520.lockid;
   state = other520.state;
+}
+LockResponse& LockResponse::operator=(const LockResponse& other521) {
+  lockid = other521.lockid;
+  state = other521.state;
   return *this;
 }
 void LockResponse::printTo(std::ostream& out) const {
@@ -11576,17 +11640,17 @@ void swap(CheckLockRequest &a, CheckLockRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CheckLockRequest::CheckLockRequest(const CheckLockRequest& other521) {
-  lockid = other521.lockid;
-  txnid = other521.txnid;
-  elapsed_ms = other521.elapsed_ms;
-  __isset = other521.__isset;
-}
-CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other522) {
+CheckLockRequest::CheckLockRequest(const CheckLockRequest& other522) {
   lockid = other522.lockid;
   txnid = other522.txnid;
   elapsed_ms = other522.elapsed_ms;
   __isset = other522.__isset;
+}
+CheckLockRequest& CheckLockRequest::operator=(const CheckLockRequest& other523) {
+  lockid = other523.lockid;
+  txnid = other523.txnid;
+  elapsed_ms = other523.elapsed_ms;
+  __isset = other523.__isset;
   return *this;
 }
 void CheckLockRequest::printTo(std::ostream& out) const {
@@ -11670,11 +11734,11 @@ void swap(UnlockRequest &a, UnlockRequest &b) {
   swap(a.lockid, b.lockid);
 }
 
-UnlockRequest::UnlockRequest(const UnlockRequest& other523) {
-  lockid = other523.lockid;
-}
-UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other524) {
+UnlockRequest::UnlockRequest(const UnlockRequest& other524) {
   lockid = other524.lockid;
+}
+UnlockRequest& UnlockRequest::operator=(const UnlockRequest& other525) {
+  lockid = other525.lockid;
   return *this;
 }
 void UnlockRequest::printTo(std::ostream& out) const {
@@ -11813,19 +11877,19 @@ void swap(ShowLocksRequest &a, ShowLocksRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other525) {
-  dbname = other525.dbname;
-  tablename = other525.tablename;
-  partname = other525.partname;
-  isExtended = other525.isExtended;
-  __isset = other525.__isset;
-}
-ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other526) {
+ShowLocksRequest::ShowLocksRequest(const ShowLocksRequest& other526) {
   dbname = other526.dbname;
   tablename = other526.tablename;
   partname = other526.partname;
   isExtended = other526.isExtended;
   __isset = other526.__isset;
+}
+ShowLocksRequest& ShowLocksRequest::operator=(const ShowLocksRequest& other527) {
+  dbname = other527.dbname;
+  tablename = other527.tablename;
+  partname = other527.partname;
+  isExtended = other527.isExtended;
+  __isset = other527.__isset;
   return *this;
 }
 void ShowLocksRequest::printTo(std::ostream& out) const {
@@ -11978,9 +12042,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 5:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast527;
-          xfer += iprot->readI32(ecast527);
-          this->state = (LockState::type)ecast527;
+          int32_t ecast528;
+          xfer += iprot->readI32(ecast528);
+          this->state = (LockState::type)ecast528;
           isset_state = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -11988,9 +12052,9 @@ uint32_t ShowLocksResponseElement::read(::apache::thrift::protocol::TProtocol* i
         break;
       case 6:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast528;
-          xfer += iprot->readI32(ecast528);
-          this->type = (LockType::type)ecast528;
+          int32_t ecast529;
+          xfer += iprot->readI32(ecast529);
+          this->type = (LockType::type)ecast529;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12206,26 +12270,7 @@ void swap(ShowLocksResponseElement &a, ShowLocksResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other529) {
-  lockid = other529.lockid;
-  dbname = other529.dbname;
-  tablename = other529.tablename;
-  partname = other529.partname;
-  state = other529.state;
-  type = other529.type;
-  txnid = other529.txnid;
-  lastheartbeat = other529.lastheartbeat;
-  acquiredat = other529.acquiredat;
-  user = other529.user;
-  hostname = other529.hostname;
-  heartbeatCount = other529.heartbeatCount;
-  agentInfo = other529.agentInfo;
-  blockedByExtId = other529.blockedByExtId;
-  blockedByIntId = other529.blockedByIntId;
-  lockIdInternal = other529.lockIdInternal;
-  __isset = other529.__isset;
-}
-ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other530) {
+ShowLocksResponseElement::ShowLocksResponseElement(const ShowLocksResponseElement& other530) {
   lockid = other530.lockid;
   dbname = other530.dbname;
   tablename = other530.tablename;
@@ -12243,6 +12288,25 @@ ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksRes
   blockedByIntId = other530.blockedByIntId;
   lockIdInternal = other530.lockIdInternal;
   __isset = other530.__isset;
+}
+ShowLocksResponseElement& ShowLocksResponseElement::operator=(const ShowLocksResponseElement& other531) {
+  lockid = other531.lockid;
+  dbname = other531.dbname;
+  tablename = other531.tablename;
+  partname = other531.partname;
+  state = other531.state;
+  type = other531.type;
+  txnid = other531.txnid;
+  lastheartbeat = other531.lastheartbeat;
+  acquiredat = other531.acquiredat;
+  user = other531.user;
+  hostname = other531.hostname;
+  heartbeatCount = other531.heartbeatCount;
+  agentInfo = other531.agentInfo;
+  blockedByExtId = other531.blockedByExtId;
+  blockedByIntId = other531.blockedByIntId;
+  lockIdInternal = other531.lockIdInternal;
+  __isset = other531.__isset;
   return *this;
 }
 void ShowLocksResponseElement::printTo(std::ostream& out) const {
@@ -12301,14 +12365,14 @@ uint32_t ShowLocksResponse::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->locks.clear();
-            uint32_t _size531;
-            ::apache::thrift::protocol::TType _etype534;
-            xfer += iprot->readListBegin(_etype534, _size531);
-            this->locks.resize(_size531);
-            uint32_t _i535;
-            for (_i535 = 0; _i535 < _size531; ++_i535)
+            uint32_t _size532;
+            ::apache::thrift::protocol::TType _etype535;
+            xfer += iprot->readListBegin(_etype535, _size532);
+            this->locks.resize(_size532);
+            uint32_t _i536;
+            for (_i536 = 0; _i536 < _size532; ++_i536)
             {
-              xfer += this->locks[_i535].read(iprot);
+              xfer += this->locks[_i536].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12337,10 +12401,10 @@ uint32_t ShowLocksResponse::write(::apache::thrift::protocol::TProtocol* oprot)
   xfer += oprot->writeFieldBegin("locks", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->locks.size()));
-    std::vector<ShowLocksResponseElement> ::const_iterator _iter536;
-    for (_iter536 = this->locks.begin(); _iter536 != this->locks.end(); ++_iter536)
+    std::vector<ShowLocksResponseElement> ::const_iterator _iter537;
+    for (_iter537 = this->locks.begin(); _iter537 != this->locks.end(); ++_iter537)
     {
-      xfer += (*_iter536).write(oprot);
+      xfer += (*_iter537).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -12357,13 +12421,13 @@ void swap(ShowLocksResponse &a, ShowLocksResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other537) {
-  locks = other537.locks;
-  __isset = other537.__isset;
-}
-ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other538) {
+ShowLocksResponse::ShowLocksResponse(const ShowLocksResponse& other538) {
   locks = other538.locks;
   __isset = other538.__isset;
+}
+ShowLocksResponse& ShowLocksResponse::operator=(const ShowLocksResponse& other539) {
+  locks = other539.locks;
+  __isset = other539.__isset;
   return *this;
 }
 void ShowLocksResponse::printTo(std::ostream& out) const {
@@ -12464,15 +12528,15 @@ void swap(HeartbeatRequest &a, HeartbeatRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other539) {
-  lockid = other539.lockid;
-  txnid = other539.txnid;
-  __isset = other539.__isset;
-}
-HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other540) {
+HeartbeatRequest::HeartbeatRequest(const HeartbeatRequest& other540) {
   lockid = other540.lockid;
   txnid = other540.txnid;
   __isset = other540.__isset;
+}
+HeartbeatRequest& HeartbeatRequest::operator=(const HeartbeatRequest& other541) {
+  lockid = other541.lockid;
+  txnid = other541.txnid;
+  __isset = other541.__isset;
   return *this;
 }
 void HeartbeatRequest::printTo(std::ostream& out) const {
@@ -12575,13 +12639,13 @@ void swap(HeartbeatTxnRangeRequest &a, HeartbeatTxnRangeRequest &b) {
   swap(a.max, b.max);
 }
 
-HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other541) {
-  min = other541.min;
-  max = other541.max;
-}
-HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other542) {
+HeartbeatTxnRangeRequest::HeartbeatTxnRangeRequest(const HeartbeatTxnRangeRequest& other542) {
   min = other542.min;
   max = other542.max;
+}
+HeartbeatTxnRangeRequest& HeartbeatTxnRangeRequest::operator=(const HeartbeatTxnRangeRequest& other543) {
+  min = other543.min;
+  max = other543.max;
   return *this;
 }
 void HeartbeatTxnRangeRequest::printTo(std::ostream& out) const {
@@ -12632,15 +12696,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->aborted.clear();
-            uint32_t _size543;
-            ::apache::thrift::protocol::TType _etype546;
-            xfer += iprot->readSetBegin(_etype546, _size543);
-            uint32_t _i547;
-            for (_i547 = 0; _i547 < _size543; ++_i547)
+            uint32_t _size544;
+            ::apache::thrift::protocol::TType _etype547;
+            xfer += iprot->readSetBegin(_etype547, _size544);
+            uint32_t _i548;
+            for (_i548 = 0; _i548 < _size544; ++_i548)
             {
-              int64_t _elem548;
-              xfer += iprot->readI64(_elem548);
-              this->aborted.insert(_elem548);
+              int64_t _elem549;
+              xfer += iprot->readI64(_elem549);
+              this->aborted.insert(_elem549);
             }
             xfer += iprot->readSetEnd();
           }
@@ -12653,15 +12717,15 @@ uint32_t HeartbeatTxnRangeResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_SET) {
           {
             this->nosuch.clear();
-            uint32_t _size549;
-            ::apache::thrift::protocol::TType _etype552;
-            xfer += iprot->readSetBegin(_etype552, _size549);
-            uint32_t _i553;
-            for (_i553 = 0; _i553 < _size549; ++_i553)
+            uint32_t _size550;
+            ::apache::thrift::protocol::TType _etype553;
+            xfer += iprot->readSetBegin(_etype553, _size550);
+            uint32_t _i554;
+            for (_i554 = 0; _i554 < _size550; ++_i554)
             {
-              int64_t _elem554;
-              xfer += iprot->readI64(_elem554);
-              this->nosuch.insert(_elem554);
+              int64_t _elem555;
+              xfer += iprot->readI64(_elem555);
+              this->nosuch.insert(_elem555);
             }
             xfer += iprot->readSetEnd();
           }
@@ -12694,10 +12758,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("aborted", ::apache::thrift::protocol::T_SET, 1);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->aborted.size()));
-    std::set<int64_t> ::const_iterator _iter555;
-    for (_iter555 = this->aborted.begin(); _iter555 != this->aborted.end(); ++_iter555)
+    std::set<int64_t> ::const_iterator _iter556;
+    for (_iter556 = this->aborted.begin(); _iter556 != this->aborted.end(); ++_iter556)
     {
-      xfer += oprot->writeI64((*_iter555));
+      xfer += oprot->writeI64((*_iter556));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -12706,10 +12770,10 @@ uint32_t HeartbeatTxnRangeResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("nosuch", ::apache::thrift::protocol::T_SET, 2);
   {
     xfer += oprot->writeSetBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->nosuch.size()));
-    std::set<int64_t> ::const_iterator _iter556;
-    for (_iter556 = this->nosuch.begin(); _iter556 != this->nosuch.end(); ++_iter556)
+    std::set<int64_t> ::const_iterator _iter557;
+    for (_iter557 = this->nosuch.begin(); _iter557 != this->nosuch.end(); ++_iter557)
     {
-      xfer += oprot->writeI64((*_iter556));
+      xfer += oprot->writeI64((*_iter557));
     }
     xfer += oprot->writeSetEnd();
   }
@@ -12726,13 +12790,13 @@ void swap(HeartbeatTxnRangeResponse &a, HeartbeatTxnRangeResponse &b) {
   swap(a.nosuch, b.nosuch);
 }
 
-HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other557) {
-  aborted = other557.aborted;
-  nosuch = other557.nosuch;
-}
-HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other558) {
+HeartbeatTxnRangeResponse::HeartbeatTxnRangeResponse(const HeartbeatTxnRangeResponse& other558) {
   aborted = other558.aborted;
   nosuch = other558.nosuch;
+}
+HeartbeatTxnRangeResponse& HeartbeatTxnRangeResponse::operator=(const HeartbeatTxnRangeResponse& other559) {
+  aborted = other559.aborted;
+  nosuch = other559.nosuch;
   return *this;
 }
 void HeartbeatTxnRangeResponse::printTo(std::ostream& out) const {
@@ -12820,9 +12884,9 @@ uint32_t CompactionRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast559;
-          xfer += iprot->readI32(ecast559);
-          this->type = (CompactionType::type)ecast559;
+          int32_t ecast560;
+          xfer += iprot->readI32(ecast560);
+          this->type = (CompactionType::type)ecast560;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -12896,21 +12960,21 @@ void swap(CompactionRequest &a, CompactionRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-CompactionRequest::CompactionRequest(const CompactionRequest& other560) {
-  dbname = other560.dbname;
-  tablename = other560.tablename;
-  partitionname = other560.partitionname;
-  type = other560.type;
-  runas = other560.runas;
-  __isset = other560.__isset;
-}
-CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other561) {
+CompactionRequest::CompactionRequest(const CompactionRequest& other561) {
   dbname = other561.dbname;
   tablename = other561.tablename;
   partitionname = other561.partitionname;
   type = other561.type;
   runas = other561.runas;
   __isset = other561.__isset;
+}
+CompactionRequest& CompactionRequest::operator=(const CompactionRequest& other562) {
+  dbname = other562.dbname;
+  tablename = other562.tablename;
+  partitionname = other562.partitionname;
+  type = other562.type;
+  runas = other562.runas;
+  __isset = other562.__isset;
   return *this;
 }
 void CompactionRequest::printTo(std::ostream& out) const {
@@ -12973,11 +13037,11 @@ void swap(ShowCompactRequest &a, ShowCompactRequest &b) {
   (void) b;
 }
 
-ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other562) {
-  (void) other562;
-}
-ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other563) {
+ShowCompactRequest::ShowCompactRequest(const ShowCompactRequest& other563) {
   (void) other563;
+}
+ShowCompactRequest& ShowCompactRequest::operator=(const ShowCompactRequest& other564) {
+  (void) other564;
   return *this;
 }
 void ShowCompactRequest::printTo(std::ostream& out) const {
@@ -13098,9 +13162,9 @@ uint32_t ShowCompactResponseElement::read(::apache::thrift::protocol::TProtocol*
         break;
       case 4:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast564;
-          xfer += iprot->readI32(ecast564);
-          this->type = (CompactionType::type)ecast564;
+          int32_t ecast565;
+          xfer += iprot->readI32(ecast565);
+          this->type = (CompactionType::type)ecast565;
           isset_type = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -13273,22 +13337,7 @@ void swap(ShowCompactResponseElement &a, ShowCompactResponseElement &b) {
   swap(a.__isset, b.__isset);
 }
 
-ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other565) {
-  dbname = other565.dbname;
-  tablename = other565.tablename;
-  partitionname = other565.partitionname;
-  type = other565.type;
-  state = other565.state;
-  workerid = other565.workerid;
-  start = other565.start;
-  runAs = other565.runAs;
-  hightestTxnId = other565.hightestTxnId;
-  metaInfo = other565.metaInfo;
-  endTime = other565.endTime;
-  hadoopJobId = other565.hadoopJobId;
-  __isset = other565.__isset;
-}
-ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other566) {
+ShowCompactResponseElement::ShowCompactResponseElement(const ShowCompactResponseElement& other566) {
   dbname = other566.dbname;
   tablename = other566.tablename;
   partitionname = other566.partitionname;
@@ -13302,6 +13351,21 @@ ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowComp
   endTime = other566.endTime;
   hadoopJobId = other566.hadoopJobId;
   __isset = other566.__isset;
+}
+ShowCompactResponseElement& ShowCompactResponseElement::operator=(const ShowCompactResponseElement& other567) {
+  dbname = other567.dbname;
+  tablename = other567.tablename;
+  partitionname = other567.partitionname;
+  type = other567.type;
+  state = other567.state;
+  workerid = other567.workerid;
+  start = other567.start;
+  runAs = other567.runAs;
+  hightestTxnId = other567.hightestTxnId;
+  metaInfo = other567.metaInfo;
+  endTime = other567.endTime;
+  hadoopJobId = other567.hadoopJobId;
+  __isset = other567.__isset;
   return *this;
 }
 void ShowCompactResponseElement::printTo(std::ostream& out) const {
@@ -13357,14 +13421,14 @@ uint32_t ShowCompactResponse::read(::apache::thrift::protocol::TProtocol* iprot)
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->compacts.clear();
-            uint32_t _size567;
-            ::apache::thrift::protocol::TType _etype570;
-            xfer += iprot->readListBegin(_etype570, _size567);
-            this->compacts.resize(_size567);
-            uint32_t _i571;
-            for (_i571 = 0; _i571 < _size567; ++_i571)
+            uint32_t _size568;
+            ::apache::thrift::protocol::TType _etype571;
+            xfer += iprot->readListBegin(_etype571, _size568);
+            this->compacts.resize(_size568);
+            uint32_t _i572;
+            for (_i572 = 0; _i572 < _size568; ++_i572)
             {
-              xfer += this->compacts[_i571].read(iprot);
+              xfer += this->compacts[_i572].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13395,10 +13459,10 @@ uint32_t ShowCompactResponse::write(::apache::thrift::protocol::TProtocol* oprot
   xfer += oprot->writeFieldBegin("compacts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->compacts.size()));
-    std::vector<ShowCompactResponseElement> ::const_iterator _iter572;
-    for (_iter572 = this->compacts.begin(); _iter572 != this->compacts.end(); ++_iter572)
+    std::vector<ShowCompactResponseElement> ::const_iterator _iter573;
+    for (_iter573 = this->compacts.begin(); _iter573 != this->compacts.end(); ++_iter573)
     {
-      xfer += (*_iter572).write(oprot);
+      xfer += (*_iter573).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -13414,11 +13478,11 @@ void swap(ShowCompactResponse &a, ShowCompactResponse &b) {
   swap(a.compacts, b.compacts);
 }
 
-ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other573) {
-  compacts = other573.compacts;
-}
-ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other574) {
+ShowCompactResponse::ShowCompactResponse(const ShowCompactResponse& other574) {
   compacts = other574.compacts;
+}
+ShowCompactResponse& ShowCompactResponse::operator=(const ShowCompactResponse& other575) {
+  compacts = other575.compacts;
   return *this;
 }
 void ShowCompactResponse::printTo(std::ostream& out) const {
@@ -13449,6 +13513,11 @@ void AddDynamicPartitions::__set_partitionnames(const std::vector<std::string> &
   this->partitionnames = val;
 }
 
+void AddDynamicPartitions::__set_operationType(const DataOperationType::type val) {
+  this->operationType = val;
+__isset.operationType = true;
+}
+
 uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -13502,14 +13571,14 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionnames.clear();
-            uint32_t _size575;
-            ::apache::thrift::protocol::TType _etype578;
-            xfer += iprot->readListBegin(_etype578, _size575);
-            this->partitionnames.resize(_size575);
-            uint32_t _i579;
-            for (_i579 = 0; _i579 < _size575; ++_i579)
+            uint32_t _size576;
+            ::apache::thrift::protocol::TType _etype579;
+            xfer += iprot->readListBegin(_etype579, _size576);
+            this->partitionnames.resize(_size576);
+            uint32_t _i580;
+            for (_i580 = 0; _i580 < _size576; ++_i580)
             {
-              xfer += iprot->readString(this->partitionnames[_i579]);
+              xfer += iprot->readString(this->partitionnames[_i580]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13518,6 +13587,16 @@ uint32_t AddDynamicPartitions::read(::apache::thrift::protocol::TProtocol* iprot
           xfer += iprot->skip(ftype);
         }
         break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          int32_t ecast581;
+          xfer += iprot->readI32(ecast581);
+          this->operationType = (DataOperationType::type)ecast581;
+          this->__isset.operationType = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -13558,15 +13637,20 @@ uint32_t AddDynamicPartitions::write(::apache::thrift::protocol::TProtocol* opro
   xfer += oprot->writeFieldBegin("partitionnames", ::apache::thrift::protocol::T_LIST, 4);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionnames.size()));
-    std::vector<std::string> ::const_iterator _iter580;
-    for (_iter580 = this->partitionnames.begin(); _iter580 != this->partitionnames.end(); ++_iter580)
+    std::vector<std::string> ::const_iterator _iter582;
+    for (_iter582 = this->partitionnames.begin(); _iter582 != this->partitionnames.end(); ++_iter582)
     {
-      xfer += oprot->writeString((*_iter580));
+      xfer += oprot->writeString((*_iter582));
     }
     xfer += oprot->writeListEnd();
   }
   xfer += oprot->writeFieldEnd();
 
+  if (this->__isset.operationType) {
+    xfer += oprot->writeFieldBegin("operationType", ::apache::thrift::protocol::T_I32, 5);
+    xfer += oprot->writeI32((int32_t)this->operationType);
+    xfer += oprot->writeFieldEnd();
+  }
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -13578,19 +13662,25 @@ void swap(AddDynamicPartitions &a, AddDynamicPartitions &b) {
   swap(a.dbname, b.dbname);
   swap(a.tablename, b.tablename);
   swap(a.partitionnames, b.partitionnames);
+  swap(a.operationType, b.operationType);
+  swap(a.__isset, b.__isset);
 }
 
-AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other581) {
-  txnid = other581.txnid;
-  dbname = other581.dbname;
-  tablename = other581.tablename;
-  partitionnames = other581.partitionnames;
+AddDynamicPartitions::AddDynamicPartitions(const AddDynamicPartitions& other583) {
+  txnid = other583.txnid;
+  dbname = other583.dbname;
+  tablename = other583.tablename;
+  partitionnames = other583.partitionnames;
+  operationType = other583.operationType;
+  __isset = other583.__isset;
 }
-AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other582) {
-  txnid = other582.txnid;
-  dbname = other582.dbname;
-  tablename = other582.tablename;
-  partitionnames = other582.partitionnames;
+AddDynamicPartitions& AddDynamicPartitions::operator=(const AddDynamicPartitions& other584) {
+  txnid = other584.txnid;
+  dbname = other584.dbname;
+  tablename = other584.tablename;
+  partitionnames = other584.partitionnames;
+  operationType = other584.operationType;
+  __isset = other584.__isset;
   return *this;
 }
 void AddDynamicPartitions::printTo(std::ostream& out) const {
@@ -13600,6 +13690,7 @@ void AddDynamicPartitions::printTo(std::ostream& out) const {
   out << ", " << "dbname=" << to_string(dbname);
   out << ", " << "tablename=" << to_string(tablename);
   out << ", " << "partitionnames=" << to_string(partitionnames);
+  out << ", " << "operationType="; (__isset.operationType ? (out << to_string(operationType)) : (out << "<null>"));
   out << ")";
 }
 
@@ -13695,15 +13786,15 @@ void swap(NotificationEventRequest &a, NotificationEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other583) {
-  lastEvent = other583.lastEvent;
-  maxEvents = other583.maxEvents;
-  __isset = other583.__isset;
+NotificationEventRequest::NotificationEventRequest(const NotificationEventRequest& other585) {
+  lastEvent = other585.lastEvent;
+  maxEvents = other585.maxEvents;
+  __isset = other585.__isset;
 }
-NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other584) {
-  lastEvent = other584.lastEvent;
-  maxEvents = other584.maxEvents;
-  __isset = other584.__isset;
+NotificationEventRequest& NotificationEventRequest::operator=(const NotificationEventRequest& other586) {
+  lastEvent = other586.lastEvent;
+  maxEvents = other586.maxEvents;
+  __isset = other586.__isset;
   return *this;
 }
 void NotificationEventRequest::printTo(std::ostream& out) const {
@@ -13885,23 +13976,23 @@ void swap(NotificationEvent &a, NotificationEvent &b) {
   swap(a.__isset, b.__isset);
 }
 
-NotificationEvent::NotificationEvent(const NotificationEvent& other585) {
-  eventId = other585.eventId;
-  eventTime = other585.eventTime;
-  eventType = other585.eventType;
-  dbName = other585.dbName;
-  tableName = other585.tableName;
-  message = other585.message;
-  __isset = other585.__isset;
-}
-NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other586) {
-  eventId = other586.eventId;
-  eventTime = other586.eventTime;
-  eventType = other586.eventType;
-  dbName = other586.dbName;
-  tableName = other586.tableName;
-  message = other586.message;
-  __isset = other586.__isset;
+NotificationEvent::NotificationEvent(const NotificationEvent& other587) {
+  eventId = other587.eventId;
+  eventTime = other587.eventTime;
+  eventType = other587.eventType;
+  dbName = other587.dbName;
+  tableName = other587.tableName;
+  message = other587.message;
+  __isset = other587.__isset;
+}
+NotificationEvent& NotificationEvent::operator=(const NotificationEvent& other588) {
+  eventId = other588.eventId;
+  eventTime = other588.eventTime;
+  eventType = other588.eventType;
+  dbName = other588.dbName;
+  tableName = other588.tableName;
+  message = other588.message;
+  __isset = other588.__isset;
   return *this;
 }
 void NotificationEvent::printTo(std::ostream& out) const {
@@ -13951,14 +14042,14 @@ uint32_t NotificationEventResponse::read(::apache::thrift::protocol::TProtocol*
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->events.clear();
-            uint32_t _size587;
-            ::apache::thrift::protocol::TType _etype590;
-            xfer += iprot->readListBegin(_etype590, _size587);
-            this->events.resize(_size587);
-            uint32_t _i591;
-            for (_i591 = 0; _i591 < _size587; ++_i591)
+            uint32_t _size589;
+            ::apache::thrift::protocol::TType _etype592;
+            xfer += iprot->readListBegin(_etype592, _size589);
+            this->events.resize(_size589);
+            uint32_t _i593;
+            for (_i593 = 0; _i593 < _size589; ++_i593)
             {
-              xfer += this->events[_i591].read(iprot);
+              xfer += this->events[_i593].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13989,10 +14080,10 @@ uint32_t NotificationEventResponse::write(::apache::thrift::protocol::TProtocol*
   xfer += oprot->writeFieldBegin("events", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->events.size()));
-    std::vector<NotificationEvent> ::const_iterator _iter592;
-    for (_iter592 = this->events.begin(); _iter592 != this->events.end(); ++_iter592)
+    std::vector<NotificationEvent> ::const_iterator _iter594;
+    for (_iter594 = this->events.begin(); _iter594 != this->events.end(); ++_iter594)
     {
-      xfer += (*_iter592).write(oprot);
+      xfer += (*_iter594).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -14008,11 +14099,11 @@ void swap(NotificationEventResponse &a, NotificationEventResponse &b) {
   swap(a.events, b.events);
 }
 
-NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other593) {
-  events = other593.events;
+NotificationEventResponse::NotificationEventResponse(const NotificationEventResponse& other595) {
+  events = other595.events;
 }
-NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other594) {
-  events = other594.events;
+NotificationEventResponse& NotificationEventResponse::operator=(const NotificationEventResponse& other596) {
+  events = other596.events;
   return *this;
 }
 void NotificationEventResponse::printTo(std::ostream& out) const {
@@ -14094,11 +14185,11 @@ void swap(CurrentNotificationEventId &a, CurrentNotificationEventId &b) {
   swap(a.eventId, b.eventId);
 }
 
-CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other595) {
-  eventId = other595.eventId;
+CurrentNotificationEventId::CurrentNotificationEventId(const CurrentNotificationEventId& other597) {
+  eventId = other597.eventId;
 }
-CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other596) {
-  eventId = other596.eventId;
+CurrentNotificationEventId& CurrentNotificationEventId::operator=(const CurrentNotificationEventId& other598) {
+  eventId = other598.eventId;
   return *this;
 }
 void CurrentNotificationEventId::printTo(std::ostream& out) const {
@@ -14143,14 +14234,14 @@ uint32_t InsertEventRequestData::read(::apache::thrift::protocol::TProtocol* ipr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->filesAdded.clear();
-            uint32_t _size597;
-            ::apache::thrift::protocol::TType _etype600;
-            xfer += iprot->readListBegin(_etype600, _size597);
-            this->filesAdded.resize(_size597);
-            uint32_t _i601;
-            for (_i601 = 0; _i601 < _size597; ++_i601)
+            uint32_t _size599;
+            ::apache::thrift::protocol::TType _etype602;
+            xfer += iprot->readListBegin(_etype602, _size599);
+            this->filesAdded.resize(_size599);
+            uint32_t _i603;
+            for (_i603 = 0; _i603 < _size599; ++_i603)
             {
-              xfer += iprot->readString(this->filesAdded[_i601]);
+              xfer += iprot->readString(this->filesAdded[_i603]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14181,10 +14272,10 @@ uint32_t InsertEventRequestData::write(::apache::thrift::protocol::TProtocol* op
   xfer += oprot->writeFieldBegin("filesAdded", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->filesAdded.size()));
-    std::vector<std::string> ::const_iterator _iter602;
-    for (_iter602 = this->filesAdded.begin(); _iter602 != this->filesAdded.end(); ++_iter602)
+    std::vector<std::string> ::const_iterator _iter604;
+    for (_iter604 = this->filesAdded.begin(); _iter604 != this->filesAdded.end(); ++_iter604)
     {
-      xfer += oprot->writeString((*_iter602));
+      xfer += oprot->writeString((*_iter604));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14200,11 +14291,11 @@ void swap(InsertEventRequestData &a, InsertEventRequestData &b) {
   swap(a.filesAdded, b.filesAdded);
 }
 
-InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other603) {
-  filesAdded = other603.filesAdded;
+InsertEventRequestData::InsertEventRequestData(const InsertEventRequestData& other605) {
+  filesAdded = other605.filesAdded;
 }
-InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other604) {
-  filesAdded = other604.filesAdded;
+InsertEventRequestData& InsertEventRequestData::operator=(const InsertEventRequestData& other606) {
+  filesAdded = other606.filesAdded;
   return *this;
 }
 void InsertEventRequestData::printTo(std::ostream& out) const {
@@ -14284,13 +14375,13 @@ void swap(FireEventRequestData &a, FireEventRequestData &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequestData::FireEventRequestData(const FireEventRequestData& other605) {
-  insertData = other605.insertData;
-  __isset = other605.__isset;
+FireEventRequestData::FireEventRequestData(const FireEventRequestData& other607) {
+  insertData = other607.insertData;
+  __isset = other607.__isset;
 }
-FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other606) {
-  insertData = other606.insertData;
-  __isset = other606.__isset;
+FireEventRequestData& FireEventRequestData::operator=(const FireEventRequestData& other608) {
+  insertData = other608.insertData;
+  __isset = other608.__isset;
   return *this;
 }
 void FireEventRequestData::printTo(std::ostream& out) const {
@@ -14387,14 +14478,14 @@ uint32_t FireEventRequest::read(::apache::thrift::protocol::TProtocol* iprot) {
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->partitionVals.clear();
-            uint32_t _size607;
-            ::apache::thrift::protocol::TType _etype610;
-            xfer += iprot->readListBegin(_etype610, _size607);
-            this->partitionVals.resize(_size607);
-            uint32_t _i611;
-            for (_i611 = 0; _i611 < _size607; ++_i611)
+            uint32_t _size609;
+            ::apache::thrift::protocol::TType _etype612;
+            xfer += iprot->readListBegin(_etype612, _size609);
+            this->partitionVals.resize(_size609);
+            uint32_t _i613;
+            for (_i613 = 0; _i613 < _size609; ++_i613)
             {
-              xfer += iprot->readString(this->partitionVals[_i611]);
+              xfer += iprot->readString(this->partitionVals[_i613]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14446,10 +14537,10 @@ uint32_t FireEventRequest::write(::apache::thrift::protocol::TProtocol* oprot) c
     xfer += oprot->writeFieldBegin("partitionVals", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionVals.size()));
-      std::vector<std::string> ::const_iterator _iter612;
-      for (_iter612 = this->partitionVals.begin(); _iter612 != this->partitionVals.end(); ++_iter612)
+      std::vector<std::string> ::const_iterator _iter614;
+      for (_iter614 = this->partitionVals.begin(); _iter614 != this->partitionVals.end(); ++_iter614)
       {
-        xfer += oprot->writeString((*_iter612));
+        xfer += oprot->writeString((*_iter614));
       }
       xfer += oprot->writeListEnd();
     }
@@ -14470,21 +14561,21 @@ void swap(FireEventRequest &a, FireEventRequest &b) {
   swap(a.__isset, b.__isset);
 }
 
-FireEventRequest::FireEventRequest(const FireEventRequest& other613) {
-  successful = other613.successful;
-  data = other613.data;
-  dbName = other613.dbName;
-  tableName = other613.tableName;
-  partitionVals = other613.partitionVals;
-  __isset = other613.__isset;
-}
-FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other614) {
-  successful = other614.successful;
-  data = other614.data;
-  dbName = other614.dbName;
-  tableName = other614.tableName;
-  partitionVals = other614.partitionVals;
-  __isset = other614.__isset;
+FireEventRequest::FireEventRequest(const FireEventRequest& other615) {
+  successful = other615.successful;
+  data = other615.data;
+  dbName = other615.dbName;
+  tableName = other615.tableName;
+  partitionVals = other615.partitionVals;
+  __isset = other615.__isset;
+}
+FireEventRequest& FireEventRequest::operator=(const FireEventRequest& other616) {
+  successful = other616.successful;
+  data = other616.data;
+  dbName = other616.dbName;
+  tableName = other616.tableName;
+  partitionVals = other616.partitionVals;
+  __isset = other616.__isset;
   return *this;
 }
 void FireEventRequest::printTo(std::ostream& out) const {
@@ -14547,11 +14638,11 @@ void swap(FireEventResponse &a, FireEventResponse &b) {
   (void) b;
 }
 
-FireEventResponse::FireEventResponse(const FireEventResponse& other615) {
-  (void) other615;
+FireEventResponse::FireEventResponse(const FireEventResponse& other617) {
+  (void) other617;
 }
-FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other616) {
-  (void) other616;
+FireEventResponse& FireEventResponse::operator=(const FireEventResponse& other618) {
+  (void) other618;
   return *this;
 }
 void FireEventResponse::printTo(std::ostream& out) const {
@@ -14595,14 +14686,14 @@ uint32_t GetAllFunctionsResponse::read(::apache::thrift::protocol::TProtocol* ip
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->functions.clear();
-            uint32_t _size617;
-            ::apache::thrift::protocol::TType _etype620;
-            xfer += iprot->readListBegin(_etype620, _size617);
-            this->functions.resize(_size617);
-            uint32_t _i621;
-            for (_i621 = 0; _i621 < _size617; ++_i621)
+            uint32_t _size619;
+            ::apache::thrift::protocol::TType _etype622;
+            xfer += iprot->readListBegin(_etype622, _size619);
+            this->functions.resize(_size619);
+            uint32_t _i623;
+            for (_i623 = 0; _i623 < _size619; ++_i623)
             {
-              xfer += this->functions[_i621].read(iprot);
+              xfer += this->functions[_i623].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14632,10 +14723,10 @@ uint32_t GetAllFunctionsResponse::write(::apache::thrift::protocol::TProtocol* o
     xfer += oprot->writeFieldBegin("functions", ::apache::thrift::protocol::T_LIST, 1);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->functions.size()));
-      std::vector<Function> ::const_iterator _iter622;
-      for (_iter622 = this->functions.begin(); _iter622 != this->functions.end(); ++_iter622)
+      std::vector<Function> ::const_iterator _iter624;
+      for (_iter624 = this->functions.begin(); _iter624 != this->functions.end(); ++_iter624)
       {
-        xfer += (*_iter622).write(oprot);
+        xfer += (*_iter624).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14652,13 +14743,13 @@ void swap(GetAllFunctionsResponse &a, GetAllFunctionsResponse &b) {
   swap(a.__isset, b.__isset);
 }
 
-GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other623) {
-  functions = other623.functions;
-  __isset = other623.__isset;
+GetAllFunctionsResponse::GetAllFunctionsResponse(const GetAllFunctionsResponse& other625) {
+  functions = other625.functions;
+  __isset = other625.__isset;
 }
-GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other624) {
-  functions = other624.functions;
-  __isset = other624.__isset;
+GetAllFunctionsResponse& GetAllFunctionsResponse::operator=(const GetAllFunctionsResponse& other626) {
+  functions = other626.functions;
+  __isset = other626.__isset;
   return *this;
 }
 void GetAllFunctionsResponse::printTo(std::ostream& out) const {
@@ -14738,13 +14829,13 @@ void swap(MetaException &a, MetaException &b) {
   swap(a.__isset, b.__isset);
 }
 
-MetaException::MetaException(const MetaException& other625) : TException() {
-  message = other625.message;
-  __isset = other625.__isset;
+MetaException::MetaException(const MetaException& other627) : TException() {
+  message = other627.message;
+  __isset = other627.__isset;
 }
-MetaException& MetaException::operator=(const MetaException& other626) {
-  message = other626.message;
-  __isset = other626.__isset;
+MetaException& MetaException::operator=(const MetaException& other628) {
+  message = other628.message;
+  __isset = other628.__isset;
   return *this;
 }
 void MetaException::printTo(std::ostream& out) const {
@@ -14835,13 +14926,13 @@ void swap(UnknownTableException &a, UnknownTableException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownTableException::UnknownTableException(const UnknownTableException& other627) : TException() {
-  message = other627.message;
-  __isset = other627.__isset;
+UnknownTableException::UnknownTableException(const UnknownTableException& other629) : TException() {
+  message = other629.message;
+  __isset = other629.__isset;
 }
-UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other628) {
-  message = other628.message;
-  __isset = other628.__isset;
+UnknownTableException& UnknownTableException::operator=(const UnknownTableException& other630) {
+  message = other630.message;
+  __isset = other630.__isset;
   return *this;
 }
 void UnknownTableException::printTo(std::ostream& out) const {
@@ -14932,13 +15023,13 @@ void swap(UnknownDBException &a, UnknownDBException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownDBException::UnknownDBException(const UnknownDBException& other629) : TException() {
-  message = other629.message;
-  __isset = other629.__isset;
+UnknownDBException::UnknownDBException(const UnknownDBException& other631) : TException() {
+  message = other631.message;
+  __isset = other631.__isset;
 }
-UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other630) {
-  message = other630.message;
-  __isset = other630.__isset;
+UnknownDBException& UnknownDBException::operator=(const UnknownDBException& other632) {
+  message = other632.message;
+  __isset = other632.__isset;
   return *this;
 }
 void UnknownDBException::printTo(std::ostream& out) const {
@@ -15029,13 +15120,13 @@ void swap(AlreadyExistsException &a, AlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other631) : TException() {
-  message = other631.message;
-  __isset = other631.__isset;
+AlreadyExistsException::AlreadyExistsException(const AlreadyExistsException& other633) : TException() {
+  message = other633.message;
+  __isset = other633.__isset;
 }
-AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other632) {
-  message = other632.message;
-  __isset = other632.__isset;
+AlreadyExistsException& AlreadyExistsException::operator=(const AlreadyExistsException& other634) {
+  message = other634.message;
+  __isset = other634.__isset;
   return *this;
 }
 void AlreadyExistsException::printTo(std::ostream& out) const {
@@ -15126,13 +15217,13 @@ void swap(InvalidPartitionException &a, InvalidPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other633) : TException() {
-  message = other633.message;
-  __isset = other633.__isset;
+InvalidPartitionException::InvalidPartitionException(const InvalidPartitionException& other635) : TException() {
+  message = other635.message;
+  __isset = other635.__isset;
 }
-InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other634) {
-  message = other634.message;
-  __isset = other634.__isset;
+InvalidPartitionException& InvalidPartitionException::operator=(const InvalidPartitionException& other636) {
+  message = other636.message;
+  __isset = other636.__isset;
   return *this;
 }
 void InvalidPartitionException::printTo(std::ostream& out) const {
@@ -15223,13 +15314,13 @@ void swap(UnknownPartitionException &a, UnknownPartitionException &b) {
   swap(a.__isset, b.__isset);
 }
 
-UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other635) : TException() {
-  message = other635.message;
-  __isset = other635.__isset;
+UnknownPartitionException::UnknownPartitionException(const UnknownPartitionException& other637) : TException() {
+  message = other637.message;
+  __isset = other637.__isset;
 }
-UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other636) {
-  message = other636.message;
-  __isset = other636.__isset;
+UnknownPartitionException& UnknownPartitionException::operator=(const UnknownPartitionException& other638) {
+  message = other638.message;
+  __isset = other638.__isset;
   return *this;
 }
 void UnknownPartitionException::printTo(std::ostream& out) const {
@@ -15320,13 +15411,13 @@ void swap(InvalidObjectException &a, InvalidObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidObjectException::InvalidObjectException(const InvalidObjectException& other637) : TException() {
-  message = other637.message;
-  __isset = other637.__isset;
+InvalidObjectException::InvalidObjectException(const InvalidObjectException& other639) : TException() {
+  message = other639.message;
+  __isset = other639.__isset;
 }
-InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other638) {
-  message = other638.message;
-  __isset = other638.__isset;
+InvalidObjectException& InvalidObjectException::operator=(const InvalidObjectException& other640) {
+  message = other640.message;
+  __isset = other640.__isset;
   return *this;
 }
 void InvalidObjectException::printTo(std::ostream& out) const {
@@ -15417,13 +15508,13 @@ void swap(NoSuchObjectException &a, NoSuchObjectException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other639) : TException() {
-  message = other639.message;
-  __isset = other639.__isset;
+NoSuchObjectException::NoSuchObjectException(const NoSuchObjectException& other641) : TException() {
+  message = other641.message;
+  __isset = other641.__isset;
 }
-NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other640) {
-  message = other640.message;
-  __isset = other640.__isset;
+NoSuchObjectException& NoSuchObjectException::operator=(const NoSuchObjectException& other642) {
+  message = other642.message;
+  __isset = other642.__isset;
   return *this;
 }
 void NoSuchObjectException::printTo(std::ostream& out) const {
@@ -15514,13 +15605,13 @@ void swap(IndexAlreadyExistsException &a, IndexAlreadyExistsException &b) {
   swap(a.__isset, b.__isset);
 }
 
-IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other641) : TException() {
-  message = other641.message;
-  __isset = other641.__isset;
+IndexAlreadyExistsException::IndexAlreadyExistsException(const IndexAlreadyExistsException& other643) : TException() {
+  message = other643.message;
+  __isset = other643.__isset;
 }
-IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other642) {
-  message = other642.message;
-  __isset = other642.__isset;
+IndexAlreadyExistsException& IndexAlreadyExistsException::operator=(const IndexAlreadyExistsException& other644) {
+  message = other644.message;
+  __isset = other644.__isset;
   return *this;
 }
 void IndexAlreadyExistsException::printTo(std::ostream& out) const {
@@ -15611,13 +15702,13 @@ void swap(InvalidOperationException &a, InvalidOperationException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidOperationException::InvalidOperationException(const InvalidOperationException& other643) : TException() {
-  message = other643.message;
-  __isset = other643.__isset;
+InvalidOperationException::InvalidOperationException(const InvalidOperationException& other645) : TException() {
+  message = other645.message;
+  __isset = other645.__isset;
 }
-InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other644) {
-  message = other644.message;
-  __isset = other644.__isset;
+InvalidOperationException& InvalidOperationException::operator=(const InvalidOperationException& other646) {
+  message = other646.message;
+  __isset = other646.__isset;
   return *this;
 }
 void InvalidOperationException::printTo(std::ostream& out) const {
@@ -15708,13 +15799,13 @@ void swap(ConfigValSecurityException &a, ConfigValSecurityException &b) {
   swap(a.__isset, b.__isset);
 }
 
-ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other645) : TException() {
-  message = other645.message;
-  __isset = other645.__isset;
+ConfigValSecurityException::ConfigValSecurityException(const ConfigValSecurityException& other647) : TException() {
+  message = other647.message;
+  __isset = other647.__isset;
 }
-ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other646) {
-  message = other646.message;
-  __isset = other646.__isset;
+ConfigValSecurityException& ConfigValSecurityException::operator=(const ConfigValSecurityException& other648) {
+  message = other648.message;
+  __isset = other648.__isset;
   return *this;
 }
 void ConfigValSecurityException::printTo(std::ostream& out) const {
@@ -15805,13 +15896,13 @@ void swap(InvalidInputException &a, InvalidInputException &b) {
   swap(a.__isset, b.__isset);
 }
 
-InvalidInputException::InvalidInputException(const InvalidInputException& other647) : TException() {
-  message = other647.message;
-  __isset = other647.__isset;
+InvalidInputException::InvalidInputException(const InvalidInputException& other649) : TException() {
+  message = other649.message;
+  __isset = other649.__isset;
 }
-InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other648) {
-  message = other648.message;
-  __isset = other648.__isset;
+InvalidInputException& InvalidInputException::operator=(const InvalidInputException& other650) {
+  message = other650.message;
+  __isset = other650.__isset;
   return *this;
 }
 void InvalidInputException::printTo(std::ostream& out) const {
@@ -15902,13 +15993,13 @@ void swap(NoSuchTxnException &a, NoSuchTxnException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other649) : TException() {
-  message = other649.message;
-  __isset = other649.__isset;
+NoSuchTxnException::NoSuchTxnException(const NoSuchTxnException& other651) : TException() {
+  message = other651.message;
+  __isset = other651.__isset;
 }
-NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other650) {
-  message = other650.message;
-  __isset = other650.__isset;
+NoSuchTxnException& NoSuchTxnException::operator=(const NoSuchTxnException& other652) {
+  message = other652.message;
+  __isset = other652.__isset;
   return *this;
 }
 void NoSuchTxnException::printTo(std::ostream& out) const {
@@ -15999,13 +16090,13 @@ void swap(TxnAbortedException &a, TxnAbortedException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnAbortedException::TxnAbortedException(const TxnAbortedException& other651) : TException() {
-  message = other651.message;
-  __isset = other651.__isset;
+TxnAbortedException::TxnAbortedException(const TxnAbortedException& other653) : TException() {
+  message = other653.message;
+  __isset = other653.__isset;
 }
-TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other652) {
-  message = other652.message;
-  __isset = other652.__isset;
+TxnAbortedException& TxnAbortedException::operator=(const TxnAbortedException& other654) {
+  message = other654.message;
+  __isset = other654.__isset;
   return *this;
 }
 void TxnAbortedException::printTo(std::ostream& out) const {
@@ -16096,13 +16187,13 @@ void swap(TxnOpenException &a, TxnOpenException &b) {
   swap(a.__isset, b.__isset);
 }
 
-TxnOpenException::TxnOpenException(const TxnOpenException& other653) : TException() {
-  message = other653.message;
-  __isset = other653.__isset;
+TxnOpenException::TxnOpenException(const TxnOpenException& other655) : TException() {
+  message = other655.message;
+  __isset = other655.__isset;
 }
-TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other654) {
-  message = other654.message;
-  __isset = other654.__isset;
+TxnOpenException& TxnOpenException::operator=(const TxnOpenException& other656) {
+  message = other656.message;
+  __isset = other656.__isset;
   return *this;
 }
 void TxnOpenException::printTo(std::ostream& out) const {
@@ -16193,13 +16284,13 @@ void swap(NoSuchLockException &a, NoSuchLockException &b) {
   swap(a.__isset, b.__isset);
 }
 
-NoSuchLockException::NoSuchLockException(const NoSuchLockException& other655) : TException() {
-  message = other655.message;
-  __isset = other655.__isset;
+NoSuchLockException::NoSuchLockException(const NoSuchLockException& other657) : TException() {
+  message = other657.message;
+  __isset = other657.__isset;
 }
-NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other656) {
-  message = other656.message;
-  __isset = other656.__isset;
+NoSuchLockException& NoSuchLockException::operator=(const NoSuchLockException& other658) {
+  message = other658.message;
+  __isset = other658.__isset;
   return *this;
 }
 void NoSuchLockException::printTo(std::ostream& out) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index fbe99c1..390db33 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -109,6 +109,19 @@ struct GrantRevokeType {
 
 extern const std::map<int, const char*> _GrantRevokeType_VALUES_TO_NAMES;
 
+struct DataOperationType {
+  enum type {
+    SELECT = 1,
+    INSERT = 2,
+    UPDATE = 3,
+    DELETE = 4,
+    UNSET = 5,
+    NO_TXN = 6
+  };
+};
+
+extern const std::map<int, const char*> _DataOperationType_VALUES_TO_NAMES;
+
 struct EventRequestType {
   enum type {
     INSERT = 1,
@@ -4472,9 +4485,11 @@ inline std::ostream& operator<<(std::ostream& out, const CommitTxnRequest& obj)
 }
 
 typedef struct _LockComponent__isset {
-  _LockComponent__isset() : tablename(false), partitionname(false) {}
+  _LockComponent__isset() : tablename(false), partitionname(false), operationType(true), isAcid(true) {}
   bool tablename :1;
   bool partitionname :1;
+  bool operationType :1;
+  bool isAcid :1;
 } _LockComponent__isset;
 
 class LockComponent {
@@ -4482,7 +4497,9 @@ class LockComponent {
 
   LockComponent(const LockComponent&);
   LockComponent& operator=(const LockComponent&);
-  LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname() {
+  LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname(), operationType((DataOperationType::type)5), isAcid(false) {
+    operationType = (DataOperationType::type)5;
+
   }
 
   virtual ~LockComponent() throw();
@@ -4491,6 +4508,8 @@ class LockComponent {
   std::string dbname;
   std::string tablename;
   std::string partitionname;
+  DataOperationType::type operationType;
+  bool isAcid;
 
   _LockComponent__isset __isset;
 
@@ -4504,6 +4523,10 @@ class LockComponent {
 
   void __set_partitionname(const std::string& val);
 
+  void __set_operationType(const DataOperationType::type val);
+
+  void __set_isAcid(const bool val);
+
   bool operator == (const LockComponent & rhs) const
   {
     if (!(type == rhs.type))
@@ -4520,6 +4543,14 @@ class LockComponent {
       return false;
     else if (__isset.partitionname && !(partitionname == rhs.partitionname))
       return false;
+    if (__isset.operationType != rhs.__isset.operationType)
+      return false;
+    else if (__isset.operationType && !(operationType == rhs.operationType))
+      return false;
+    if (__isset.isAcid != rhs.__isset.isAcid)
+      return false;
+    else if (__isset.isAcid && !(isAcid == rhs.isAcid))
+      return false;
     return true;
   }
   bool operator != (const LockComponent &rhs) const {
@@ -5440,13 +5471,19 @@ inline std::ostream& operator<<(std::ostream& out, const ShowCompactResponse& ob
   return out;
 }
 
+typedef struct _AddDynamicPartitions__isset {
+  _AddDynamicPartitions__isset() : operationType(true) {}
+  bool operationType :1;
+} _AddDynamicPartitions__isset;
 
 class AddDynamicPartitions {
  public:
 
   AddDynamicPartitions(const AddDynamicPartitions&);
   AddDynamicPartitions& operator=(const AddDynamicPartitions&);
-  AddDynamicPartitions() : txnid(0), dbname(), tablename() {
+  AddDynamicPartitions() : txnid(0), dbname(), tablename(), operationType((DataOperationType::type)5) {
+    operationType = (DataOperationType::type)5;
+
   }
 
   virtual ~AddDynamicPartitions() throw();
@@ -5454,6 +5491,9 @@ class AddDynamicPartitions {
   std::string dbname;
   std::string tablename;
   std::vector<std::string>  partitionnames;
+  DataOperationType::type operationType;
+
+  _AddDynamicPartitions__isset __isset;
 
   void __set_txnid(const int64_t val);
 
@@ -5463,6 +5503,8 @@ class AddDynamicPartitions {
 
   void __set_partitionnames(const std::vector<std::string> & val);
 
+  void __set_operationType(const DataOperationType::type val);
+
   bool operator == (const AddDynamicPartitions & rhs) const
   {
     if (!(txnid == rhs.txnid))
@@ -5473,6 +5515,10 @@ class AddDynamicPartitions {
       return false;
     if (!(partitionnames == rhs.partitionnames))
       return false;
+    if (__isset.operationType != rhs.__isset.operationType)
+      return false;
+    else if (__isset.operationType && !(operationType == rhs.operationType))
+      return false;
     return true;
   }
   bool operator != (const AddDynamicPartitions &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
index 45140bc..7543ef4 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/AddDynamicPartitions.java
@@ -42,6 +42,7 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
   private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)2);
   private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField PARTITIONNAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionnames", org.apache.thrift.protocol.TType.LIST, (short)4);
+  private static final org.apache.thrift.protocol.TField OPERATION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationType", org.apache.thrift.protocol.TType.I32, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -53,13 +54,19 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
   private String dbname; // required
   private String tablename; // required
   private List<String> partitionnames; // required
+  private DataOperationType operationType; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     TXNID((short)1, "txnid"),
     DBNAME((short)2, "dbname"),
     TABLENAME((short)3, "tablename"),
-    PARTITIONNAMES((short)4, "partitionnames");
+    PARTITIONNAMES((short)4, "partitionnames"),
+    /**
+     * 
+     * @see DataOperationType
+     */
+    OPERATION_TYPE((short)5, "operationType");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -82,6 +89,8 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           return TABLENAME;
         case 4: // PARTITIONNAMES
           return PARTITIONNAMES;
+        case 5: // OPERATION_TYPE
+          return OPERATION_TYPE;
         default:
           return null;
       }
@@ -124,6 +133,7 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
   // isset id assignments
   private static final int __TXNID_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.OPERATION_TYPE};
   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);
@@ -136,11 +146,15 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
     tmpMap.put(_Fields.PARTITIONNAMES, new org.apache.thrift.meta_data.FieldMetaData("partitionnames", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+    tmpMap.put(_Fields.OPERATION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("operationType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, DataOperationType.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(AddDynamicPartitions.class, metaDataMap);
   }
 
   public AddDynamicPartitions() {
+    this.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.UNSET;
+
   }
 
   public AddDynamicPartitions(
@@ -173,6 +187,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       List<String> __this__partitionnames = new ArrayList<String>(other.partitionnames);
       this.partitionnames = __this__partitionnames;
     }
+    if (other.isSetOperationType()) {
+      this.operationType = other.operationType;
+    }
   }
 
   public AddDynamicPartitions deepCopy() {
@@ -186,6 +203,8 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
     this.dbname = null;
     this.tablename = null;
     this.partitionnames = null;
+    this.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.UNSET;
+
   }
 
   public long getTxnid() {
@@ -294,6 +313,37 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
     }
   }
 
+  /**
+   * 
+   * @see DataOperationType
+   */
+  public DataOperationType getOperationType() {
+    return this.operationType;
+  }
+
+  /**
+   * 
+   * @see DataOperationType
+   */
+  public void setOperationType(DataOperationType operationType) {
+    this.operationType = operationType;
+  }
+
+  public void unsetOperationType() {
+    this.operationType = null;
+  }
+
+  /** Returns true if field operationType is set (has been assigned a value) and false otherwise */
+  public boolean isSetOperationType() {
+    return this.operationType != null;
+  }
+
+  public void setOperationTypeIsSet(boolean value) {
+    if (!value) {
+      this.operationType = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TXNID:
@@ -328,6 +378,14 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       }
       break;
 
+    case OPERATION_TYPE:
+      if (value == null) {
+        unsetOperationType();
+      } else {
+        setOperationType((DataOperationType)value);
+      }
+      break;
+
     }
   }
 
@@ -345,6 +403,9 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
     case PARTITIONNAMES:
       return getPartitionnames();
 
+    case OPERATION_TYPE:
+      return getOperationType();
+
     }
     throw new IllegalStateException();
   }
@@ -364,6 +425,8 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       return isSetTablename();
     case PARTITIONNAMES:
       return isSetPartitionnames();
+    case OPERATION_TYPE:
+      return isSetOperationType();
     }
     throw new IllegalStateException();
   }
@@ -417,6 +480,15 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         return false;
     }
 
+    boolean this_present_operationType = true && this.isSetOperationType();
+    boolean that_present_operationType = true && that.isSetOperationType();
+    if (this_present_operationType || that_present_operationType) {
+      if (!(this_present_operationType && that_present_operationType))
+        return false;
+      if (!this.operationType.equals(that.operationType))
+        return false;
+    }
+
     return true;
   }
 
@@ -444,6 +516,11 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
     if (present_partitionnames)
       list.add(partitionnames);
 
+    boolean present_operationType = true && (isSetOperationType());
+    list.add(present_operationType);
+    if (present_operationType)
+      list.add(operationType.getValue());
+
     return list.hashCode();
   }
 
@@ -495,6 +572,16 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetOperationType()).compareTo(other.isSetOperationType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetOperationType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationType, other.operationType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -542,6 +629,16 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
       sb.append(this.partitionnames);
     }
     first = false;
+    if (isSetOperationType()) {
+      if (!first) sb.append(", ");
+      sb.append("operationType:");
+      if (this.operationType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.operationType);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -645,6 +742,14 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // OPERATION_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.findByValue(iprot.readI32());
+              struct.setOperationTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -683,6 +788,13 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         }
         oprot.writeFieldEnd();
       }
+      if (struct.operationType != null) {
+        if (struct.isSetOperationType()) {
+          oprot.writeFieldBegin(OPERATION_TYPE_FIELD_DESC);
+          oprot.writeI32(struct.operationType.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -710,6 +822,14 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
           oprot.writeString(_iter504);
         }
       }
+      BitSet optionals = new BitSet();
+      if (struct.isSetOperationType()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetOperationType()) {
+        oprot.writeI32(struct.operationType.getValue());
+      }
     }
 
     @Override
@@ -732,6 +852,11 @@ public class AddDynamicPartitions implements org.apache.thrift.TBase<AddDynamicP
         }
       }
       struct.setPartitionnamesIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.findByValue(iprot.readI32());
+        struct.setOperationTypeIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DataOperationType.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DataOperationType.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DataOperationType.java
new file mode 100644
index 0000000..15a6e9a
--- /dev/null
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/DataOperationType.java
@@ -0,0 +1,57 @@
+/**
+ * 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 java.util.Map;
+import java.util.HashMap;
+import org.apache.thrift.TEnum;
+
+public enum DataOperationType implements org.apache.thrift.TEnum {
+  SELECT(1),
+  INSERT(2),
+  UPDATE(3),
+  DELETE(4),
+  UNSET(5),
+  NO_TXN(6);
+
+  private final int value;
+
+  private DataOperationType(int value) {
+    this.value = value;
+  }
+
+  /**
+   * Get the integer value of this enum value, as defined in the Thrift IDL.
+   */
+  public int getValue() {
+    return value;
+  }
+
+  /**
+   * Find a the enum type by its integer value, as defined in the Thrift IDL.
+   * @return null if the value is not found.
+   */
+  public static DataOperationType findByValue(int value) { 
+    switch (value) {
+      case 1:
+        return SELECT;
+      case 2:
+        return INSERT;
+      case 3:
+        return UPDATE;
+      case 4:
+        return DELETE;
+      case 5:
+        return UNSET;
+      case 6:
+        return NO_TXN;
+      default:
+        return null;
+    }
+  }
+}


[5/5] hive git commit: HIVE-13622 WriteSet tracking optimizations (Eugene Koifman, reviewed by Alan Gates)

Posted by ek...@apache.org.
HIVE-13622 WriteSet tracking optimizations (Eugene Koifman, reviewed by Alan Gates)


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

Branch: refs/heads/branch-1
Commit: c0b532fceec3ae06f06eb94c0075ac7e86187c35
Parents: 6c65832
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Thu May 19 12:50:06 2016 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Thu May 19 12:50:06 2016 -0700

----------------------------------------------------------------------
 .../hive/hcatalog/streaming/HiveEndPoint.java   |    4 +-
 .../hive/metastore/TestHiveMetaStoreTxns.java   |    7 +
 metastore/if/hive_metastore.thrift              |   12 +
 .../gen/thrift/gen-cpp/ThriftHiveMetastore.cpp  | 1814 +++++++++---------
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  905 +++++----
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |   52 +-
 .../metastore/api/AddDynamicPartitions.java     |  127 +-
 .../hive/metastore/api/DataOperationType.java   |   57 +
 .../hive/metastore/api/LockComponent.java       |  234 ++-
 .../src/gen/thrift/gen-php/metastore/Types.php  |   86 +
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |   69 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   27 +-
 .../hive/metastore/HiveMetaStoreClient.java     |    9 +
 .../hadoop/hive/metastore/IMetaStoreClient.java |   11 +-
 .../hive/metastore/LockComponentBuilder.java    |   10 +
 .../hive/metastore/LockRequestBuilder.java      |    9 +
 .../hadoop/hive/metastore/txn/TxnHandler.java   |  108 +-
 .../metastore/txn/TestCompactionTxnHandler.java |   16 +-
 .../hive/metastore/txn/TestTxnHandler.java      |   84 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |    3 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java |   18 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   34 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |    6 +-
 .../hive/ql/lockmgr/TestDbTxnManager2.java      |  181 +-
 .../hive/ql/txn/compactor/TestCleaner.java      |    6 +
 .../hive/ql/txn/compactor/TestInitiator.java    |   20 +
 .../gen-py/hive_service/ThriftHive-remote       |    7 +
 27 files changed, 2473 insertions(+), 1443 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
index 41d2834..b9c1e23 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.LockComponentBuilder;
 import org.apache.hadoop.hive.metastore.LockRequestBuilder;
 import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 import org.apache.hadoop.hive.metastore.api.LockRequest;
@@ -964,7 +965,8 @@ public class HiveEndPoint {
       LockComponentBuilder lockCompBuilder = new LockComponentBuilder()
               .setDbName(hiveEndPoint.database)
               .setTableName(hiveEndPoint.table)
-              .setShared();
+              .setShared()
+              .setOperationType(DataOperationType.INSERT);
       if (partNameForLock!=null && !partNameForLock.isEmpty() ) {
           lockCompBuilder.setPartitionName(partNameForLock);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
index d5ecf98..f0646f6 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
@@ -21,6 +21,7 @@ import junit.framework.Assert;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.common.ValidReadTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
 import org.apache.hadoop.hive.metastore.api.LockResponse;
 import org.apache.hadoop.hive.metastore.api.LockState;
@@ -151,14 +152,17 @@ public class TestHiveMetaStoreTxns {
         .setTableName("mytable")
         .setPartitionName("mypartition")
         .setExclusive()
+        .setOperationType(DataOperationType.NO_TXN)
         .build());
     rqstBuilder.addLockComponent(new LockComponentBuilder()
         .setDbName("mydb")
         .setTableName("yourtable")
         .setSemiShared()
+        .setOperationType(DataOperationType.NO_TXN)
         .build());
     rqstBuilder.addLockComponent(new LockComponentBuilder()
         .setDbName("yourdb")
+        .setOperationType(DataOperationType.NO_TXN)
         .setShared()
         .build());
     rqstBuilder.setUser("fred");
@@ -187,15 +191,18 @@ public class TestHiveMetaStoreTxns {
         .setTableName("mytable")
         .setPartitionName("mypartition")
         .setSemiShared()
+        .setOperationType(DataOperationType.UPDATE)
         .build())
       .addLockComponent(new LockComponentBuilder()
         .setDbName("mydb")
         .setTableName("yourtable")
         .setSemiShared()
+        .setOperationType(DataOperationType.UPDATE)
         .build())
       .addLockComponent(new LockComponentBuilder()
         .setDbName("yourdb")
         .setShared()
+        .setOperationType(DataOperationType.SELECT)
         .build())
       .setUser("fred");
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index 4b5d207..f3d1d42 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -106,6 +106,15 @@ enum GrantRevokeType {
     REVOKE = 2,
 }
 
+enum DataOperationType {
+    SELECT = 1,
+    INSERT = 2
+    UPDATE = 3,
+    DELETE = 4,
+    UNSET = 5,//this is the default to distinguish from NULL from old clients
+    NO_TXN = 6,//drop table, insert overwrite, etc - something non-transactional
+}
+
 // Types of events the client can request that the metastore fire.  For now just support DML operations, as the metastore knows
 // about DDL operations and there's no reason for the client to request such an event.
 enum EventRequestType {
@@ -587,6 +596,8 @@ struct LockComponent {
     3: required string dbname,
     4: optional string tablename,
     5: optional string partitionname,
+    6: optional DataOperationType operationType = DataOperationType.UNSET,
+    7: optional bool isAcid = false
 }
 
 struct LockRequest {
@@ -692,6 +703,7 @@ struct AddDynamicPartitions {
     2: required string dbname,
     3: required string tablename,
     4: required list<string> partitionnames,
+    5: optional DataOperationType operationType = DataOperationType.UNSET
 }
 
 struct NotificationEventRequest {


[4/5] hive git commit: HIVE-13622 WriteSet tracking optimizations (Eugene Koifman, reviewed by Alan Gates)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
index 176b634..028c647 100644
--- a/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/ThriftHiveMetastore.cpp
@@ -1240,14 +1240,14 @@ uint32_t ThriftHiveMetastore_get_databases_result::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size657;
-            ::apache::thrift::protocol::TType _etype660;
-            xfer += iprot->readListBegin(_etype660, _size657);
-            this->success.resize(_size657);
-            uint32_t _i661;
-            for (_i661 = 0; _i661 < _size657; ++_i661)
+            uint32_t _size659;
+            ::apache::thrift::protocol::TType _etype662;
+            xfer += iprot->readListBegin(_etype662, _size659);
+            this->success.resize(_size659);
+            uint32_t _i663;
+            for (_i663 = 0; _i663 < _size659; ++_i663)
             {
-              xfer += iprot->readString(this->success[_i661]);
+              xfer += iprot->readString(this->success[_i663]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1286,10 +1286,10 @@ uint32_t ThriftHiveMetastore_get_databases_result::write(::apache::thrift::proto
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter662;
-      for (_iter662 = this->success.begin(); _iter662 != this->success.end(); ++_iter662)
+      std::vector<std::string> ::const_iterator _iter664;
+      for (_iter664 = this->success.begin(); _iter664 != this->success.end(); ++_iter664)
       {
-        xfer += oprot->writeString((*_iter662));
+        xfer += oprot->writeString((*_iter664));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1334,14 +1334,14 @@ uint32_t ThriftHiveMetastore_get_databases_presult::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size663;
-            ::apache::thrift::protocol::TType _etype666;
-            xfer += iprot->readListBegin(_etype666, _size663);
-            (*(this->success)).resize(_size663);
-            uint32_t _i667;
-            for (_i667 = 0; _i667 < _size663; ++_i667)
+            uint32_t _size665;
+            ::apache::thrift::protocol::TType _etype668;
+            xfer += iprot->readListBegin(_etype668, _size665);
+            (*(this->success)).resize(_size665);
+            uint32_t _i669;
+            for (_i669 = 0; _i669 < _size665; ++_i669)
             {
-              xfer += iprot->readString((*(this->success))[_i667]);
+              xfer += iprot->readString((*(this->success))[_i669]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1458,14 +1458,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size668;
-            ::apache::thrift::protocol::TType _etype671;
-            xfer += iprot->readListBegin(_etype671, _size668);
-            this->success.resize(_size668);
-            uint32_t _i672;
-            for (_i672 = 0; _i672 < _size668; ++_i672)
+            uint32_t _size670;
+            ::apache::thrift::protocol::TType _etype673;
+            xfer += iprot->readListBegin(_etype673, _size670);
+            this->success.resize(_size670);
+            uint32_t _i674;
+            for (_i674 = 0; _i674 < _size670; ++_i674)
             {
-              xfer += iprot->readString(this->success[_i672]);
+              xfer += iprot->readString(this->success[_i674]);
             }
             xfer += iprot->readListEnd();
           }
@@ -1504,10 +1504,10 @@ uint32_t ThriftHiveMetastore_get_all_databases_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter673;
-      for (_iter673 = this->success.begin(); _iter673 != this->success.end(); ++_iter673)
+      std::vector<std::string> ::const_iterator _iter675;
+      for (_iter675 = this->success.begin(); _iter675 != this->success.end(); ++_iter675)
       {
-        xfer += oprot->writeString((*_iter673));
+        xfer += oprot->writeString((*_iter675));
       }
       xfer += oprot->writeListEnd();
     }
@@ -1552,14 +1552,14 @@ uint32_t ThriftHiveMetastore_get_all_databases_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size674;
-            ::apache::thrift::protocol::TType _etype677;
-            xfer += iprot->readListBegin(_etype677, _size674);
-            (*(this->success)).resize(_size674);
-            uint32_t _i678;
-            for (_i678 = 0; _i678 < _size674; ++_i678)
+            uint32_t _size676;
+            ::apache::thrift::protocol::TType _etype679;
+            xfer += iprot->readListBegin(_etype679, _size676);
+            (*(this->success)).resize(_size676);
+            uint32_t _i680;
+            for (_i680 = 0; _i680 < _size676; ++_i680)
             {
-              xfer += iprot->readString((*(this->success))[_i678]);
+              xfer += iprot->readString((*(this->success))[_i680]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2621,17 +2621,17 @@ uint32_t ThriftHiveMetastore_get_type_all_result::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->success.clear();
-            uint32_t _size679;
-            ::apache::thrift::protocol::TType _ktype680;
-            ::apache::thrift::protocol::TType _vtype681;
-            xfer += iprot->readMapBegin(_ktype680, _vtype681, _size679);
-            uint32_t _i683;
-            for (_i683 = 0; _i683 < _size679; ++_i683)
+            uint32_t _size681;
+            ::apache::thrift::protocol::TType _ktype682;
+            ::apache::thrift::protocol::TType _vtype683;
+            xfer += iprot->readMapBegin(_ktype682, _vtype683, _size681);
+            uint32_t _i685;
+            for (_i685 = 0; _i685 < _size681; ++_i685)
             {
-              std::string _key684;
-              xfer += iprot->readString(_key684);
-              Type& _val685 = this->success[_key684];
-              xfer += _val685.read(iprot);
+              std::string _key686;
+              xfer += iprot->readString(_key686);
+              Type& _val687 = this->success[_key686];
+              xfer += _val687.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2670,11 +2670,11 @@ uint32_t ThriftHiveMetastore_get_type_all_result::write(::apache::thrift::protoc
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_MAP, 0);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::map<std::string, Type> ::const_iterator _iter686;
-      for (_iter686 = this->success.begin(); _iter686 != this->success.end(); ++_iter686)
+      std::map<std::string, Type> ::const_iterator _iter688;
+      for (_iter688 = this->success.begin(); _iter688 != this->success.end(); ++_iter688)
       {
-        xfer += oprot->writeString(_iter686->first);
-        xfer += _iter686->second.write(oprot);
+        xfer += oprot->writeString(_iter688->first);
+        xfer += _iter688->second.write(oprot);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2719,17 +2719,17 @@ uint32_t ThriftHiveMetastore_get_type_all_presult::read(::apache::thrift::protoc
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             (*(this->success)).clear();
-            uint32_t _size687;
-            ::apache::thrift::protocol::TType _ktype688;
-            ::apache::thrift::protocol::TType _vtype689;
-            xfer += iprot->readMapBegin(_ktype688, _vtype689, _size687);
-            uint32_t _i691;
-            for (_i691 = 0; _i691 < _size687; ++_i691)
+            uint32_t _size689;
+            ::apache::thrift::protocol::TType _ktype690;
+            ::apache::thrift::protocol::TType _vtype691;
+            xfer += iprot->readMapBegin(_ktype690, _vtype691, _size689);
+            uint32_t _i693;
+            for (_i693 = 0; _i693 < _size689; ++_i693)
             {
-              std::string _key692;
-              xfer += iprot->readString(_key692);
-              Type& _val693 = (*(this->success))[_key692];
-              xfer += _val693.read(iprot);
+              std::string _key694;
+              xfer += iprot->readString(_key694);
+              Type& _val695 = (*(this->success))[_key694];
+              xfer += _val695.read(iprot);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2883,14 +2883,14 @@ uint32_t ThriftHiveMetastore_get_fields_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size694;
-            ::apache::thrift::protocol::TType _etype697;
-            xfer += iprot->readListBegin(_etype697, _size694);
-            this->success.resize(_size694);
-            uint32_t _i698;
-            for (_i698 = 0; _i698 < _size694; ++_i698)
+            uint32_t _size696;
+            ::apache::thrift::protocol::TType _etype699;
+            xfer += iprot->readListBegin(_etype699, _size696);
+            this->success.resize(_size696);
+            uint32_t _i700;
+            for (_i700 = 0; _i700 < _size696; ++_i700)
             {
-              xfer += this->success[_i698].read(iprot);
+              xfer += this->success[_i700].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2945,10 +2945,10 @@ uint32_t ThriftHiveMetastore_get_fields_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter699;
-      for (_iter699 = this->success.begin(); _iter699 != this->success.end(); ++_iter699)
+      std::vector<FieldSchema> ::const_iterator _iter701;
+      for (_iter701 = this->success.begin(); _iter701 != this->success.end(); ++_iter701)
       {
-        xfer += (*_iter699).write(oprot);
+        xfer += (*_iter701).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3001,14 +3001,14 @@ uint32_t ThriftHiveMetastore_get_fields_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size700;
-            ::apache::thrift::protocol::TType _etype703;
-            xfer += iprot->readListBegin(_etype703, _size700);
-            (*(this->success)).resize(_size700);
-            uint32_t _i704;
-            for (_i704 = 0; _i704 < _size700; ++_i704)
+            uint32_t _size702;
+            ::apache::thrift::protocol::TType _etype705;
+            xfer += iprot->readListBegin(_etype705, _size702);
+            (*(this->success)).resize(_size702);
+            uint32_t _i706;
+            for (_i706 = 0; _i706 < _size702; ++_i706)
             {
-              xfer += (*(this->success))[_i704].read(iprot);
+              xfer += (*(this->success))[_i706].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3194,14 +3194,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size705;
-            ::apache::thrift::protocol::TType _etype708;
-            xfer += iprot->readListBegin(_etype708, _size705);
-            this->success.resize(_size705);
-            uint32_t _i709;
-            for (_i709 = 0; _i709 < _size705; ++_i709)
+            uint32_t _size707;
+            ::apache::thrift::protocol::TType _etype710;
+            xfer += iprot->readListBegin(_etype710, _size707);
+            this->success.resize(_size707);
+            uint32_t _i711;
+            for (_i711 = 0; _i711 < _size707; ++_i711)
             {
-              xfer += this->success[_i709].read(iprot);
+              xfer += this->success[_i711].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3256,10 +3256,10 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter710;
-      for (_iter710 = this->success.begin(); _iter710 != this->success.end(); ++_iter710)
+      std::vector<FieldSchema> ::const_iterator _iter712;
+      for (_iter712 = this->success.begin(); _iter712 != this->success.end(); ++_iter712)
       {
-        xfer += (*_iter710).write(oprot);
+        xfer += (*_iter712).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3312,14 +3312,14 @@ uint32_t ThriftHiveMetastore_get_fields_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size711;
-            ::apache::thrift::protocol::TType _etype714;
-            xfer += iprot->readListBegin(_etype714, _size711);
-            (*(this->success)).resize(_size711);
-            uint32_t _i715;
-            for (_i715 = 0; _i715 < _size711; ++_i715)
+            uint32_t _size713;
+            ::apache::thrift::protocol::TType _etype716;
+            xfer += iprot->readListBegin(_etype716, _size713);
+            (*(this->success)).resize(_size713);
+            uint32_t _i717;
+            for (_i717 = 0; _i717 < _size713; ++_i717)
             {
-              xfer += (*(this->success))[_i715].read(iprot);
+              xfer += (*(this->success))[_i717].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3489,14 +3489,14 @@ uint32_t ThriftHiveMetastore_get_schema_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size716;
-            ::apache::thrift::protocol::TType _etype719;
-            xfer += iprot->readListBegin(_etype719, _size716);
-            this->success.resize(_size716);
-            uint32_t _i720;
-            for (_i720 = 0; _i720 < _size716; ++_i720)
+            uint32_t _size718;
+            ::apache::thrift::protocol::TType _etype721;
+            xfer += iprot->readListBegin(_etype721, _size718);
+            this->success.resize(_size718);
+            uint32_t _i722;
+            for (_i722 = 0; _i722 < _size718; ++_i722)
             {
-              xfer += this->success[_i720].read(iprot);
+              xfer += this->success[_i722].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3551,10 +3551,10 @@ uint32_t ThriftHiveMetastore_get_schema_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter721;
-      for (_iter721 = this->success.begin(); _iter721 != this->success.end(); ++_iter721)
+      std::vector<FieldSchema> ::const_iterator _iter723;
+      for (_iter723 = this->success.begin(); _iter723 != this->success.end(); ++_iter723)
       {
-        xfer += (*_iter721).write(oprot);
+        xfer += (*_iter723).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3607,14 +3607,14 @@ uint32_t ThriftHiveMetastore_get_schema_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size722;
-            ::apache::thrift::protocol::TType _etype725;
-            xfer += iprot->readListBegin(_etype725, _size722);
-            (*(this->success)).resize(_size722);
-            uint32_t _i726;
-            for (_i726 = 0; _i726 < _size722; ++_i726)
+            uint32_t _size724;
+            ::apache::thrift::protocol::TType _etype727;
+            xfer += iprot->readListBegin(_etype727, _size724);
+            (*(this->success)).resize(_size724);
+            uint32_t _i728;
+            for (_i728 = 0; _i728 < _size724; ++_i728)
             {
-              xfer += (*(this->success))[_i726].read(iprot);
+              xfer += (*(this->success))[_i728].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3800,14 +3800,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::read(::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size727;
-            ::apache::thrift::protocol::TType _etype730;
-            xfer += iprot->readListBegin(_etype730, _size727);
-            this->success.resize(_size727);
-            uint32_t _i731;
-            for (_i731 = 0; _i731 < _size727; ++_i731)
+            uint32_t _size729;
+            ::apache::thrift::protocol::TType _etype732;
+            xfer += iprot->readListBegin(_etype732, _size729);
+            this->success.resize(_size729);
+            uint32_t _i733;
+            for (_i733 = 0; _i733 < _size729; ++_i733)
             {
-              xfer += this->success[_i731].read(iprot);
+              xfer += this->success[_i733].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -3862,10 +3862,10 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_result::write(:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<FieldSchema> ::const_iterator _iter732;
-      for (_iter732 = this->success.begin(); _iter732 != this->success.end(); ++_iter732)
+      std::vector<FieldSchema> ::const_iterator _iter734;
+      for (_iter734 = this->success.begin(); _iter734 != this->success.end(); ++_iter734)
       {
-        xfer += (*_iter732).write(oprot);
+        xfer += (*_iter734).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -3918,14 +3918,14 @@ uint32_t ThriftHiveMetastore_get_schema_with_environment_context_presult::read(:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size733;
-            ::apache::thrift::protocol::TType _etype736;
-            xfer += iprot->readListBegin(_etype736, _size733);
-            (*(this->success)).resize(_size733);
-            uint32_t _i737;
-            for (_i737 = 0; _i737 < _size733; ++_i737)
+            uint32_t _size735;
+            ::apache::thrift::protocol::TType _etype738;
+            xfer += iprot->readListBegin(_etype738, _size735);
+            (*(this->success)).resize(_size735);
+            uint32_t _i739;
+            for (_i739 = 0; _i739 < _size735; ++_i739)
             {
-              xfer += (*(this->success))[_i737].read(iprot);
+              xfer += (*(this->success))[_i739].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5099,14 +5099,14 @@ uint32_t ThriftHiveMetastore_get_tables_result::read(::apache::thrift::protocol:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size738;
-            ::apache::thrift::protocol::TType _etype741;
-            xfer += iprot->readListBegin(_etype741, _size738);
-            this->success.resize(_size738);
-            uint32_t _i742;
-            for (_i742 = 0; _i742 < _size738; ++_i742)
+            uint32_t _size740;
+            ::apache::thrift::protocol::TType _etype743;
+            xfer += iprot->readListBegin(_etype743, _size740);
+            this->success.resize(_size740);
+            uint32_t _i744;
+            for (_i744 = 0; _i744 < _size740; ++_i744)
             {
-              xfer += iprot->readString(this->success[_i742]);
+              xfer += iprot->readString(this->success[_i744]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5145,10 +5145,10 @@ uint32_t ThriftHiveMetastore_get_tables_result::write(::apache::thrift::protocol
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter743;
-      for (_iter743 = this->success.begin(); _iter743 != this->success.end(); ++_iter743)
+      std::vector<std::string> ::const_iterator _iter745;
+      for (_iter745 = this->success.begin(); _iter745 != this->success.end(); ++_iter745)
       {
-        xfer += oprot->writeString((*_iter743));
+        xfer += oprot->writeString((*_iter745));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5193,14 +5193,14 @@ uint32_t ThriftHiveMetastore_get_tables_presult::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size744;
-            ::apache::thrift::protocol::TType _etype747;
-            xfer += iprot->readListBegin(_etype747, _size744);
-            (*(this->success)).resize(_size744);
-            uint32_t _i748;
-            for (_i748 = 0; _i748 < _size744; ++_i748)
+            uint32_t _size746;
+            ::apache::thrift::protocol::TType _etype749;
+            xfer += iprot->readListBegin(_etype749, _size746);
+            (*(this->success)).resize(_size746);
+            uint32_t _i750;
+            for (_i750 = 0; _i750 < _size746; ++_i750)
             {
-              xfer += iprot->readString((*(this->success))[_i748]);
+              xfer += iprot->readString((*(this->success))[_i750]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5338,14 +5338,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size749;
-            ::apache::thrift::protocol::TType _etype752;
-            xfer += iprot->readListBegin(_etype752, _size749);
-            this->success.resize(_size749);
-            uint32_t _i753;
-            for (_i753 = 0; _i753 < _size749; ++_i753)
+            uint32_t _size751;
+            ::apache::thrift::protocol::TType _etype754;
+            xfer += iprot->readListBegin(_etype754, _size751);
+            this->success.resize(_size751);
+            uint32_t _i755;
+            for (_i755 = 0; _i755 < _size751; ++_i755)
             {
-              xfer += iprot->readString(this->success[_i753]);
+              xfer += iprot->readString(this->success[_i755]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5384,10 +5384,10 @@ uint32_t ThriftHiveMetastore_get_all_tables_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter754;
-      for (_iter754 = this->success.begin(); _iter754 != this->success.end(); ++_iter754)
+      std::vector<std::string> ::const_iterator _iter756;
+      for (_iter756 = this->success.begin(); _iter756 != this->success.end(); ++_iter756)
       {
-        xfer += oprot->writeString((*_iter754));
+        xfer += oprot->writeString((*_iter756));
       }
       xfer += oprot->writeListEnd();
     }
@@ -5432,14 +5432,14 @@ uint32_t ThriftHiveMetastore_get_all_tables_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size755;
-            ::apache::thrift::protocol::TType _etype758;
-            xfer += iprot->readListBegin(_etype758, _size755);
-            (*(this->success)).resize(_size755);
-            uint32_t _i759;
-            for (_i759 = 0; _i759 < _size755; ++_i759)
+            uint32_t _size757;
+            ::apache::thrift::protocol::TType _etype760;
+            xfer += iprot->readListBegin(_etype760, _size757);
+            (*(this->success)).resize(_size757);
+            uint32_t _i761;
+            for (_i761 = 0; _i761 < _size757; ++_i761)
             {
-              xfer += iprot->readString((*(this->success))[_i759]);
+              xfer += iprot->readString((*(this->success))[_i761]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5749,14 +5749,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::read(::apache::thri
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tbl_names.clear();
-            uint32_t _size760;
-            ::apache::thrift::protocol::TType _etype763;
-            xfer += iprot->readListBegin(_etype763, _size760);
-            this->tbl_names.resize(_size760);
-            uint32_t _i764;
-            for (_i764 = 0; _i764 < _size760; ++_i764)
+            uint32_t _size762;
+            ::apache::thrift::protocol::TType _etype765;
+            xfer += iprot->readListBegin(_etype765, _size762);
+            this->tbl_names.resize(_size762);
+            uint32_t _i766;
+            for (_i766 = 0; _i766 < _size762; ++_i766)
             {
-              xfer += iprot->readString(this->tbl_names[_i764]);
+              xfer += iprot->readString(this->tbl_names[_i766]);
             }
             xfer += iprot->readListEnd();
           }
@@ -5789,10 +5789,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_args::write(::apache::thr
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tbl_names.size()));
-    std::vector<std::string> ::const_iterator _iter765;
-    for (_iter765 = this->tbl_names.begin(); _iter765 != this->tbl_names.end(); ++_iter765)
+    std::vector<std::string> ::const_iterator _iter767;
+    for (_iter767 = this->tbl_names.begin(); _iter767 != this->tbl_names.end(); ++_iter767)
     {
-      xfer += oprot->writeString((*_iter765));
+      xfer += oprot->writeString((*_iter767));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5820,10 +5820,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_pargs::write(::apache::th
   xfer += oprot->writeFieldBegin("tbl_names", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->tbl_names)).size()));
-    std::vector<std::string> ::const_iterator _iter766;
-    for (_iter766 = (*(this->tbl_names)).begin(); _iter766 != (*(this->tbl_names)).end(); ++_iter766)
+    std::vector<std::string> ::const_iterator _iter768;
+    for (_iter768 = (*(this->tbl_names)).begin(); _iter768 != (*(this->tbl_names)).end(); ++_iter768)
     {
-      xfer += oprot->writeString((*_iter766));
+      xfer += oprot->writeString((*_iter768));
     }
     xfer += oprot->writeListEnd();
   }
@@ -5864,14 +5864,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size767;
-            ::apache::thrift::protocol::TType _etype770;
-            xfer += iprot->readListBegin(_etype770, _size767);
-            this->success.resize(_size767);
-            uint32_t _i771;
-            for (_i771 = 0; _i771 < _size767; ++_i771)
+            uint32_t _size769;
+            ::apache::thrift::protocol::TType _etype772;
+            xfer += iprot->readListBegin(_etype772, _size769);
+            this->success.resize(_size769);
+            uint32_t _i773;
+            for (_i773 = 0; _i773 < _size769; ++_i773)
             {
-              xfer += this->success[_i771].read(iprot);
+              xfer += this->success[_i773].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -5926,10 +5926,10 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Table> ::const_iterator _iter772;
-      for (_iter772 = this->success.begin(); _iter772 != this->success.end(); ++_iter772)
+      std::vector<Table> ::const_iterator _iter774;
+      for (_iter774 = this->success.begin(); _iter774 != this->success.end(); ++_iter774)
       {
-        xfer += (*_iter772).write(oprot);
+        xfer += (*_iter774).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -5982,14 +5982,14 @@ uint32_t ThriftHiveMetastore_get_table_objects_by_name_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size773;
-            ::apache::thrift::protocol::TType _etype776;
-            xfer += iprot->readListBegin(_etype776, _size773);
-            (*(this->success)).resize(_size773);
-            uint32_t _i777;
-            for (_i777 = 0; _i777 < _size773; ++_i777)
+            uint32_t _size775;
+            ::apache::thrift::protocol::TType _etype778;
+            xfer += iprot->readListBegin(_etype778, _size775);
+            (*(this->success)).resize(_size775);
+            uint32_t _i779;
+            for (_i779 = 0; _i779 < _size775; ++_i779)
             {
-              xfer += (*(this->success))[_i777].read(iprot);
+              xfer += (*(this->success))[_i779].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -6175,14 +6175,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size778;
-            ::apache::thrift::protocol::TType _etype781;
-            xfer += iprot->readListBegin(_etype781, _size778);
-            this->success.resize(_size778);
-            uint32_t _i782;
-            for (_i782 = 0; _i782 < _size778; ++_i782)
+            uint32_t _size780;
+            ::apache::thrift::protocol::TType _etype783;
+            xfer += iprot->readListBegin(_etype783, _size780);
+            this->success.resize(_size780);
+            uint32_t _i784;
+            for (_i784 = 0; _i784 < _size780; ++_i784)
             {
-              xfer += iprot->readString(this->success[_i782]);
+              xfer += iprot->readString(this->success[_i784]);
             }
             xfer += iprot->readListEnd();
           }
@@ -6237,10 +6237,10 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_result::write(::apache::t
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter783;
-      for (_iter783 = this->success.begin(); _iter783 != this->success.end(); ++_iter783)
+      std::vector<std::string> ::const_iterator _iter785;
+      for (_iter785 = this->success.begin(); _iter785 != this->success.end(); ++_iter785)
       {
-        xfer += oprot->writeString((*_iter783));
+        xfer += oprot->writeString((*_iter785));
       }
       xfer += oprot->writeListEnd();
     }
@@ -6293,14 +6293,14 @@ uint32_t ThriftHiveMetastore_get_table_names_by_filter_presult::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size784;
-            ::apache::thrift::protocol::TType _etype787;
-            xfer += iprot->readListBegin(_etype787, _size784);
-            (*(this->success)).resize(_size784);
-            uint32_t _i788;
-            for (_i788 = 0; _i788 < _size784; ++_i788)
+            uint32_t _size786;
+            ::apache::thrift::protocol::TType _etype789;
+            xfer += iprot->readListBegin(_etype789, _size786);
+            (*(this->success)).resize(_size786);
+            uint32_t _i790;
+            for (_i790 = 0; _i790 < _size786; ++_i790)
             {
-              xfer += iprot->readString((*(this->success))[_i788]);
+              xfer += iprot->readString((*(this->success))[_i790]);
             }
             xfer += iprot->readListEnd();
           }
@@ -7634,14 +7634,14 @@ uint32_t ThriftHiveMetastore_add_partitions_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size789;
-            ::apache::thrift::protocol::TType _etype792;
-            xfer += iprot->readListBegin(_etype792, _size789);
-            this->new_parts.resize(_size789);
-            uint32_t _i793;
-            for (_i793 = 0; _i793 < _size789; ++_i793)
+            uint32_t _size791;
+            ::apache::thrift::protocol::TType _etype794;
+            xfer += iprot->readListBegin(_etype794, _size791);
+            this->new_parts.resize(_size791);
+            uint32_t _i795;
+            for (_i795 = 0; _i795 < _size791; ++_i795)
             {
-              xfer += this->new_parts[_i793].read(iprot);
+              xfer += this->new_parts[_i795].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7670,10 +7670,10 @@ uint32_t ThriftHiveMetastore_add_partitions_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<Partition> ::const_iterator _iter794;
-    for (_iter794 = this->new_parts.begin(); _iter794 != this->new_parts.end(); ++_iter794)
+    std::vector<Partition> ::const_iterator _iter796;
+    for (_iter796 = this->new_parts.begin(); _iter796 != this->new_parts.end(); ++_iter796)
     {
-      xfer += (*_iter794).write(oprot);
+      xfer += (*_iter796).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7697,10 +7697,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<Partition> ::const_iterator _iter795;
-    for (_iter795 = (*(this->new_parts)).begin(); _iter795 != (*(this->new_parts)).end(); ++_iter795)
+    std::vector<Partition> ::const_iterator _iter797;
+    for (_iter797 = (*(this->new_parts)).begin(); _iter797 != (*(this->new_parts)).end(); ++_iter797)
     {
-      xfer += (*_iter795).write(oprot);
+      xfer += (*_iter797).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7909,14 +7909,14 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->new_parts.clear();
-            uint32_t _size796;
-            ::apache::thrift::protocol::TType _etype799;
-            xfer += iprot->readListBegin(_etype799, _size796);
-            this->new_parts.resize(_size796);
-            uint32_t _i800;
-            for (_i800 = 0; _i800 < _size796; ++_i800)
+            uint32_t _size798;
+            ::apache::thrift::protocol::TType _etype801;
+            xfer += iprot->readListBegin(_etype801, _size798);
+            this->new_parts.resize(_size798);
+            uint32_t _i802;
+            for (_i802 = 0; _i802 < _size798; ++_i802)
             {
-              xfer += this->new_parts[_i800].read(iprot);
+              xfer += this->new_parts[_i802].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -7945,10 +7945,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_args::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->new_parts.size()));
-    std::vector<PartitionSpec> ::const_iterator _iter801;
-    for (_iter801 = this->new_parts.begin(); _iter801 != this->new_parts.end(); ++_iter801)
+    std::vector<PartitionSpec> ::const_iterator _iter803;
+    for (_iter803 = this->new_parts.begin(); _iter803 != this->new_parts.end(); ++_iter803)
     {
-      xfer += (*_iter801).write(oprot);
+      xfer += (*_iter803).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -7972,10 +7972,10 @@ uint32_t ThriftHiveMetastore_add_partitions_pspec_pargs::write(::apache::thrift:
   xfer += oprot->writeFieldBegin("new_parts", ::apache::thrift::protocol::T_LIST, 1);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>((*(this->new_parts)).size()));
-    std::vector<PartitionSpec> ::const_iterator _iter802;
-    for (_iter802 = (*(this->new_parts)).begin(); _iter802 != (*(this->new_parts)).end(); ++_iter802)
+    std::vector<PartitionSpec> ::const_iterator _iter804;
+    for (_iter804 = (*(this->new_parts)).begin(); _iter804 != (*(this->new_parts)).end(); ++_iter804)
     {
-      xfer += (*_iter802).write(oprot);
+      xfer += (*_iter804).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -8200,14 +8200,14 @@ uint32_t ThriftHiveMetastore_append_partition_args::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size803;
-            ::apache::thrift::protocol::TType _etype806;
-            xfer += iprot->readListBegin(_etype806, _size803);
-            this->part_vals.resize(_size803);
-            uint32_t _i807;
-            for (_i807 = 0; _i807 < _size803; ++_i807)
+            uint32_t _size805;
+            ::apache::thrift::protocol::TType _etype808;
+            xfer += iprot->readListBegin(_etype808, _size805);
+            this->part_vals.resize(_size805);
+            uint32_t _i809;
+            for (_i809 = 0; _i809 < _size805; ++_i809)
             {
-              xfer += iprot->readString(this->part_vals[_i807]);
+              xfer += iprot->readString(this->part_vals[_i809]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8244,10 +8244,10 @@ uint32_t ThriftHiveMetastore_append_partition_args::write(::apache::thrift::prot
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter808;
-    for (_iter808 = this->part_vals.begin(); _iter808 != this->part_vals.end(); ++_iter808)
+    std::vector<std::string> ::const_iterator _iter810;
+    for (_iter810 = this->part_vals.begin(); _iter810 != this->part_vals.end(); ++_iter810)
     {
-      xfer += oprot->writeString((*_iter808));
+      xfer += oprot->writeString((*_iter810));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8279,10 +8279,10 @@ uint32_t ThriftHiveMetastore_append_partition_pargs::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter809;
-    for (_iter809 = (*(this->part_vals)).begin(); _iter809 != (*(this->part_vals)).end(); ++_iter809)
+    std::vector<std::string> ::const_iterator _iter811;
+    for (_iter811 = (*(this->part_vals)).begin(); _iter811 != (*(this->part_vals)).end(); ++_iter811)
     {
-      xfer += oprot->writeString((*_iter809));
+      xfer += oprot->writeString((*_iter811));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8754,14 +8754,14 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::rea
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size810;
-            ::apache::thrift::protocol::TType _etype813;
-            xfer += iprot->readListBegin(_etype813, _size810);
-            this->part_vals.resize(_size810);
-            uint32_t _i814;
-            for (_i814 = 0; _i814 < _size810; ++_i814)
+            uint32_t _size812;
+            ::apache::thrift::protocol::TType _etype815;
+            xfer += iprot->readListBegin(_etype815, _size812);
+            this->part_vals.resize(_size812);
+            uint32_t _i816;
+            for (_i816 = 0; _i816 < _size812; ++_i816)
             {
-              xfer += iprot->readString(this->part_vals[_i814]);
+              xfer += iprot->readString(this->part_vals[_i816]);
             }
             xfer += iprot->readListEnd();
           }
@@ -8806,10 +8806,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_args::wri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter815;
-    for (_iter815 = this->part_vals.begin(); _iter815 != this->part_vals.end(); ++_iter815)
+    std::vector<std::string> ::const_iterator _iter817;
+    for (_iter817 = this->part_vals.begin(); _iter817 != this->part_vals.end(); ++_iter817)
     {
-      xfer += oprot->writeString((*_iter815));
+      xfer += oprot->writeString((*_iter817));
     }
     xfer += oprot->writeListEnd();
   }
@@ -8845,10 +8845,10 @@ uint32_t ThriftHiveMetastore_append_partition_with_environment_context_pargs::wr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter816;
-    for (_iter816 = (*(this->part_vals)).begin(); _iter816 != (*(this->part_vals)).end(); ++_iter816)
+    std::vector<std::string> ::const_iterator _iter818;
+    for (_iter818 = (*(this->part_vals)).begin(); _iter818 != (*(this->part_vals)).end(); ++_iter818)
     {
-      xfer += oprot->writeString((*_iter816));
+      xfer += oprot->writeString((*_iter818));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9651,14 +9651,14 @@ uint32_t ThriftHiveMetastore_drop_partition_args::read(::apache::thrift::protoco
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size817;
-            ::apache::thrift::protocol::TType _etype820;
-            xfer += iprot->readListBegin(_etype820, _size817);
-            this->part_vals.resize(_size817);
-            uint32_t _i821;
-            for (_i821 = 0; _i821 < _size817; ++_i821)
+            uint32_t _size819;
+            ::apache::thrift::protocol::TType _etype822;
+            xfer += iprot->readListBegin(_etype822, _size819);
+            this->part_vals.resize(_size819);
+            uint32_t _i823;
+            for (_i823 = 0; _i823 < _size819; ++_i823)
             {
-              xfer += iprot->readString(this->part_vals[_i821]);
+              xfer += iprot->readString(this->part_vals[_i823]);
             }
             xfer += iprot->readListEnd();
           }
@@ -9703,10 +9703,10 @@ uint32_t ThriftHiveMetastore_drop_partition_args::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter822;
-    for (_iter822 = this->part_vals.begin(); _iter822 != this->part_vals.end(); ++_iter822)
+    std::vector<std::string> ::const_iterator _iter824;
+    for (_iter824 = this->part_vals.begin(); _iter824 != this->part_vals.end(); ++_iter824)
     {
-      xfer += oprot->writeString((*_iter822));
+      xfer += oprot->writeString((*_iter824));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9742,10 +9742,10 @@ uint32_t ThriftHiveMetastore_drop_partition_pargs::write(::apache::thrift::proto
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter823;
-    for (_iter823 = (*(this->part_vals)).begin(); _iter823 != (*(this->part_vals)).end(); ++_iter823)
+    std::vector<std::string> ::const_iterator _iter825;
+    for (_iter825 = (*(this->part_vals)).begin(); _iter825 != (*(this->part_vals)).end(); ++_iter825)
     {
-      xfer += oprot->writeString((*_iter823));
+      xfer += oprot->writeString((*_iter825));
     }
     xfer += oprot->writeListEnd();
   }
@@ -9954,14 +9954,14 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::read(
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size824;
-            ::apache::thrift::protocol::TType _etype827;
-            xfer += iprot->readListBegin(_etype827, _size824);
-            this->part_vals.resize(_size824);
-            uint32_t _i828;
-            for (_i828 = 0; _i828 < _size824; ++_i828)
+            uint32_t _size826;
+            ::apache::thrift::protocol::TType _etype829;
+            xfer += iprot->readListBegin(_etype829, _size826);
+            this->part_vals.resize(_size826);
+            uint32_t _i830;
+            for (_i830 = 0; _i830 < _size826; ++_i830)
             {
-              xfer += iprot->readString(this->part_vals[_i828]);
+              xfer += iprot->readString(this->part_vals[_i830]);
             }
             xfer += iprot->readListEnd();
           }
@@ -10014,10 +10014,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_args::write
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter829;
-    for (_iter829 = this->part_vals.begin(); _iter829 != this->part_vals.end(); ++_iter829)
+    std::vector<std::string> ::const_iterator _iter831;
+    for (_iter831 = this->part_vals.begin(); _iter831 != this->part_vals.end(); ++_iter831)
     {
-      xfer += oprot->writeString((*_iter829));
+      xfer += oprot->writeString((*_iter831));
     }
     xfer += oprot->writeListEnd();
   }
@@ -10057,10 +10057,10 @@ uint32_t ThriftHiveMetastore_drop_partition_with_environment_context_pargs::writ
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter830;
-    for (_iter830 = (*(this->part_vals)).begin(); _iter830 != (*(this->part_vals)).end(); ++_iter830)
+    std::vector<std::string> ::const_iterator _iter832;
+    for (_iter832 = (*(this->part_vals)).begin(); _iter832 != (*(this->part_vals)).end(); ++_iter832)
     {
-      xfer += oprot->writeString((*_iter830));
+      xfer += oprot->writeString((*_iter832));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11066,14 +11066,14 @@ uint32_t ThriftHiveMetastore_get_partition_args::read(::apache::thrift::protocol
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size831;
-            ::apache::thrift::protocol::TType _etype834;
-            xfer += iprot->readListBegin(_etype834, _size831);
-            this->part_vals.resize(_size831);
-            uint32_t _i835;
-            for (_i835 = 0; _i835 < _size831; ++_i835)
+            uint32_t _size833;
+            ::apache::thrift::protocol::TType _etype836;
+            xfer += iprot->readListBegin(_etype836, _size833);
+            this->part_vals.resize(_size833);
+            uint32_t _i837;
+            for (_i837 = 0; _i837 < _size833; ++_i837)
             {
-              xfer += iprot->readString(this->part_vals[_i835]);
+              xfer += iprot->readString(this->part_vals[_i837]);
             }
             xfer += iprot->readListEnd();
           }
@@ -11110,10 +11110,10 @@ uint32_t ThriftHiveMetastore_get_partition_args::write(::apache::thrift::protoco
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter836;
-    for (_iter836 = this->part_vals.begin(); _iter836 != this->part_vals.end(); ++_iter836)
+    std::vector<std::string> ::const_iterator _iter838;
+    for (_iter838 = this->part_vals.begin(); _iter838 != this->part_vals.end(); ++_iter838)
     {
-      xfer += oprot->writeString((*_iter836));
+      xfer += oprot->writeString((*_iter838));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11145,10 +11145,10 @@ uint32_t ThriftHiveMetastore_get_partition_pargs::write(::apache::thrift::protoc
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter837;
-    for (_iter837 = (*(this->part_vals)).begin(); _iter837 != (*(this->part_vals)).end(); ++_iter837)
+    std::vector<std::string> ::const_iterator _iter839;
+    for (_iter839 = (*(this->part_vals)).begin(); _iter839 != (*(this->part_vals)).end(); ++_iter839)
     {
-      xfer += oprot->writeString((*_iter837));
+      xfer += oprot->writeString((*_iter839));
     }
     xfer += oprot->writeListEnd();
   }
@@ -11337,17 +11337,17 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::read(::apache::thrift::pro
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size838;
-            ::apache::thrift::protocol::TType _ktype839;
-            ::apache::thrift::protocol::TType _vtype840;
-            xfer += iprot->readMapBegin(_ktype839, _vtype840, _size838);
-            uint32_t _i842;
-            for (_i842 = 0; _i842 < _size838; ++_i842)
+            uint32_t _size840;
+            ::apache::thrift::protocol::TType _ktype841;
+            ::apache::thrift::protocol::TType _vtype842;
+            xfer += iprot->readMapBegin(_ktype841, _vtype842, _size840);
+            uint32_t _i844;
+            for (_i844 = 0; _i844 < _size840; ++_i844)
             {
-              std::string _key843;
-              xfer += iprot->readString(_key843);
-              std::string& _val844 = this->partitionSpecs[_key843];
-              xfer += iprot->readString(_val844);
+              std::string _key845;
+              xfer += iprot->readString(_key845);
+              std::string& _val846 = this->partitionSpecs[_key845];
+              xfer += iprot->readString(_val846);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11408,11 +11408,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_args::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter845;
-    for (_iter845 = this->partitionSpecs.begin(); _iter845 != this->partitionSpecs.end(); ++_iter845)
+    std::map<std::string, std::string> ::const_iterator _iter847;
+    for (_iter847 = this->partitionSpecs.begin(); _iter847 != this->partitionSpecs.end(); ++_iter847)
     {
-      xfer += oprot->writeString(_iter845->first);
-      xfer += oprot->writeString(_iter845->second);
+      xfer += oprot->writeString(_iter847->first);
+      xfer += oprot->writeString(_iter847->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11452,11 +11452,11 @@ uint32_t ThriftHiveMetastore_exchange_partition_pargs::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter846;
-    for (_iter846 = (*(this->partitionSpecs)).begin(); _iter846 != (*(this->partitionSpecs)).end(); ++_iter846)
+    std::map<std::string, std::string> ::const_iterator _iter848;
+    for (_iter848 = (*(this->partitionSpecs)).begin(); _iter848 != (*(this->partitionSpecs)).end(); ++_iter848)
     {
-      xfer += oprot->writeString(_iter846->first);
-      xfer += oprot->writeString(_iter846->second);
+      xfer += oprot->writeString(_iter848->first);
+      xfer += oprot->writeString(_iter848->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11701,17 +11701,17 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->partitionSpecs.clear();
-            uint32_t _size847;
-            ::apache::thrift::protocol::TType _ktype848;
-            ::apache::thrift::protocol::TType _vtype849;
-            xfer += iprot->readMapBegin(_ktype848, _vtype849, _size847);
-            uint32_t _i851;
-            for (_i851 = 0; _i851 < _size847; ++_i851)
+            uint32_t _size849;
+            ::apache::thrift::protocol::TType _ktype850;
+            ::apache::thrift::protocol::TType _vtype851;
+            xfer += iprot->readMapBegin(_ktype850, _vtype851, _size849);
+            uint32_t _i853;
+            for (_i853 = 0; _i853 < _size849; ++_i853)
             {
-              std::string _key852;
-              xfer += iprot->readString(_key852);
-              std::string& _val853 = this->partitionSpecs[_key852];
-              xfer += iprot->readString(_val853);
+              std::string _key854;
+              xfer += iprot->readString(_key854);
+              std::string& _val855 = this->partitionSpecs[_key854];
+              xfer += iprot->readString(_val855);
             }
             xfer += iprot->readMapEnd();
           }
@@ -11772,11 +11772,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_args::write(::apache::thrift::p
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->partitionSpecs.size()));
-    std::map<std::string, std::string> ::const_iterator _iter854;
-    for (_iter854 = this->partitionSpecs.begin(); _iter854 != this->partitionSpecs.end(); ++_iter854)
+    std::map<std::string, std::string> ::const_iterator _iter856;
+    for (_iter856 = this->partitionSpecs.begin(); _iter856 != this->partitionSpecs.end(); ++_iter856)
     {
-      xfer += oprot->writeString(_iter854->first);
-      xfer += oprot->writeString(_iter854->second);
+      xfer += oprot->writeString(_iter856->first);
+      xfer += oprot->writeString(_iter856->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11816,11 +11816,11 @@ uint32_t ThriftHiveMetastore_exchange_partitions_pargs::write(::apache::thrift::
   xfer += oprot->writeFieldBegin("partitionSpecs", ::apache::thrift::protocol::T_MAP, 1);
   {
     xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->partitionSpecs)).size()));
-    std::map<std::string, std::string> ::const_iterator _iter855;
-    for (_iter855 = (*(this->partitionSpecs)).begin(); _iter855 != (*(this->partitionSpecs)).end(); ++_iter855)
+    std::map<std::string, std::string> ::const_iterator _iter857;
+    for (_iter857 = (*(this->partitionSpecs)).begin(); _iter857 != (*(this->partitionSpecs)).end(); ++_iter857)
     {
-      xfer += oprot->writeString(_iter855->first);
-      xfer += oprot->writeString(_iter855->second);
+      xfer += oprot->writeString(_iter857->first);
+      xfer += oprot->writeString(_iter857->second);
     }
     xfer += oprot->writeMapEnd();
   }
@@ -11877,14 +11877,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size856;
-            ::apache::thrift::protocol::TType _etype859;
-            xfer += iprot->readListBegin(_etype859, _size856);
-            this->success.resize(_size856);
-            uint32_t _i860;
-            for (_i860 = 0; _i860 < _size856; ++_i860)
+            uint32_t _size858;
+            ::apache::thrift::protocol::TType _etype861;
+            xfer += iprot->readListBegin(_etype861, _size858);
+            this->success.resize(_size858);
+            uint32_t _i862;
+            for (_i862 = 0; _i862 < _size858; ++_i862)
             {
-              xfer += this->success[_i860].read(iprot);
+              xfer += this->success[_i862].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -11947,10 +11947,10 @@ uint32_t ThriftHiveMetastore_exchange_partitions_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter861;
-      for (_iter861 = this->success.begin(); _iter861 != this->success.end(); ++_iter861)
+      std::vector<Partition> ::const_iterator _iter863;
+      for (_iter863 = this->success.begin(); _iter863 != this->success.end(); ++_iter863)
       {
-        xfer += (*_iter861).write(oprot);
+        xfer += (*_iter863).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12007,14 +12007,14 @@ uint32_t ThriftHiveMetastore_exchange_partitions_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size862;
-            ::apache::thrift::protocol::TType _etype865;
-            xfer += iprot->readListBegin(_etype865, _size862);
-            (*(this->success)).resize(_size862);
-            uint32_t _i866;
-            for (_i866 = 0; _i866 < _size862; ++_i866)
+            uint32_t _size864;
+            ::apache::thrift::protocol::TType _etype867;
+            xfer += iprot->readListBegin(_etype867, _size864);
+            (*(this->success)).resize(_size864);
+            uint32_t _i868;
+            for (_i868 = 0; _i868 < _size864; ++_i868)
             {
-              xfer += (*(this->success))[_i866].read(iprot);
+              xfer += (*(this->success))[_i868].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12113,14 +12113,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size867;
-            ::apache::thrift::protocol::TType _etype870;
-            xfer += iprot->readListBegin(_etype870, _size867);
-            this->part_vals.resize(_size867);
-            uint32_t _i871;
-            for (_i871 = 0; _i871 < _size867; ++_i871)
+            uint32_t _size869;
+            ::apache::thrift::protocol::TType _etype872;
+            xfer += iprot->readListBegin(_etype872, _size869);
+            this->part_vals.resize(_size869);
+            uint32_t _i873;
+            for (_i873 = 0; _i873 < _size869; ++_i873)
             {
-              xfer += iprot->readString(this->part_vals[_i871]);
+              xfer += iprot->readString(this->part_vals[_i873]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12141,14 +12141,14 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size872;
-            ::apache::thrift::protocol::TType _etype875;
-            xfer += iprot->readListBegin(_etype875, _size872);
-            this->group_names.resize(_size872);
-            uint32_t _i876;
-            for (_i876 = 0; _i876 < _size872; ++_i876)
+            uint32_t _size874;
+            ::apache::thrift::protocol::TType _etype877;
+            xfer += iprot->readListBegin(_etype877, _size874);
+            this->group_names.resize(_size874);
+            uint32_t _i878;
+            for (_i878 = 0; _i878 < _size874; ++_i878)
             {
-              xfer += iprot->readString(this->group_names[_i876]);
+              xfer += iprot->readString(this->group_names[_i878]);
             }
             xfer += iprot->readListEnd();
           }
@@ -12185,10 +12185,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter877;
-    for (_iter877 = this->part_vals.begin(); _iter877 != this->part_vals.end(); ++_iter877)
+    std::vector<std::string> ::const_iterator _iter879;
+    for (_iter879 = this->part_vals.begin(); _iter879 != this->part_vals.end(); ++_iter879)
     {
-      xfer += oprot->writeString((*_iter877));
+      xfer += oprot->writeString((*_iter879));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12201,10 +12201,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_args::write(::apache::thrif
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter878;
-    for (_iter878 = this->group_names.begin(); _iter878 != this->group_names.end(); ++_iter878)
+    std::vector<std::string> ::const_iterator _iter880;
+    for (_iter880 = this->group_names.begin(); _iter880 != this->group_names.end(); ++_iter880)
     {
-      xfer += oprot->writeString((*_iter878));
+      xfer += oprot->writeString((*_iter880));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12236,10 +12236,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter879;
-    for (_iter879 = (*(this->part_vals)).begin(); _iter879 != (*(this->part_vals)).end(); ++_iter879)
+    std::vector<std::string> ::const_iterator _iter881;
+    for (_iter881 = (*(this->part_vals)).begin(); _iter881 != (*(this->part_vals)).end(); ++_iter881)
     {
-      xfer += oprot->writeString((*_iter879));
+      xfer += oprot->writeString((*_iter881));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12252,10 +12252,10 @@ uint32_t ThriftHiveMetastore_get_partition_with_auth_pargs::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter880;
-    for (_iter880 = (*(this->group_names)).begin(); _iter880 != (*(this->group_names)).end(); ++_iter880)
+    std::vector<std::string> ::const_iterator _iter882;
+    for (_iter882 = (*(this->group_names)).begin(); _iter882 != (*(this->group_names)).end(); ++_iter882)
     {
-      xfer += oprot->writeString((*_iter880));
+      xfer += oprot->writeString((*_iter882));
     }
     xfer += oprot->writeListEnd();
   }
@@ -12814,14 +12814,14 @@ uint32_t ThriftHiveMetastore_get_partitions_result::read(::apache::thrift::proto
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size881;
-            ::apache::thrift::protocol::TType _etype884;
-            xfer += iprot->readListBegin(_etype884, _size881);
-            this->success.resize(_size881);
-            uint32_t _i885;
-            for (_i885 = 0; _i885 < _size881; ++_i885)
+            uint32_t _size883;
+            ::apache::thrift::protocol::TType _etype886;
+            xfer += iprot->readListBegin(_etype886, _size883);
+            this->success.resize(_size883);
+            uint32_t _i887;
+            for (_i887 = 0; _i887 < _size883; ++_i887)
             {
-              xfer += this->success[_i885].read(iprot);
+              xfer += this->success[_i887].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -12868,10 +12868,10 @@ uint32_t ThriftHiveMetastore_get_partitions_result::write(::apache::thrift::prot
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter886;
-      for (_iter886 = this->success.begin(); _iter886 != this->success.end(); ++_iter886)
+      std::vector<Partition> ::const_iterator _iter888;
+      for (_iter888 = this->success.begin(); _iter888 != this->success.end(); ++_iter888)
       {
-        xfer += (*_iter886).write(oprot);
+        xfer += (*_iter888).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -12920,14 +12920,14 @@ uint32_t ThriftHiveMetastore_get_partitions_presult::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size887;
-            ::apache::thrift::protocol::TType _etype890;
-            xfer += iprot->readListBegin(_etype890, _size887);
-            (*(this->success)).resize(_size887);
-            uint32_t _i891;
-            for (_i891 = 0; _i891 < _size887; ++_i891)
+            uint32_t _size889;
+            ::apache::thrift::protocol::TType _etype892;
+            xfer += iprot->readListBegin(_etype892, _size889);
+            (*(this->success)).resize(_size889);
+            uint32_t _i893;
+            for (_i893 = 0; _i893 < _size889; ++_i893)
             {
-              xfer += (*(this->success))[_i891].read(iprot);
+              xfer += (*(this->success))[_i893].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13026,14 +13026,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::read(::apache::thrif
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size892;
-            ::apache::thrift::protocol::TType _etype895;
-            xfer += iprot->readListBegin(_etype895, _size892);
-            this->group_names.resize(_size892);
-            uint32_t _i896;
-            for (_i896 = 0; _i896 < _size892; ++_i896)
+            uint32_t _size894;
+            ::apache::thrift::protocol::TType _etype897;
+            xfer += iprot->readListBegin(_etype897, _size894);
+            this->group_names.resize(_size894);
+            uint32_t _i898;
+            for (_i898 = 0; _i898 < _size894; ++_i898)
             {
-              xfer += iprot->readString(this->group_names[_i896]);
+              xfer += iprot->readString(this->group_names[_i898]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13078,10 +13078,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_args::write(::apache::thri
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter897;
-    for (_iter897 = this->group_names.begin(); _iter897 != this->group_names.end(); ++_iter897)
+    std::vector<std::string> ::const_iterator _iter899;
+    for (_iter899 = this->group_names.begin(); _iter899 != this->group_names.end(); ++_iter899)
     {
-      xfer += oprot->writeString((*_iter897));
+      xfer += oprot->writeString((*_iter899));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13121,10 +13121,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_pargs::write(::apache::thr
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 5);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->group_names)).size()));
-    std::vector<std::string> ::const_iterator _iter898;
-    for (_iter898 = (*(this->group_names)).begin(); _iter898 != (*(this->group_names)).end(); ++_iter898)
+    std::vector<std::string> ::const_iterator _iter900;
+    for (_iter900 = (*(this->group_names)).begin(); _iter900 != (*(this->group_names)).end(); ++_iter900)
     {
-      xfer += oprot->writeString((*_iter898));
+      xfer += oprot->writeString((*_iter900));
     }
     xfer += oprot->writeListEnd();
   }
@@ -13165,14 +13165,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::read(::apache::thr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size899;
-            ::apache::thrift::protocol::TType _etype902;
-            xfer += iprot->readListBegin(_etype902, _size899);
-            this->success.resize(_size899);
-            uint32_t _i903;
-            for (_i903 = 0; _i903 < _size899; ++_i903)
+            uint32_t _size901;
+            ::apache::thrift::protocol::TType _etype904;
+            xfer += iprot->readListBegin(_etype904, _size901);
+            this->success.resize(_size901);
+            uint32_t _i905;
+            for (_i905 = 0; _i905 < _size901; ++_i905)
             {
-              xfer += this->success[_i903].read(iprot);
+              xfer += this->success[_i905].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13219,10 +13219,10 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_result::write(::apache::th
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter904;
-      for (_iter904 = this->success.begin(); _iter904 != this->success.end(); ++_iter904)
+      std::vector<Partition> ::const_iterator _iter906;
+      for (_iter906 = this->success.begin(); _iter906 != this->success.end(); ++_iter906)
       {
-        xfer += (*_iter904).write(oprot);
+        xfer += (*_iter906).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13271,14 +13271,14 @@ uint32_t ThriftHiveMetastore_get_partitions_with_auth_presult::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size905;
-            ::apache::thrift::protocol::TType _etype908;
-            xfer += iprot->readListBegin(_etype908, _size905);
-            (*(this->success)).resize(_size905);
-            uint32_t _i909;
-            for (_i909 = 0; _i909 < _size905; ++_i909)
+            uint32_t _size907;
+            ::apache::thrift::protocol::TType _etype910;
+            xfer += iprot->readListBegin(_etype910, _size907);
+            (*(this->success)).resize(_size907);
+            uint32_t _i911;
+            for (_i911 = 0; _i911 < _size907; ++_i911)
             {
-              xfer += (*(this->success))[_i909].read(iprot);
+              xfer += (*(this->success))[_i911].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13456,14 +13456,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size910;
-            ::apache::thrift::protocol::TType _etype913;
-            xfer += iprot->readListBegin(_etype913, _size910);
-            this->success.resize(_size910);
-            uint32_t _i914;
-            for (_i914 = 0; _i914 < _size910; ++_i914)
+            uint32_t _size912;
+            ::apache::thrift::protocol::TType _etype915;
+            xfer += iprot->readListBegin(_etype915, _size912);
+            this->success.resize(_size912);
+            uint32_t _i916;
+            for (_i916 = 0; _i916 < _size912; ++_i916)
             {
-              xfer += this->success[_i914].read(iprot);
+              xfer += this->success[_i916].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13510,10 +13510,10 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_result::write(::apache::thrift
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<PartitionSpec> ::const_iterator _iter915;
-      for (_iter915 = this->success.begin(); _iter915 != this->success.end(); ++_iter915)
+      std::vector<PartitionSpec> ::const_iterator _iter917;
+      for (_iter917 = this->success.begin(); _iter917 != this->success.end(); ++_iter917)
       {
-        xfer += (*_iter915).write(oprot);
+        xfer += (*_iter917).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -13562,14 +13562,14 @@ uint32_t ThriftHiveMetastore_get_partitions_pspec_presult::read(::apache::thrift
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size916;
-            ::apache::thrift::protocol::TType _etype919;
-            xfer += iprot->readListBegin(_etype919, _size916);
-            (*(this->success)).resize(_size916);
-            uint32_t _i920;
-            for (_i920 = 0; _i920 < _size916; ++_i920)
+            uint32_t _size918;
+            ::apache::thrift::protocol::TType _etype921;
+            xfer += iprot->readListBegin(_etype921, _size918);
+            (*(this->success)).resize(_size918);
+            uint32_t _i922;
+            for (_i922 = 0; _i922 < _size918; ++_i922)
             {
-              xfer += (*(this->success))[_i920].read(iprot);
+              xfer += (*(this->success))[_i922].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -13747,14 +13747,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::read(::apache::thrift::
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size921;
-            ::apache::thrift::protocol::TType _etype924;
-            xfer += iprot->readListBegin(_etype924, _size921);
-            this->success.resize(_size921);
-            uint32_t _i925;
-            for (_i925 = 0; _i925 < _size921; ++_i925)
+            uint32_t _size923;
+            ::apache::thrift::protocol::TType _etype926;
+            xfer += iprot->readListBegin(_etype926, _size923);
+            this->success.resize(_size923);
+            uint32_t _i927;
+            for (_i927 = 0; _i927 < _size923; ++_i927)
             {
-              xfer += iprot->readString(this->success[_i925]);
+              xfer += iprot->readString(this->success[_i927]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13793,10 +13793,10 @@ uint32_t ThriftHiveMetastore_get_partition_names_result::write(::apache::thrift:
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->success.size()));
-      std::vector<std::string> ::const_iterator _iter926;
-      for (_iter926 = this->success.begin(); _iter926 != this->success.end(); ++_iter926)
+      std::vector<std::string> ::const_iterator _iter928;
+      for (_iter928 = this->success.begin(); _iter928 != this->success.end(); ++_iter928)
       {
-        xfer += oprot->writeString((*_iter926));
+        xfer += oprot->writeString((*_iter928));
       }
       xfer += oprot->writeListEnd();
     }
@@ -13841,14 +13841,14 @@ uint32_t ThriftHiveMetastore_get_partition_names_presult::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size927;
-            ::apache::thrift::protocol::TType _etype930;
-            xfer += iprot->readListBegin(_etype930, _size927);
-            (*(this->success)).resize(_size927);
-            uint32_t _i931;
-            for (_i931 = 0; _i931 < _size927; ++_i931)
+            uint32_t _size929;
+            ::apache::thrift::protocol::TType _etype932;
+            xfer += iprot->readListBegin(_etype932, _size929);
+            (*(this->success)).resize(_size929);
+            uint32_t _i933;
+            for (_i933 = 0; _i933 < _size929; ++_i933)
             {
-              xfer += iprot->readString((*(this->success))[_i931]);
+              xfer += iprot->readString((*(this->success))[_i933]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13923,14 +13923,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::read(::apache::thrift::prot
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size932;
-            ::apache::thrift::protocol::TType _etype935;
-            xfer += iprot->readListBegin(_etype935, _size932);
-            this->part_vals.resize(_size932);
-            uint32_t _i936;
-            for (_i936 = 0; _i936 < _size932; ++_i936)
+            uint32_t _size934;
+            ::apache::thrift::protocol::TType _etype937;
+            xfer += iprot->readListBegin(_etype937, _size934);
+            this->part_vals.resize(_size934);
+            uint32_t _i938;
+            for (_i938 = 0; _i938 < _size934; ++_i938)
             {
-              xfer += iprot->readString(this->part_vals[_i936]);
+              xfer += iprot->readString(this->part_vals[_i938]);
             }
             xfer += iprot->readListEnd();
           }
@@ -13975,10 +13975,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_args::write(::apache::thrift::pro
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter937;
-    for (_iter937 = this->part_vals.begin(); _iter937 != this->part_vals.end(); ++_iter937)
+    std::vector<std::string> ::const_iterator _iter939;
+    for (_iter939 = this->part_vals.begin(); _iter939 != this->part_vals.end(); ++_iter939)
     {
-      xfer += oprot->writeString((*_iter937));
+      xfer += oprot->writeString((*_iter939));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14014,10 +14014,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_pargs::write(::apache::thrift::pr
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>((*(this->part_vals)).size()));
-    std::vector<std::string> ::const_iterator _iter938;
-    for (_iter938 = (*(this->part_vals)).begin(); _iter938 != (*(this->part_vals)).end(); ++_iter938)
+    std::vector<std::string> ::const_iterator _iter940;
+    for (_iter940 = (*(this->part_vals)).begin(); _iter940 != (*(this->part_vals)).end(); ++_iter940)
     {
-      xfer += oprot->writeString((*_iter938));
+      xfer += oprot->writeString((*_iter940));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14062,14 +14062,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::read(::apache::thrift::pr
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->success.clear();
-            uint32_t _size939;
-            ::apache::thrift::protocol::TType _etype942;
-            xfer += iprot->readListBegin(_etype942, _size939);
-            this->success.resize(_size939);
-            uint32_t _i943;
-            for (_i943 = 0; _i943 < _size939; ++_i943)
+            uint32_t _size941;
+            ::apache::thrift::protocol::TType _etype944;
+            xfer += iprot->readListBegin(_etype944, _size941);
+            this->success.resize(_size941);
+            uint32_t _i945;
+            for (_i945 = 0; _i945 < _size941; ++_i945)
             {
-              xfer += this->success[_i943].read(iprot);
+              xfer += this->success[_i945].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14116,10 +14116,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_result::write(::apache::thrift::p
     xfer += oprot->writeFieldBegin("success", ::apache::thrift::protocol::T_LIST, 0);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->success.size()));
-      std::vector<Partition> ::const_iterator _iter944;
-      for (_iter944 = this->success.begin(); _iter944 != this->success.end(); ++_iter944)
+      std::vector<Partition> ::const_iterator _iter946;
+      for (_iter946 = this->success.begin(); _iter946 != this->success.end(); ++_iter946)
       {
-        xfer += (*_iter944).write(oprot);
+        xfer += (*_iter946).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -14168,14 +14168,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_presult::read(::apache::thrift::p
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             (*(this->success)).clear();
-            uint32_t _size945;
-            ::apache::thrift::protocol::TType _etype948;
-            xfer += iprot->readListBegin(_etype948, _size945);
-            (*(this->success)).resize(_size945);
-            uint32_t _i949;
-            for (_i949 = 0; _i949 < _size945; ++_i949)
+            uint32_t _size947;
+            ::apache::thrift::protocol::TType _etype950;
+            xfer += iprot->readListBegin(_etype950, _size947);
+            (*(this->success)).resize(_size947);
+            uint32_t _i951;
+            for (_i951 = 0; _i951 < _size947; ++_i951)
             {
-              xfer += (*(this->success))[_i949].read(iprot);
+              xfer += (*(this->success))[_i951].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -14258,14 +14258,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->part_vals.clear();
-            uint32_t _size950;
-            ::apache::thrift::protocol::TType _etype953;
-            xfer += iprot->readListBegin(_etype953, _size950);
-            this->part_vals.resize(_size950);
-            uint32_t _i954;
-            for (_i954 = 0; _i954 < _size950; ++_i954)
+            uint32_t _size952;
+            ::apache::thrift::protocol::TType _etype955;
+            xfer += iprot->readListBegin(_etype955, _size952);
+            this->part_vals.resize(_size952);
+            uint32_t _i956;
+            for (_i956 = 0; _i956 < _size952; ++_i956)
             {
-              xfer += iprot->readString(this->part_vals[_i954]);
+              xfer += iprot->readString(this->part_vals[_i956]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14294,14 +14294,14 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::read(::apache::th
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->group_names.clear();
-            uint32_t _size955;
-            ::apache::thrift::protocol::TType _etype958;
-            xfer += iprot->readListBegin(_etype958, _size955);
-            this->group_names.resize(_size955);
-            uint32_t _i959;
-            for (_i959 = 0; _i959 < _size955; ++_i959)
+            uint32_t _size957;
+            ::apache::thrift::protocol::TType _etype960;
+            xfer += iprot->readListBegin(_etype960, _size957);
+            this->group_names.resize(_size957);
+            uint32_t _i961;
+            for (_i961 = 0; _i961 < _size957; ++_i961)
             {
-              xfer += iprot->readString(this->group_names[_i959]);
+              xfer += iprot->readString(this->group_names[_i961]);
             }
             xfer += iprot->readListEnd();
           }
@@ -14338,10 +14338,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("part_vals", ::apache::thrift::protocol::T_LIST, 3);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->part_vals.size()));
-    std::vector<std::string> ::const_iterator _iter960;
-    for (_iter960 = this->part_vals.begin(); _iter960 != this->part_vals.end(); ++_iter960)
+    std::vector<std::string> ::const_iterator _iter962;
+    for (_iter962 = this->part_vals.begin(); _iter962 != this->part_vals.end(); ++_iter962)
     {
-      xfer += oprot->writeString((*_iter960));
+      xfer += oprot->writeString((*_iter962));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14358,10 +14358,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_args::write(::apache::t
   xfer += oprot->writeFieldBegin("group_names", ::apache::thrift::protocol::T_LIST, 6);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->group_names.size()));
-    std::vector<std::string> ::const_iterator _iter961;
-    for (_iter961 = this->group_names.begin(); _iter961 != this->group_names.end(); ++_iter961)
+    std::vector<std::string> ::const_iterator _iter963;
+    for (_iter963 = this->group_names.begin(); _iter963 != this->group_names.end(); ++_iter963)
     {
-      xfer += oprot->writeString((*_iter961));
+      xfer += oprot->writeString((*_iter963));
     }
     xfer += oprot->writeListEnd();
   }
@@ -14393,10 +14393,10 @@ uint32_t ThriftHiveMetastore_get_partitions_ps_with_auth_pargs

<TRUNCATED>

[2/5] hive git commit: HIVE-13622 WriteSet tracking optimizations (Eugene Koifman, reviewed by Alan Gates)

Posted by ek...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
index adb0c44..26d1b76 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
@@ -43,6 +43,8 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
   private static final org.apache.thrift.protocol.TField DBNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("dbname", org.apache.thrift.protocol.TType.STRING, (short)3);
   private static final org.apache.thrift.protocol.TField TABLENAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tablename", org.apache.thrift.protocol.TType.STRING, (short)4);
   private static final org.apache.thrift.protocol.TField PARTITIONNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("partitionname", org.apache.thrift.protocol.TType.STRING, (short)5);
+  private static final org.apache.thrift.protocol.TField OPERATION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationType", org.apache.thrift.protocol.TType.I32, (short)6);
+  private static final org.apache.thrift.protocol.TField IS_ACID_FIELD_DESC = new org.apache.thrift.protocol.TField("isAcid", org.apache.thrift.protocol.TType.BOOL, (short)7);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -55,6 +57,8 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
   private String dbname; // required
   private String tablename; // optional
   private String partitionname; // optional
+  private DataOperationType operationType; // optional
+  private boolean isAcid; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
@@ -70,7 +74,13 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
     LEVEL((short)2, "level"),
     DBNAME((short)3, "dbname"),
     TABLENAME((short)4, "tablename"),
-    PARTITIONNAME((short)5, "partitionname");
+    PARTITIONNAME((short)5, "partitionname"),
+    /**
+     * 
+     * @see DataOperationType
+     */
+    OPERATION_TYPE((short)6, "operationType"),
+    IS_ACID((short)7, "isAcid");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -95,6 +105,10 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
           return TABLENAME;
         case 5: // PARTITIONNAME
           return PARTITIONNAME;
+        case 6: // OPERATION_TYPE
+          return OPERATION_TYPE;
+        case 7: // IS_ACID
+          return IS_ACID;
         default:
           return null;
       }
@@ -135,7 +149,9 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
   }
 
   // isset id assignments
-  private static final _Fields optionals[] = {_Fields.TABLENAME,_Fields.PARTITIONNAME};
+  private static final int __ISACID_ISSET_ID = 0;
+  private byte __isset_bitfield = 0;
+  private static final _Fields optionals[] = {_Fields.TABLENAME,_Fields.PARTITIONNAME,_Fields.OPERATION_TYPE,_Fields.IS_ACID};
   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);
@@ -149,11 +165,19 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.PARTITIONNAME, new org.apache.thrift.meta_data.FieldMetaData("partitionname", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.OPERATION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("operationType", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, DataOperationType.class)));
+    tmpMap.put(_Fields.IS_ACID, new org.apache.thrift.meta_data.FieldMetaData("isAcid", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(LockComponent.class, metaDataMap);
   }
 
   public LockComponent() {
+    this.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.UNSET;
+
+    this.isAcid = false;
+
   }
 
   public LockComponent(
@@ -171,6 +195,7 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
    * Performs a deep copy on <i>other</i>.
    */
   public LockComponent(LockComponent other) {
+    __isset_bitfield = other.__isset_bitfield;
     if (other.isSetType()) {
       this.type = other.type;
     }
@@ -186,6 +211,10 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
     if (other.isSetPartitionname()) {
       this.partitionname = other.partitionname;
     }
+    if (other.isSetOperationType()) {
+      this.operationType = other.operationType;
+    }
+    this.isAcid = other.isAcid;
   }
 
   public LockComponent deepCopy() {
@@ -199,6 +228,10 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
     this.dbname = null;
     this.tablename = null;
     this.partitionname = null;
+    this.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.UNSET;
+
+    this.isAcid = false;
+
   }
 
   /**
@@ -332,6 +365,59 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
     }
   }
 
+  /**
+   * 
+   * @see DataOperationType
+   */
+  public DataOperationType getOperationType() {
+    return this.operationType;
+  }
+
+  /**
+   * 
+   * @see DataOperationType
+   */
+  public void setOperationType(DataOperationType operationType) {
+    this.operationType = operationType;
+  }
+
+  public void unsetOperationType() {
+    this.operationType = null;
+  }
+
+  /** Returns true if field operationType is set (has been assigned a value) and false otherwise */
+  public boolean isSetOperationType() {
+    return this.operationType != null;
+  }
+
+  public void setOperationTypeIsSet(boolean value) {
+    if (!value) {
+      this.operationType = null;
+    }
+  }
+
+  public boolean isIsAcid() {
+    return this.isAcid;
+  }
+
+  public void setIsAcid(boolean isAcid) {
+    this.isAcid = isAcid;
+    setIsAcidIsSet(true);
+  }
+
+  public void unsetIsAcid() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISACID_ISSET_ID);
+  }
+
+  /** Returns true if field isAcid is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsAcid() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISACID_ISSET_ID);
+  }
+
+  public void setIsAcidIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISACID_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case TYPE:
@@ -374,6 +460,22 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
       }
       break;
 
+    case OPERATION_TYPE:
+      if (value == null) {
+        unsetOperationType();
+      } else {
+        setOperationType((DataOperationType)value);
+      }
+      break;
+
+    case IS_ACID:
+      if (value == null) {
+        unsetIsAcid();
+      } else {
+        setIsAcid((Boolean)value);
+      }
+      break;
+
     }
   }
 
@@ -394,6 +496,12 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
     case PARTITIONNAME:
       return getPartitionname();
 
+    case OPERATION_TYPE:
+      return getOperationType();
+
+    case IS_ACID:
+      return isIsAcid();
+
     }
     throw new IllegalStateException();
   }
@@ -415,6 +523,10 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
       return isSetTablename();
     case PARTITIONNAME:
       return isSetPartitionname();
+    case OPERATION_TYPE:
+      return isSetOperationType();
+    case IS_ACID:
+      return isSetIsAcid();
     }
     throw new IllegalStateException();
   }
@@ -477,6 +589,24 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
         return false;
     }
 
+    boolean this_present_operationType = true && this.isSetOperationType();
+    boolean that_present_operationType = true && that.isSetOperationType();
+    if (this_present_operationType || that_present_operationType) {
+      if (!(this_present_operationType && that_present_operationType))
+        return false;
+      if (!this.operationType.equals(that.operationType))
+        return false;
+    }
+
+    boolean this_present_isAcid = true && this.isSetIsAcid();
+    boolean that_present_isAcid = true && that.isSetIsAcid();
+    if (this_present_isAcid || that_present_isAcid) {
+      if (!(this_present_isAcid && that_present_isAcid))
+        return false;
+      if (this.isAcid != that.isAcid)
+        return false;
+    }
+
     return true;
   }
 
@@ -509,6 +639,16 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
     if (present_partitionname)
       list.add(partitionname);
 
+    boolean present_operationType = true && (isSetOperationType());
+    list.add(present_operationType);
+    if (present_operationType)
+      list.add(operationType.getValue());
+
+    boolean present_isAcid = true && (isSetIsAcid());
+    list.add(present_isAcid);
+    if (present_isAcid)
+      list.add(isAcid);
+
     return list.hashCode();
   }
 
@@ -570,6 +710,26 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetOperationType()).compareTo(other.isSetOperationType());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetOperationType()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationType, other.operationType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetIsAcid()).compareTo(other.isSetIsAcid());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetIsAcid()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isAcid, other.isAcid);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -633,6 +793,22 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
       }
       first = false;
     }
+    if (isSetOperationType()) {
+      if (!first) sb.append(", ");
+      sb.append("operationType:");
+      if (this.operationType == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.operationType);
+      }
+      first = false;
+    }
+    if (isSetIsAcid()) {
+      if (!first) sb.append(", ");
+      sb.append("isAcid:");
+      sb.append(this.isAcid);
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -664,6 +840,8 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
 
   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);
@@ -728,6 +906,22 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 6: // OPERATION_TYPE
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.findByValue(iprot.readI32());
+              struct.setOperationTypeIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 7: // IS_ACID
+            if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
+              struct.isAcid = iprot.readBool();
+              struct.setIsAcidIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -770,6 +964,18 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
           oprot.writeFieldEnd();
         }
       }
+      if (struct.operationType != null) {
+        if (struct.isSetOperationType()) {
+          oprot.writeFieldBegin(OPERATION_TYPE_FIELD_DESC);
+          oprot.writeI32(struct.operationType.getValue());
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.isSetIsAcid()) {
+        oprot.writeFieldBegin(IS_ACID_FIELD_DESC);
+        oprot.writeBool(struct.isAcid);
+        oprot.writeFieldEnd();
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -797,13 +1003,25 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
       if (struct.isSetPartitionname()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetOperationType()) {
+        optionals.set(2);
+      }
+      if (struct.isSetIsAcid()) {
+        optionals.set(3);
+      }
+      oprot.writeBitSet(optionals, 4);
       if (struct.isSetTablename()) {
         oprot.writeString(struct.tablename);
       }
       if (struct.isSetPartitionname()) {
         oprot.writeString(struct.partitionname);
       }
+      if (struct.isSetOperationType()) {
+        oprot.writeI32(struct.operationType.getValue());
+      }
+      if (struct.isSetIsAcid()) {
+        oprot.writeBool(struct.isAcid);
+      }
     }
 
     @Override
@@ -815,7 +1033,7 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
       struct.setLevelIsSet(true);
       struct.dbname = iprot.readString();
       struct.setDbnameIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(4);
       if (incoming.get(0)) {
         struct.tablename = iprot.readString();
         struct.setTablenameIsSet(true);
@@ -824,6 +1042,14 @@ public class LockComponent implements org.apache.thrift.TBase<LockComponent, Loc
         struct.partitionname = iprot.readString();
         struct.setPartitionnameIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.findByValue(iprot.readI32());
+        struct.setOperationTypeIsSet(true);
+      }
+      if (incoming.get(3)) {
+        struct.isAcid = iprot.readBool();
+        struct.setIsAcidIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index fe25366..045864a 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -114,6 +114,23 @@ final class GrantRevokeType {
   );
 }
 
+final class DataOperationType {
+  const SELECT = 1;
+  const INSERT = 2;
+  const UPDATE = 3;
+  const DELETE = 4;
+  const UNSET = 5;
+  const NO_TXN = 6;
+  static public $__names = array(
+    1 => 'SELECT',
+    2 => 'INSERT',
+    3 => 'UPDATE',
+    4 => 'DELETE',
+    5 => 'UNSET',
+    6 => 'NO_TXN',
+  );
+}
+
 final class EventRequestType {
   const INSERT = 1;
   const UPDATE = 2;
@@ -11049,6 +11066,14 @@ class LockComponent {
    * @var string
    */
   public $partitionname = null;
+  /**
+   * @var int
+   */
+  public $operationType =   5;
+  /**
+   * @var bool
+   */
+  public $isAcid = false;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -11073,6 +11098,14 @@ class LockComponent {
           'var' => 'partitionname',
           'type' => TType::STRING,
           ),
+        6 => array(
+          'var' => 'operationType',
+          'type' => TType::I32,
+          ),
+        7 => array(
+          'var' => 'isAcid',
+          'type' => TType::BOOL,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -11091,6 +11124,12 @@ class LockComponent {
       if (isset($vals['partitionname'])) {
         $this->partitionname = $vals['partitionname'];
       }
+      if (isset($vals['operationType'])) {
+        $this->operationType = $vals['operationType'];
+      }
+      if (isset($vals['isAcid'])) {
+        $this->isAcid = $vals['isAcid'];
+      }
     }
   }
 
@@ -11148,6 +11187,20 @@ class LockComponent {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 6:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->operationType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
+        case 7:
+          if ($ftype == TType::BOOL) {
+            $xfer += $input->readBool($this->isAcid);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -11186,6 +11239,16 @@ class LockComponent {
       $xfer += $output->writeString($this->partitionname);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->operationType !== null) {
+      $xfer += $output->writeFieldBegin('operationType', TType::I32, 6);
+      $xfer += $output->writeI32($this->operationType);
+      $xfer += $output->writeFieldEnd();
+    }
+    if ($this->isAcid !== null) {
+      $xfer += $output->writeFieldBegin('isAcid', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->isAcid);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;
@@ -13378,6 +13441,10 @@ class AddDynamicPartitions {
    * @var string[]
    */
   public $partitionnames = null;
+  /**
+   * @var int
+   */
+  public $operationType =   5;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -13402,6 +13469,10 @@ class AddDynamicPartitions {
             'type' => TType::STRING,
             ),
           ),
+        5 => array(
+          'var' => 'operationType',
+          'type' => TType::I32,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -13417,6 +13488,9 @@ class AddDynamicPartitions {
       if (isset($vals['partitionnames'])) {
         $this->partitionnames = $vals['partitionnames'];
       }
+      if (isset($vals['operationType'])) {
+        $this->operationType = $vals['operationType'];
+      }
     }
   }
 
@@ -13477,6 +13551,13 @@ class AddDynamicPartitions {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 5:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->operationType);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -13522,6 +13603,11 @@ class AddDynamicPartitions {
       }
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->operationType !== null) {
+      $xfer += $output->writeFieldBegin('operationType', TType::I32, 5);
+      $xfer += $output->writeI32($this->operationType);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index c59fa3e..29ba9b1 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -168,6 +168,32 @@ class GrantRevokeType:
     "REVOKE": 2,
   }
 
+class DataOperationType:
+  SELECT = 1
+  INSERT = 2
+  UPDATE = 3
+  DELETE = 4
+  UNSET = 5
+  NO_TXN = 6
+
+  _VALUES_TO_NAMES = {
+    1: "SELECT",
+    2: "INSERT",
+    3: "UPDATE",
+    4: "DELETE",
+    5: "UNSET",
+    6: "NO_TXN",
+  }
+
+  _NAMES_TO_VALUES = {
+    "SELECT": 1,
+    "INSERT": 2,
+    "UPDATE": 3,
+    "DELETE": 4,
+    "UNSET": 5,
+    "NO_TXN": 6,
+  }
+
 class EventRequestType:
   INSERT = 1
   UPDATE = 2
@@ -7633,6 +7659,8 @@ class LockComponent:
    - dbname
    - tablename
    - partitionname
+   - operationType
+   - isAcid
   """
 
   thrift_spec = (
@@ -7642,14 +7670,18 @@ class LockComponent:
     (3, TType.STRING, 'dbname', None, None, ), # 3
     (4, TType.STRING, 'tablename', None, None, ), # 4
     (5, TType.STRING, 'partitionname', None, None, ), # 5
+    (6, TType.I32, 'operationType', None,     5, ), # 6
+    (7, TType.BOOL, 'isAcid', None, False, ), # 7
   )
 
-  def __init__(self, type=None, level=None, dbname=None, tablename=None, partitionname=None,):
+  def __init__(self, type=None, level=None, dbname=None, tablename=None, partitionname=None, operationType=thrift_spec[6][4], isAcid=thrift_spec[7][4],):
     self.type = type
     self.level = level
     self.dbname = dbname
     self.tablename = tablename
     self.partitionname = partitionname
+    self.operationType = operationType
+    self.isAcid = isAcid
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -7685,6 +7717,16 @@ class LockComponent:
           self.partitionname = iprot.readString()
         else:
           iprot.skip(ftype)
+      elif fid == 6:
+        if ftype == TType.I32:
+          self.operationType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
+      elif fid == 7:
+        if ftype == TType.BOOL:
+          self.isAcid = iprot.readBool()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -7715,6 +7757,14 @@ class LockComponent:
       oprot.writeFieldBegin('partitionname', TType.STRING, 5)
       oprot.writeString(self.partitionname)
       oprot.writeFieldEnd()
+    if self.operationType is not None:
+      oprot.writeFieldBegin('operationType', TType.I32, 6)
+      oprot.writeI32(self.operationType)
+      oprot.writeFieldEnd()
+    if self.isAcid is not None:
+      oprot.writeFieldBegin('isAcid', TType.BOOL, 7)
+      oprot.writeBool(self.isAcid)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -7735,6 +7785,8 @@ class LockComponent:
     value = (value * 31) ^ hash(self.dbname)
     value = (value * 31) ^ hash(self.tablename)
     value = (value * 31) ^ hash(self.partitionname)
+    value = (value * 31) ^ hash(self.operationType)
+    value = (value * 31) ^ hash(self.isAcid)
     return value
 
   def __repr__(self):
@@ -9300,6 +9352,7 @@ class AddDynamicPartitions:
    - dbname
    - tablename
    - partitionnames
+   - operationType
   """
 
   thrift_spec = (
@@ -9308,13 +9361,15 @@ class AddDynamicPartitions:
     (2, TType.STRING, 'dbname', None, None, ), # 2
     (3, TType.STRING, 'tablename', None, None, ), # 3
     (4, TType.LIST, 'partitionnames', (TType.STRING,None), None, ), # 4
+    (5, TType.I32, 'operationType', None,     5, ), # 5
   )
 
-  def __init__(self, txnid=None, dbname=None, tablename=None, partitionnames=None,):
+  def __init__(self, txnid=None, dbname=None, tablename=None, partitionnames=None, operationType=thrift_spec[5][4],):
     self.txnid = txnid
     self.dbname = dbname
     self.tablename = tablename
     self.partitionnames = partitionnames
+    self.operationType = operationType
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -9350,6 +9405,11 @@ class AddDynamicPartitions:
           iprot.readListEnd()
         else:
           iprot.skip(ftype)
+      elif fid == 5:
+        if ftype == TType.I32:
+          self.operationType = iprot.readI32()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -9379,6 +9439,10 @@ class AddDynamicPartitions:
         oprot.writeString(iter447)
       oprot.writeListEnd()
       oprot.writeFieldEnd()
+    if self.operationType is not None:
+      oprot.writeFieldBegin('operationType', TType.I32, 5)
+      oprot.writeI32(self.operationType)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -9400,6 +9464,7 @@ class AddDynamicPartitions:
     value = (value * 31) ^ hash(self.dbname)
     value = (value * 31) ^ hash(self.tablename)
     value = (value * 31) ^ hash(self.partitionnames)
+    value = (value * 31) ^ hash(self.operationType)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 2874308..662658c 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -79,6 +79,17 @@ module GrantRevokeType
   VALID_VALUES = Set.new([GRANT, REVOKE]).freeze
 end
 
+module DataOperationType
+  SELECT = 1
+  INSERT = 2
+  UPDATE = 3
+  DELETE = 4
+  UNSET = 5
+  NO_TXN = 6
+  VALUE_MAP = {1 => "SELECT", 2 => "INSERT", 3 => "UPDATE", 4 => "DELETE", 5 => "UNSET", 6 => "NO_TXN"}
+  VALID_VALUES = Set.new([SELECT, INSERT, UPDATE, DELETE, UNSET, NO_TXN]).freeze
+end
+
 module EventRequestType
   INSERT = 1
   UPDATE = 2
@@ -1721,13 +1732,17 @@ class LockComponent
   DBNAME = 3
   TABLENAME = 4
   PARTITIONNAME = 5
+  OPERATIONTYPE = 6
+  ISACID = 7
 
   FIELDS = {
     TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::LockType},
     LEVEL => {:type => ::Thrift::Types::I32, :name => 'level', :enum_class => ::LockLevel},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename', :optional => true},
-    PARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'partitionname', :optional => true}
+    PARTITIONNAME => {:type => ::Thrift::Types::STRING, :name => 'partitionname', :optional => true},
+    OPERATIONTYPE => {:type => ::Thrift::Types::I32, :name => 'operationType', :default =>     5, :optional => true, :enum_class => ::DataOperationType},
+    ISACID => {:type => ::Thrift::Types::BOOL, :name => 'isAcid', :default => false, :optional => true}
   }
 
   def struct_fields; FIELDS; end
@@ -1742,6 +1757,9 @@ class LockComponent
     unless @level.nil? || ::LockLevel::VALID_VALUES.include?(@level)
       raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field level!')
     end
+    unless @operationType.nil? || ::DataOperationType::VALID_VALUES.include?(@operationType)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field operationType!')
+    end
   end
 
   ::Thrift::Struct.generate_accessors self
@@ -2103,12 +2121,14 @@ class AddDynamicPartitions
   DBNAME = 2
   TABLENAME = 3
   PARTITIONNAMES = 4
+  OPERATIONTYPE = 5
 
   FIELDS = {
     TXNID => {:type => ::Thrift::Types::I64, :name => 'txnid'},
     DBNAME => {:type => ::Thrift::Types::STRING, :name => 'dbname'},
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename'},
-    PARTITIONNAMES => {:type => ::Thrift::Types::LIST, :name => 'partitionnames', :element => {:type => ::Thrift::Types::STRING}}
+    PARTITIONNAMES => {:type => ::Thrift::Types::LIST, :name => 'partitionnames', :element => {:type => ::Thrift::Types::STRING}},
+    OPERATIONTYPE => {:type => ::Thrift::Types::I32, :name => 'operationType', :default =>     5, :optional => true, :enum_class => ::DataOperationType}
   }
 
   def struct_fields; FIELDS; end
@@ -2118,6 +2138,9 @@ class AddDynamicPartitions
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field dbname is unset!') unless @dbname
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field tablename is unset!') unless @tablename
     raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field partitionnames is unset!') unless @partitionnames
+    unless @operationType.nil? || ::DataOperationType::VALID_VALUES.include?(@operationType)
+      raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field operationType!')
+    end
   end
 
   ::Thrift::Struct.generate_accessors self

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
index 94d5d86..83d533f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
@@ -69,6 +69,7 @@ import org.apache.hadoop.hive.metastore.api.CompactionRequest;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.DropPartitionsExpr;
 import org.apache.hadoop.hive.metastore.api.DropPartitionsRequest;
@@ -1958,10 +1959,18 @@ public class HiveMetaStoreClient implements IMetaStoreClient {
     return client.show_compact(new ShowCompactRequest());
   }
 
+  @Deprecated
   @Override
   public void addDynamicPartitions(long txnId, String dbName, String tableName,
                                    List<String> partNames) throws TException {
     client.add_dynamic_partitions(new AddDynamicPartitions(txnId, dbName, tableName, partNames));
+  }  
+  @Override
+  public void addDynamicPartitions(long txnId, String dbName, String tableName,
+                                   List<String> partNames, DataOperationType operationType) throws TException {
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnId, dbName, tableName, partNames);
+    adp.setOperationType(operationType);
+    client.add_dynamic_partitions(adp);
   }
 
   @InterfaceAudience.LimitedPrivate({"HCatalog"})

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
index da693f7..fc7b70f 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
 import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FireEventRequest;
 import org.apache.hadoop.hive.metastore.api.FireEventResponse;
@@ -61,7 +62,6 @@ import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Function;
 import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
-import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
 import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
 import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
@@ -1402,6 +1402,12 @@ public interface IMetaStoreClient {
   ShowCompactResponse showCompactions() throws TException;
 
   /**
+   * @deprecated in Hive 1.3.0/2.1.0 - will be removed in 2 releases
+   */
+  @Deprecated
+  void addDynamicPartitions(long txnId, String dbName, String tableName, List<String> partNames)
+    throws TException;
+  /**
    * Send a list of partitions to the metastore to indicate which partitions were loaded
    * dynamically.
    * @param txnId id of the transaction
@@ -1410,7 +1416,8 @@ public interface IMetaStoreClient {
    * @param partNames partition name, as constructed by Warehouse.makePartName
    * @throws TException
    */
-  void addDynamicPartitions(long txnId, String dbName, String tableName, List<String> partNames)
+  void addDynamicPartitions(long txnId, String dbName, String tableName, List<String> partNames,
+                            DataOperationType operationType)
     throws TException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java b/metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
index acd4653..3e8f193 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.metastore;
 
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.LockComponent;
 import org.apache.hadoop.hive.metastore.api.LockLevel;
 import org.apache.hadoop.hive.metastore.api.LockType;
@@ -70,7 +71,16 @@ public class LockComponentBuilder {
     component.setDbname(dbName);
     return this;
   }
+  
+  public LockComponentBuilder setOperationType(DataOperationType dop) {
+    component.setOperationType(dop);
+    return this;
+  }
 
+  public LockComponentBuilder setIsAcid(boolean t) {
+    component.setIsAcid(t);
+    return this;
+  }
   /**
    * Set the table name.
    * @param tableName table name

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java b/metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java
index 2fa7e07..6317a96 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/LockRequestBuilder.java
@@ -35,10 +35,19 @@ public class LockRequestBuilder {
   private LockTrie trie;
   private boolean userSet;
 
+  /**
+   * @deprecated 
+   */
   public LockRequestBuilder() {
+    this(null);
+  }
+  public LockRequestBuilder(String agentInfo) {
     req = new LockRequest();
     trie = new LockTrie();
     userSet = false;
+    if(agentInfo != null) {
+      req.setAgentInfo(agentInfo);
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index 0023f08..4da5542 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -131,7 +131,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   static private boolean doRetryOnConnPool = false;
   
   private enum OpertaionType {
-    INSERT('i'), UPDATE('u'), DELETE('d');
+    SELECT('s'), INSERT('i'), UPDATE('u'), DELETE('d');
     private final char sqlConst;
     OpertaionType(char sqlConst) {
       this.sqlConst = sqlConst;
@@ -141,6 +141,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
     }
     public static OpertaionType fromString(char sqlConst) {
       switch (sqlConst) {
+        case 's':
+          return SELECT;
         case 'i':
           return INSERT;
         case 'u':
@@ -151,16 +153,18 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           throw new IllegalArgumentException(quoteChar(sqlConst));
       }
     }
-    //we should instead just pass in OpertaionType from client (HIVE-13622)
-    @Deprecated
-    public static OpertaionType fromLockType(LockType lockType) {
-      switch (lockType) {
-        case SHARED_READ:
-          return INSERT;
-        case SHARED_WRITE:
-          return UPDATE;
+    public static OpertaionType fromDataOperationType(DataOperationType dop) {
+      switch (dop) {
+        case SELECT:
+          return OpertaionType.SELECT;
+        case INSERT:
+          return OpertaionType.INSERT;
+        case UPDATE:
+          return OpertaionType.UPDATE;
+        case DELETE:
+          return OpertaionType.DELETE;
         default:
-          throw new IllegalArgumentException("Unexpected lock type: " + lockType);
+          throw new IllegalArgumentException("Unexpected value: " + dop);
       }
     }
   }
@@ -674,20 +678,21 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         String s = "insert into COMPLETED_TXN_COMPONENTS select tc_txnid, tc_database, tc_table, " +
           "tc_partition from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute insert <" + s + ">");
-        if (stmt.executeUpdate(s) < 1) {
+        int modCount = 0;
+        if ((modCount = stmt.executeUpdate(s)) < 1) {
           //this can be reasonable for an empty txn START/COMMIT or read-only txn
           LOG.info("Expected to move at least one record from txn_components to " +
             "completed_txn_components when committing txn! " + JavaUtils.txnIdToString(txnid));
         }
         s = "delete from TXN_COMPONENTS where tc_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        stmt.executeUpdate(s);
+        modCount = stmt.executeUpdate(s);
         s = "delete from HIVE_LOCKS where hl_txnid = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        stmt.executeUpdate(s);
+        modCount = stmt.executeUpdate(s);
         s = "delete from TXNS where txn_id = " + txnid;
         LOG.debug("Going to execute update <" + s + ">");
-        stmt.executeUpdate(s);
+        modCount = stmt.executeUpdate(s);
         LOG.debug("Going to commit");
         dbConn.commit();
       } catch (SQLException e) {
@@ -829,7 +834,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         /** Get the next lock id.
          * This has to be atomic with adding entries to HIVE_LOCK entries (1st add in W state) to prevent a race.
          * Suppose ID gen is a separate txn and 2 concurrent lock() methods are running.  1st one generates nl_next=7,
-         * 2nd nl_next=8.  Then 8 goes first to insert into HIVE_LOCKS and aquires the locks.  Then 7 unblocks,
+         * 2nd nl_next=8.  Then 8 goes first to insert into HIVE_LOCKS and acquires the locks.  Then 7 unblocks,
          * and add it's W locks but it won't see locks from 8 since to be 'fair' {@link #checkLock(java.sql.Connection, long)}
          * doesn't block on locks acquired later than one it's checking*/
         String s = addForUpdateClause("select nl_next from NEXT_LOCK_ID");
@@ -847,13 +852,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         stmt.executeUpdate(s);
 
         if (txnid > 0) {
-          /**DBTxnManager#acquireLocks() knows if it's I/U/D (that's how it decides what lock to get)
-           * So if we add that to LockRequest we'll know that here 
-           * Should probably add it to LockComponent so that if in the future we decide wo allow 1 LockRequest
-           * to contain LockComponent for multiple operations.
-           * Deriving it from lock info doesn't distinguish between Update and Delete
-           * 
-           * QueryPlan has BaseSemanticAnalyzer which has acidFileSinks list of FileSinkDesc
+          /**
+           * todo QueryPlan has BaseSemanticAnalyzer which has acidFileSinks list of FileSinkDesc
            * FileSinkDesc.table is ql.metadata.Table
            * Table.tableSpec which is TableSpec, which has specType which is SpecType
            * So maybe this can work to know that this is part of dynamic partition insert in which case
@@ -862,8 +862,35 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
            */
           // For each component in this lock request,
           // add an entry to the txn_components table
-          // This must be done before HIVE_LOCKS is accessed
           for (LockComponent lc : rqst.getComponent()) {
+            if(lc.isSetIsAcid() && !lc.isIsAcid()) {
+              //we don't prevent using non-acid resources in a txn but we do lock them
+              continue;
+            }
+            boolean updateTxnComponents;
+            if(!lc.isSetOperationType()) {
+              //request came from old version of the client
+              updateTxnComponents = true;//this matches old behavior
+            }
+            else {
+              switch (lc.getOperationType()) {
+                case INSERT:
+                case UPDATE:
+                case DELETE:
+                  updateTxnComponents = true;
+                  break;
+                case SELECT:
+                  updateTxnComponents = false;
+                  break;
+                default:
+                  //since we have an open transaction, only 4 values above are expected 
+                  throw new IllegalStateException("Unexpected DataOperationType: " + lc.getOperationType()
+                    + " agentInfo=" + rqst.getAgentInfo() + " " + JavaUtils.txnIdToString(txnid));
+              }
+            }
+            if(!updateTxnComponents) {
+              continue;
+            }
             String dbName = lc.getDbname();
             String tblName = lc.getTablename();
             String partName = lc.getPartitionname();
@@ -872,14 +899,19 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
               "values (" + txnid + ", '" + dbName + "', " +
               (tblName == null ? "null" : "'" + tblName + "'") + ", " +
               (partName == null ? "null" : "'" + partName + "'")+ "," +
-              quoteString(OpertaionType.fromLockType(lc.getType()).toString()) + ")";
+              quoteString(OpertaionType.fromDataOperationType(lc.getOperationType()).toString()) + ")";
             LOG.debug("Going to execute update <" + s + ">");
-            stmt.executeUpdate(s);
+            int modCount = stmt.executeUpdate(s);
           }
         }
 
         long intLockId = 0;
         for (LockComponent lc : rqst.getComponent()) {
+          if(lc.isSetOperationType() && lc.getOperationType() == DataOperationType.UNSET) {
+            //old version of thrift client should have (lc.isSetOperationType() == false)
+            throw new IllegalStateException("Bug: operationType=" + lc.getOperationType() + " for component "
+              + lc + " agentInfo=" + rqst.getAgentInfo());
+          }
           intLockId++;
           String dbName = lc.getDbname();
           String tblName = lc.getTablename();
@@ -1454,21 +1486,13 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           ensureValidTxn(dbConn, rqst.getTxnid(), stmt);
           shouldNeverHappen(rqst.getTxnid());
         }
-        //we should be able to get this from AddDynamicPartitions object longer term; in fact we'd have to
-        //for multi stmt txns if same table is written more than once per tx
-        // MoveTask knows if it's I/U/D
-        // MoveTask calls Hive.loadDynamicPartitions() which calls HiveMetaStoreClient.addDynamicPartitions()
-        // which ends up here so we'd need to add a field to AddDynamicPartitions.
-        String findOperationType = " tc_operation_type from TXN_COMPONENTS where tc_txnid=" + rqst.getTxnid()
-          + " and tc_database=" + quoteString(rqst.getDbname()) + " and tc_table=" + quoteString(rqst.getTablename());
-        //do limit 1 on this; currently they will all have the same operations
-        rs = stmt.executeQuery(addLimitClause(1, findOperationType));
-        if(!rs.next()) {
-          throw new IllegalStateException("Unable to determine tc_operation_type for " + JavaUtils.txnIdToString(rqst.getTxnid()));
+        //for RU this may be null so we should default it to 'u' which is most restrictive
+        OpertaionType ot = OpertaionType.UPDATE;
+        if(rqst.isSetOperationType()) {
+          ot = OpertaionType.fromDataOperationType(rqst.getOperationType());
         }
-        OpertaionType ot = OpertaionType.fromString(rs.getString(1).charAt(0));
         
-        //what if a txn writes the same table > 1 time... let's go with this for now, but really
+        //what if a txn writes the same table > 1 time...(HIVE-9675) let's go with this for now, but really
         //need to not write this in the first place, i.e. make this delete not needed
         //see enqueueLockWithRetry() - that's where we write to TXN_COMPONENTS
         String deleteSql = "delete from TXN_COMPONENTS where tc_txnid=" + rqst.getTxnid() + " and tc_database=" +
@@ -1477,14 +1501,14 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
         //much "wider" than necessary in a lot of cases.  Here on the other hand, we know exactly which
         //partitions have been written to.  w/o this WRITE_SET would contain entries for partitions not actually
         //written to
-        stmt.executeUpdate(deleteSql);
+        int modCount = stmt.executeUpdate(deleteSql);
         for (String partName : rqst.getPartitionnames()) {
           String s =
             "insert into TXN_COMPONENTS (tc_txnid, tc_database, tc_table, tc_partition, tc_operation_type) values (" +
               rqst.getTxnid() + "," + quoteString(rqst.getDbname()) + "," + quoteString(rqst.getTablename()) +
               "," + quoteString(partName) + "," + quoteChar(ot.sqlConst) + ")";
           LOG.debug("Going to execute update <" + s + ">");
-          stmt.executeUpdate(s);
+          modCount = stmt.executeUpdate(s);
         }
         LOG.debug("Going to commit");
         dbConn.commit();
@@ -1504,8 +1528,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
   }
 
   /**
-   * Clean up corresponding records in metastore tables, specifically:
-   * TXN_COMPONENTS, COMPLETED_TXN_COMPONENTS, COMPACTION_QUEUE, COMPLETED_COMPACTIONS
+   * Clean up corresponding records in metastore tables when corresponding object is dropped,
+   * specifically: TXN_COMPONENTS, COMPLETED_TXN_COMPONENTS, COMPACTION_QUEUE, COMPLETED_COMPACTIONS
    */
   @Override
   public void cleanupRecords(HiveObjectType type, Database db, Table table,

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
index fc00e6d..23ad54e 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.hive.metastore.txn;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.log4j.Level;
@@ -270,12 +268,14 @@ public class TestCompactionTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB,
         "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB,
         "mydb");
     comp.setTablename("yourtable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.UPDATE);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
     req.setTxnid(txnid);
@@ -306,6 +306,7 @@ public class TestCompactionTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB,
         "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.INSERT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -317,6 +318,7 @@ public class TestCompactionTxnHandler {
     txnid = openTxn();
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("yourtable");
+    comp.setOperationType(DataOperationType.DELETE);
     components = new ArrayList<LockComponent>(1);
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -329,6 +331,7 @@ public class TestCompactionTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("foo");
     comp.setPartitionname("bar");
+    comp.setOperationType(DataOperationType.UPDATE);
     components = new ArrayList<LockComponent>(1);
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -339,6 +342,7 @@ public class TestCompactionTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("foo");
     comp.setPartitionname("baz");
+    comp.setOperationType(DataOperationType.UPDATE);
     components = new ArrayList<LockComponent>(1);
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -395,13 +399,17 @@ public class TestCompactionTxnHandler {
     // lock a table, as in dynamic partitions
     LockComponent lc = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, dbName);
     lc.setTablename(tableName);
+    DataOperationType dop = DataOperationType.UPDATE; 
+    lc.setOperationType(dop);
     LockRequest lr = new LockRequest(Arrays.asList(lc), "me", "localhost");
     lr.setTxnid(txnId);
     LockResponse lock = txnHandler.lock(lr);
     assertEquals(LockState.ACQUIRED, lock.getState());
 
-    txnHandler.addDynamicPartitions(new AddDynamicPartitions(txnId, dbName, tableName,
-        Arrays.asList("ds=yesterday", "ds=today")));
+    AddDynamicPartitions adp = new AddDynamicPartitions(txnId, dbName, tableName,
+      Arrays.asList("ds=yesterday", "ds=today"));
+    adp.setOperationType(dop);
+    txnHandler.addDynamicPartitions(adp);
     txnHandler.commitTxn(new CommitTxnRequest(txnId));
 
     Set<CompactionInfo> potentials = txnHandler.findPotentialCompactions(1000);

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
index ccaf91c..0d4fc59 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
@@ -177,6 +177,7 @@ public class TestTxnHandler {
   public void testLockDifferentDBs() throws Exception {
     // Test that two different databases don't collide on their locks
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -184,6 +185,7 @@ public class TestTxnHandler {
     assertTrue(res.getState() == LockState.ACQUIRED);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "yourdb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -195,6 +197,7 @@ public class TestTxnHandler {
   public void testLockSameDB() throws Exception {
     // Test that two different databases don't collide on their locks
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -202,6 +205,7 @@ public class TestTxnHandler {
     assertTrue(res.getState() == LockState.ACQUIRED);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -213,6 +217,7 @@ public class TestTxnHandler {
   public void testLockDbLocksTable() throws Exception {
     // Test that locking a database prevents locking of tables in the database
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -220,6 +225,7 @@ public class TestTxnHandler {
     assertTrue(res.getState() == LockState.ACQUIRED);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     comp.setTablename("mytable");
     components.clear();
     components.add(comp);
@@ -232,6 +238,7 @@ public class TestTxnHandler {
   public void testLockDbDoesNotLockTableInDifferentDB() throws Exception {
     // Test that locking a database prevents locking of tables in the database
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -239,6 +246,7 @@ public class TestTxnHandler {
     assertTrue(res.getState() == LockState.ACQUIRED);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "yourdb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     comp.setTablename("mytable");
     components.clear();
     components.add(comp);
@@ -251,6 +259,7 @@ public class TestTxnHandler {
   public void testLockDifferentTables() throws Exception {
     // Test that two different tables don't collide on their locks
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     comp.setTablename("mytable");
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
@@ -259,6 +268,7 @@ public class TestTxnHandler {
     assertTrue(res.getState() == LockState.ACQUIRED);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     comp.setTablename("yourtable");
     components.clear();
     components.add(comp);
@@ -272,6 +282,7 @@ public class TestTxnHandler {
     // Test that two different tables don't collide on their locks
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -280,6 +291,7 @@ public class TestTxnHandler {
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -292,6 +304,7 @@ public class TestTxnHandler {
     // Test that locking a table prevents locking of partitions of the table
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -301,6 +314,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -313,6 +327,7 @@ public class TestTxnHandler {
     // Test that locking a table prevents locking of partitions of the table
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -322,6 +337,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("yourtable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -335,6 +351,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -344,6 +361,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("yourpartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -357,6 +375,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -366,6 +385,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -379,6 +399,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.INSERT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -388,6 +409,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -401,6 +423,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -410,6 +433,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.INSERT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -419,6 +443,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -432,6 +457,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.INSERT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -441,6 +467,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -455,6 +482,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -464,6 +492,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -473,6 +502,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.UPDATE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -487,6 +517,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -496,6 +527,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -509,6 +541,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -518,6 +551,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -527,6 +561,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -540,6 +575,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -550,6 +586,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -563,6 +600,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -573,6 +611,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -583,6 +622,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.INSERT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -591,11 +631,31 @@ public class TestTxnHandler {
   }
 
   @Test
+  public void testWrongLockForOperation() throws Exception {
+    LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
+    comp.setTablename("mytable");
+    comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
+    List<LockComponent> components = new ArrayList<LockComponent>(1);
+    components.add(comp);
+    LockRequest req = new LockRequest(components, "me", "localhost");
+    req.setTxnid(openTxn());
+    Exception expectedError = null;
+    try {
+      LockResponse res = txnHandler.lock(req);
+    }
+    catch(Exception e) {
+      expectedError = e;
+    }
+    Assert.assertTrue(expectedError != null && expectedError.getMessage().contains("Unexpected DataOperationType"));
+  }
+  @Test
   public void testLockSWSWSW() throws Exception {
     // Test that write blocks two writes
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -606,6 +666,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -616,6 +677,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -630,6 +692,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -639,6 +702,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -648,6 +712,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -662,6 +727,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -671,6 +737,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -680,6 +747,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.SELECT);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -692,6 +760,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -704,6 +773,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.UPDATE);
     components.clear();
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -733,6 +803,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -755,12 +826,14 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(2);
     components.add(comp);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("anotherpartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
     LockResponse res = txnHandler.lock(req);
@@ -778,12 +851,14 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(2);
     components.add(comp);
 
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("anotherpartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
     LockResponse res = txnHandler.lock(req);
@@ -794,6 +869,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     components = new ArrayList<LockComponent>(1);
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -813,6 +889,7 @@ public class TestTxnHandler {
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB,  "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -828,6 +905,7 @@ public class TestTxnHandler {
     // Test that committing unlocks
     long txnid = openTxn();
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.UPDATE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -847,6 +925,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.SHARED_WRITE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.DELETE);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -893,6 +972,7 @@ public class TestTxnHandler {
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");
     comp.setPartitionname("mypartition");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -957,6 +1037,7 @@ public class TestTxnHandler {
       LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
       comp.setTablename("mytable");
       comp.setPartitionname("mypartition");
+      comp.setOperationType(DataOperationType.NO_TXN);
       List<LockComponent> components = new ArrayList<LockComponent>(1);
       components.add(comp);
       LockRequest req = new LockRequest(components, "me", "localhost");
@@ -1044,6 +1125,7 @@ public class TestTxnHandler {
   public void showLocks() throws Exception {
     long begining = System.currentTimeMillis();
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
+    comp.setOperationType(DataOperationType.NO_TXN);
     List<LockComponent> components = new ArrayList<LockComponent>(1);
     components.add(comp);
     LockRequest req = new LockRequest(components, "me", "localhost");
@@ -1053,6 +1135,7 @@ public class TestTxnHandler {
     long txnid = openTxn();
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "mydb");
     comp.setTablename("mytable");
+    comp.setOperationType(DataOperationType.SELECT);
     components = new ArrayList<LockComponent>(1);
     components.add(comp);
     req = new LockRequest(components, "me", "localhost");
@@ -1064,6 +1147,7 @@ public class TestTxnHandler {
     comp = new LockComponent(LockType.SHARED_READ, LockLevel.PARTITION, "yourdb");
     comp.setTablename("yourtable");
     comp.setPartitionname("yourpartition");
+    comp.setOperationType(DataOperationType.INSERT);
     components.add(comp);
     req = new LockRequest(components, "you", "remotehost");
     res = txnHandler.lock(req);

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

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 15a61d6..cf93bc5 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
@@ -28,12 +28,11 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.OutputFormat;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -219,7 +218,20 @@ public class AcidUtils {
     return result;
   }
 
-  public enum Operation { NOT_ACID, INSERT, UPDATE, DELETE }
+  public enum Operation {
+    NOT_ACID(DataOperationType.UNSET),
+    INSERT(DataOperationType.INSERT),
+    UPDATE(DataOperationType.UPDATE),
+    DELETE(DataOperationType.DELETE);
+    
+    private final DataOperationType dop;
+    private Operation(DataOperationType dop) {
+      this.dop = dop;
+    }
+    public DataOperationType toDataOperationType() {
+      return dop;
+    }
+  }
 
   public static interface Directory {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
index 21b0cb2..5a7ed17 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/DbTxnManager.java
@@ -161,7 +161,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
 
     boolean atLeastOneLock = false;
 
-    LockRequestBuilder rqstBuilder = new LockRequestBuilder();
+    LockRequestBuilder rqstBuilder = new LockRequestBuilder(plan.getQueryId());
     //link queryId to txnId
     LOG.info("Setting lock request transaction to " + JavaUtils.txnIdToString(txnId) + " for queryId=" + plan.getQueryId());
     rqstBuilder.setTransactionId(txnId)
@@ -177,6 +177,7 @@ public class DbTxnManager extends HiveTxnManagerImpl {
       }
       LockComponentBuilder compBuilder = new LockComponentBuilder();
       compBuilder.setShared();
+      compBuilder.setOperationType(DataOperationType.SELECT);
 
       Table t = null;
       switch (input.getType()) {
@@ -202,6 +203,9 @@ public class DbTxnManager extends HiveTxnManagerImpl {
           // This is a file or something we don't hold locks for.
           continue;
       }
+      if(t != null && AcidUtils.isAcidTable(t)) {
+        compBuilder.setIsAcid(true);
+      }
       LockComponent comp = compBuilder.build();
       LOG.debug("Adding lock component to lock request " + comp.toString());
       rqstBuilder.addLockComponent(comp);
@@ -225,27 +229,35 @@ public class DbTxnManager extends HiveTxnManagerImpl {
         case DDL_EXCLUSIVE:
         case INSERT_OVERWRITE:
           compBuilder.setExclusive();
+          compBuilder.setOperationType(DataOperationType.NO_TXN);
           break;
 
         case INSERT:
-          t = output.getTable();
-          if(t == null) {
-            throw new IllegalStateException("No table info for " + output);
-          }
+          t = getTable(output);
           if(AcidUtils.isAcidTable(t)) {
             compBuilder.setShared();
+            compBuilder.setIsAcid(true);
           }
           else {
             compBuilder.setExclusive();
+            compBuilder.setIsAcid(false);
           }
+          compBuilder.setOperationType(DataOperationType.INSERT);
           break;
         case DDL_SHARED:
           compBuilder.setShared();
+          compBuilder.setOperationType(DataOperationType.NO_TXN);
           break;
 
         case UPDATE:
+          compBuilder.setSemiShared();
+          compBuilder.setOperationType(DataOperationType.UPDATE);
+          t = getTable(output);
+          break;
         case DELETE:
           compBuilder.setSemiShared();
+          compBuilder.setOperationType(DataOperationType.DELETE);
+          t = getTable(output);
           break;
 
         case DDL_NO_LOCK:
@@ -279,12 +291,15 @@ public class DbTxnManager extends HiveTxnManagerImpl {
           // This is a file or something we don't hold locks for.
           continue;
       }
+      if(t != null && AcidUtils.isAcidTable(t)) {
+        compBuilder.setIsAcid(true);
+      }
       LockComponent comp = compBuilder.build();
       LOG.debug("Adding lock component to lock request " + comp.toString());
       rqstBuilder.addLockComponent(comp);
       atLeastOneLock = true;
     }
-
+    //plan
     // Make sure we need locks.  It's possible there's nothing to lock in
     // this operation.
     if (!atLeastOneLock) {
@@ -300,6 +315,13 @@ public class DbTxnManager extends HiveTxnManagerImpl {
     ctx.setHiveLocks(locks);
     return lockState;
   }
+  private static Table getTable(WriteEntity we) {
+    Table t = we.getTable();
+    if(t == null) {
+      throw new IllegalStateException("No table info for " + we);
+    }
+    return t;
+  }
   /**
    * This is for testing only.
    * @param delay time to delay for first heartbeat

http://git-wip-us.apache.org/repos/asf/hive/blob/c0b532fc/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index 20e1ef6..a67f23a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -1540,7 +1540,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
    */
   public Map<Map<String, String>, Partition> loadDynamicPartitions(Path loadPath,
       String tableName, Map<String, String> partSpec, boolean replace,
-      int numDP, boolean holdDDLTime, boolean listBucketingEnabled, boolean isAcid, long txnId)
+      int numDP, boolean holdDDLTime, boolean listBucketingEnabled, boolean isAcid, long txnId,
+      AcidUtils.Operation operation)
       throws HiveException {
 
     Set<Path> validPartitions = new HashSet<Path>();
@@ -1603,7 +1604,8 @@ private void constructOneLBLocationMap(FileStatus fSta,
         for (Partition p : partitionsMap.values()) {
           partNames.add(p.getName());
         }
-        metaStoreClient.addDynamicPartitions(txnId, tbl.getDbName(), tbl.getTableName(), partNames);
+        metaStoreClient.addDynamicPartitions(txnId, tbl.getDbName(), tbl.getTableName(),
+          partNames, operation.toDataOperationType());
       }
       return partitionsMap;
     } catch (IOException e) {