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 2018/03/15 18:19:24 UTC

hive git commit: HIVE-18693 - Snapshot Isolation does not work for Micromanaged table when a insert transaction is aborted (Steve Yeom via Eugene Koifman)

Repository: hive
Updated Branches:
  refs/heads/master d5cb7f62e -> 52c6e8938


HIVE-18693 - Snapshot Isolation does not work for Micromanaged table when a insert transaction is aborted (Steve Yeom via Eugene Koifman)


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

Branch: refs/heads/master
Commit: 52c6e8938b7b0c6179b9c8208af2d77ca6ad0c1c
Parents: d5cb7f6
Author: Eugene Koifman <ek...@apache.org>
Authored: Thu Mar 15 11:19:15 2018 -0700
Committer: Eugene Koifman <ek...@apache.org>
Committed: Thu Mar 15 11:19:15 2018 -0700

----------------------------------------------------------------------
 .../streaming/mutate/client/lock/Lock.java      |   4 +-
 .../streaming/mutate/client/lock/TestLock.java  |  10 +-
 .../hadoop/hive/ql/lockmgr/DbTxnManager.java    |   4 +-
 .../hadoop/hive/ql/txn/compactor/Initiator.java |  13 +-
 .../hive/ql/TestTxnCommandsForMmTable.java      | 121 ++++++++++++++++++-
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  24 ++--
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  14 +--
 .../hive/metastore/api/LockComponent.java       | 112 ++++++++---------
 .../src/gen/thrift/gen-php/metastore/Types.php  |  16 +--
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  18 +--
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   4 +-
 .../hive/metastore/LockComponentBuilder.java    |   4 +-
 .../hadoop/hive/metastore/txn/TxnHandler.java   |   5 +-
 .../src/main/thrift/hive_metastore.thrift       |   2 +-
 14 files changed, 236 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
index 429555f..abbb125 100644
--- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
@@ -182,9 +182,9 @@ public class Lock {
       //todo: DataOperationType is set conservatively here, we'd really want to distinguish update/delete
       //and insert/select and if resource (that is written to) is ACID or not
       if (sinks.contains(table)) {
-        componentBuilder.setSemiShared().setOperationType(DataOperationType.UPDATE).setIsFullAcid(true);
+        componentBuilder.setSemiShared().setOperationType(DataOperationType.UPDATE).setIsTransactional(true);
       } else {
-        componentBuilder.setShared().setOperationType(DataOperationType.INSERT).setIsFullAcid(true);
+        componentBuilder.setShared().setOperationType(DataOperationType.INSERT).setIsTransactional(true);
       }
       LockComponent component = componentBuilder.build();
       requestBuilder.addLockComponent(component);

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
index e454942..0a46faf 100644
--- a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
@@ -176,13 +176,13 @@ public class TestLock {
     LockComponent expected1 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
     expected1.setTablename("SOURCE_1");
     expected1.setOperationType(DataOperationType.INSERT);
-    expected1.setIsAcid(true);
+    expected1.setIsTransactional(true);
     assertTrue(components.contains(expected1));
 
     LockComponent expected2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
     expected2.setTablename("SOURCE_2");
     expected2.setOperationType(DataOperationType.INSERT);
-    expected2.setIsAcid(true);
+    expected2.setIsTransactional(true);
     assertTrue(components.contains(expected2));
   }
 
@@ -203,19 +203,19 @@ public class TestLock {
     LockComponent expected1 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
     expected1.setTablename("SOURCE_1");
     expected1.setOperationType(DataOperationType.INSERT);
-    expected1.setIsAcid(true);
+    expected1.setIsTransactional(true);
     assertTrue(components.contains(expected1));
 
     LockComponent expected2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
     expected2.setTablename("SOURCE_2");
     expected2.setOperationType(DataOperationType.INSERT);
-    expected2.setIsAcid(true);
+    expected2.setIsTransactional(true);
     assertTrue(components.contains(expected2));
 
     LockComponent expected3 = new LockComponent(LockType.SHARED_WRITE, LockLevel.TABLE, "DB");
     expected3.setTablename("SINK");
     expected3.setOperationType(DataOperationType.UPDATE);
-    expected3.setIsAcid(true);
+    expected3.setIsTransactional(true);
     assertTrue(components.contains(expected3));
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/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 683aa95..89ca1ff 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
@@ -441,7 +441,7 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
           continue;
       }
       if(t != null) {
-        compBuilder.setIsFullAcid(AcidUtils.isFullAcidTable(t));
+        compBuilder.setIsTransactional(AcidUtils.isTransactionalTable(t));
       }
       LockComponent comp = compBuilder.build();
       LOG.debug("Adding lock component to lock request " + comp.toString());
@@ -539,7 +539,7 @@ public final class DbTxnManager extends HiveTxnManagerImpl {
               output.getWriteType().toString());
       }
       if(t != null) {
-        compBuilder.setIsFullAcid(AcidUtils.isFullAcidTable(t));
+        compBuilder.setIsTransactional(AcidUtils.isTransactionalTable(t));
       }
 
       compBuilder.setIsDynamicPartitionWrite(output.isDynamicPartitionWrite());

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
index 7eda7fb..a04ac3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
@@ -157,7 +157,8 @@ public class Initiator extends CompactorThread {
               /*Future thought: checkForCompaction will check a lot of file metadata and may be expensive.
               * Long term we should consider having a thread pool here and running checkForCompactionS
               * in parallel*/
-              CompactionType compactionNeeded = checkForCompaction(ci, tblValidWriteIds, sd, t.getParameters(), runAs);
+              CompactionType compactionNeeded
+                      = checkForCompaction(ci, tblValidWriteIds, sd, t.getParameters(), runAs);
               if (compactionNeeded != null) requestCompaction(ci, runAs, compactionNeeded);
             } catch (Throwable t) {
               LOG.error("Caught exception while trying to determine if we should compact " +
@@ -235,6 +236,12 @@ public class Initiator extends CompactorThread {
           "initiating major compaction");
       return CompactionType.MAJOR;
     }
+
+    // If it is for insert-only transactional table, return null.
+    if (AcidUtils.isInsertOnlyTable(tblproperties)) {
+      return null;
+    }
+
     if (runJobAsSelf(runAs)) {
       return determineCompactionType(ci, writeIds, sd, tblproperties);
     } else {
@@ -261,10 +268,6 @@ public class Initiator extends CompactorThread {
                                                  StorageDescriptor sd, Map<String, String> tblproperties)
       throws IOException, InterruptedException {
 
-    if (AcidUtils.isInsertOnlyTable(tblproperties)) {
-      return CompactionType.MINOR;
-    }
-
     boolean noBase = false;
     Path location = new Path(sd.getLocation());
     FileSystem fs = location.getFileSystem(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java
index d4f1dd5..c053860 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommandsForMmTable.java
@@ -591,16 +591,133 @@ public class TestTxnCommandsForMmTable extends TxnCommandsBaseForTests {
     Assert.assertEquals(stringifyValues(rExpected), rs);
   }
 
+  @Test
+  public void testOperationsOnCompletedTxnComponentsForMmTable() throws Exception {
+
+    // Insert two rows into the table.
+    runStatementOnDriver("insert into " + TableExtended.MMTBL + "(a,b) values(1,2)");
+    runStatementOnDriver("insert into " + TableExtended.MMTBL + "(a,b) values(3,4)");
+    // There should be 2 delta directories
+    verifyDirAndResult(2);
+
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPLETED_TXN_COMPONENTS"),
+            2, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPLETED_TXN_COMPONENTS"));
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"),
+            0, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS"));
+
+    // Initiate a minor compaction request on the table.
+    runStatementOnDriver("alter table " + TableExtended.MMTBL  + " compact 'MAJOR'");
+
+    // Run worker.
+    runWorker(hiveConf);
+
+    // Run Cleaner.
+    runCleaner(hiveConf);
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from COMPLETED_TXN_COMPONENTS"),
+            0,
+            TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from COMPLETED_TXN_COMPONENTS"));
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"),
+            0, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS"));
+  }
+
+  @Test
+  public void testSnapshotIsolationWithAbortedTxnOnMmTable() throws Exception {
+
+    // Insert two rows into the table.
+    runStatementOnDriver("insert into " + TableExtended.MMTBL + "(a,b) values(1,2)");
+    runStatementOnDriver("insert into " + TableExtended.MMTBL + "(a,b) values(3,4)");
+    // There should be 2 delta directories
+    verifyDirAndResult(2);
+
+    // Initiate a minor compaction request on the table.
+    runStatementOnDriver("alter table " + TableExtended.MMTBL  + " compact 'MINOR'");
+
+    // Run Compaction Worker to do compaction.
+    // But we do not compact a MM table but only transit the compaction request to
+    // "ready for cleaning" state in this case.
+    runWorker(hiveConf);
+    verifyDirAndResult(2);
+
+    // Start an INSERT statement transaction and roll back this transaction.
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, true);
+    runStatementOnDriver("insert into " + TableExtended.MMTBL  + " values (5, 6)");
+    hiveConf.setBoolVar(HiveConf.ConfVars.HIVETESTMODEROLLBACKTXN, false);
+    // There should be 3 delta directories. The new one is the aborted one.
+    verifyDirAndResult(3);
+
+    // Execute SELECT statement and verify the result set (should be two rows).
+    int[][] expected = new int[][] {{1, 2}, {3, 4}};
+    List<String> rs = runStatementOnDriver("select a,b from " + TableExtended.MMTBL + " order by a,b");
+    Assert.assertEquals(stringifyValues(expected), rs);
+
+    // Run Cleaner.
+    // This run doesn't do anything for the above aborted transaction since
+    // the current compaction request entry in the compaction queue is updated
+    // to have highest_write_id when the worker is run before the aborted
+    // transaction. Specifically the id is 2 for the entry but the aborted
+    // transaction has 3 as writeId. This run does transition the entry
+    // "successful".
+    runCleaner(hiveConf);
+    verifyDirAndResult(3);
+
+    // Execute SELECT and verify that aborted operation is not counted for MM table.
+    rs = runStatementOnDriver("select a,b from " + TableExtended.MMTBL + " order by a,b");
+    Assert.assertEquals(stringifyValues(expected), rs);
+
+    // Run initiator to execute CompactionTxnHandler.cleanEmptyAbortedTxns()
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"),
+            1, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS"));
+    Initiator i = new Initiator();
+    i.setThreadId((int)i.getId());
+    i.setConf(hiveConf);
+    AtomicBoolean stop = new AtomicBoolean(true);
+    i.init(stop, new AtomicBoolean());
+    i.run();
+    // This run of Initiator doesn't add any compaction_queue entry
+    // since we only have one MM table with data - we don't compact MM tables.
+    verifyDirAndResult(3);
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"),
+            1, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS"));
+
+    // Execute SELECT statement and verify that aborted INSERT statement is not counted.
+    rs = runStatementOnDriver("select a,b from " + TableExtended.MMTBL + " order by a,b");
+    Assert.assertEquals(stringifyValues(expected), rs);
+
+    // Initiate a minor compaction request on the table.
+    runStatementOnDriver("alter table " + TableExtended.MMTBL  + " compact 'MINOR'");
+
+    // Run worker to delete aborted transaction's delta directory.
+    runWorker(hiveConf);
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"),
+            1, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS"));
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"),
+            1,
+            TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"));
+    verifyDirAndResult(2);
+
+    // Run Cleaner to delete rows for the aborted transaction
+    // from TXN_COMPONENTS.
+    runCleaner(hiveConf);
+
+    // Run initiator to clean the row fro the aborted transaction from TXNS.
+    i.run();
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXNS"),
+            0, TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXNS"));
+    Assert.assertEquals(TxnDbUtil.queryToString(hiveConf, "select * from TXN_COMPONENTS"),
+            0,
+            TxnDbUtil.countQueryAgent(hiveConf, "select count(*) from TXN_COMPONENTS"));
+  }
+
   private void verifyDirAndResult(int expectedDeltas) throws Exception {
     FileSystem fs = FileSystem.get(hiveConf);
     // Verify the content of subdirs
     FileStatus[] status = fs.listStatus(new Path(TEST_WAREHOUSE_DIR + "/" +
-        (TableExtended.MMTBL).toString().toLowerCase()), FileUtils.STAGING_DIR_PATH_FILTER);
+        (TableExtended.MMTBL).toString().toLowerCase()), FileUtils.HIDDEN_FILES_PATH_FILTER);
     int sawDeltaTimes = 0;
     for (int i = 0; i < status.length; i++) {
       Assert.assertTrue(status[i].getPath().getName().matches("delta_.*"));
       sawDeltaTimes++;
-      FileStatus[] files = fs.listStatus(status[i].getPath(), FileUtils.STAGING_DIR_PATH_FILTER);
+      FileStatus[] files = fs.listStatus(status[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
       Assert.assertEquals(1, files.length);
       Assert.assertTrue(files[0].getPath().getName().equals("000000_0"));
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 4a7c6bf..620d6ef 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -15554,9 +15554,9 @@ void LockComponent::__set_operationType(const DataOperationType::type val) {
 __isset.operationType = true;
 }
 
-void LockComponent::__set_isAcid(const bool val) {
-  this->isAcid = val;
-__isset.isAcid = true;
+void LockComponent::__set_isTransactional(const bool val) {
+  this->isTransactional = val;
+__isset.isTransactional = true;
 }
 
 void LockComponent::__set_isDynamicPartitionWrite(const bool val) {
@@ -15644,8 +15644,8 @@ uint32_t LockComponent::read(::apache::thrift::protocol::TProtocol* iprot) {
         break;
       case 7:
         if (ftype == ::apache::thrift::protocol::T_BOOL) {
-          xfer += iprot->readBool(this->isAcid);
-          this->__isset.isAcid = true;
+          xfer += iprot->readBool(this->isTransactional);
+          this->__isset.isTransactional = true;
         } else {
           xfer += iprot->skip(ftype);
         }
@@ -15708,9 +15708,9 @@ uint32_t LockComponent::write(::apache::thrift::protocol::TProtocol* oprot) cons
     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);
+  if (this->__isset.isTransactional) {
+    xfer += oprot->writeFieldBegin("isTransactional", ::apache::thrift::protocol::T_BOOL, 7);
+    xfer += oprot->writeBool(this->isTransactional);
     xfer += oprot->writeFieldEnd();
   }
   if (this->__isset.isDynamicPartitionWrite) {
@@ -15731,7 +15731,7 @@ void swap(LockComponent &a, LockComponent &b) {
   swap(a.tablename, b.tablename);
   swap(a.partitionname, b.partitionname);
   swap(a.operationType, b.operationType);
-  swap(a.isAcid, b.isAcid);
+  swap(a.isTransactional, b.isTransactional);
   swap(a.isDynamicPartitionWrite, b.isDynamicPartitionWrite);
   swap(a.__isset, b.__isset);
 }
@@ -15743,7 +15743,7 @@ LockComponent::LockComponent(const LockComponent& other673) {
   tablename = other673.tablename;
   partitionname = other673.partitionname;
   operationType = other673.operationType;
-  isAcid = other673.isAcid;
+  isTransactional = other673.isTransactional;
   isDynamicPartitionWrite = other673.isDynamicPartitionWrite;
   __isset = other673.__isset;
 }
@@ -15754,7 +15754,7 @@ LockComponent& LockComponent::operator=(const LockComponent& other674) {
   tablename = other674.tablename;
   partitionname = other674.partitionname;
   operationType = other674.operationType;
-  isAcid = other674.isAcid;
+  isTransactional = other674.isTransactional;
   isDynamicPartitionWrite = other674.isDynamicPartitionWrite;
   __isset = other674.__isset;
   return *this;
@@ -15768,7 +15768,7 @@ void LockComponent::printTo(std::ostream& out) const {
   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 << ", " << "isTransactional="; (__isset.isTransactional ? (out << to_string(isTransactional)) : (out << "<null>"));
   out << ", " << "isDynamicPartitionWrite="; (__isset.isDynamicPartitionWrite ? (out << to_string(isDynamicPartitionWrite)) : (out << "<null>"));
   out << ")";
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index ef8b694..05a7a29 100644
--- a/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/standalone-metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -6477,11 +6477,11 @@ inline std::ostream& operator<<(std::ostream& out, const AllocateTableWriteIdsRe
 }
 
 typedef struct _LockComponent__isset {
-  _LockComponent__isset() : tablename(false), partitionname(false), operationType(true), isAcid(true), isDynamicPartitionWrite(true) {}
+  _LockComponent__isset() : tablename(false), partitionname(false), operationType(true), isTransactional(true), isDynamicPartitionWrite(true) {}
   bool tablename :1;
   bool partitionname :1;
   bool operationType :1;
-  bool isAcid :1;
+  bool isTransactional :1;
   bool isDynamicPartitionWrite :1;
 } _LockComponent__isset;
 
@@ -6490,7 +6490,7 @@ class LockComponent {
 
   LockComponent(const LockComponent&);
   LockComponent& operator=(const LockComponent&);
-  LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname(), operationType((DataOperationType::type)5), isAcid(false), isDynamicPartitionWrite(false) {
+  LockComponent() : type((LockType::type)0), level((LockLevel::type)0), dbname(), tablename(), partitionname(), operationType((DataOperationType::type)5), isTransactional(false), isDynamicPartitionWrite(false) {
     operationType = (DataOperationType::type)5;
 
   }
@@ -6502,7 +6502,7 @@ class LockComponent {
   std::string tablename;
   std::string partitionname;
   DataOperationType::type operationType;
-  bool isAcid;
+  bool isTransactional;
   bool isDynamicPartitionWrite;
 
   _LockComponent__isset __isset;
@@ -6519,7 +6519,7 @@ class LockComponent {
 
   void __set_operationType(const DataOperationType::type val);
 
-  void __set_isAcid(const bool val);
+  void __set_isTransactional(const bool val);
 
   void __set_isDynamicPartitionWrite(const bool val);
 
@@ -6543,9 +6543,9 @@ class LockComponent {
       return false;
     else if (__isset.operationType && !(operationType == rhs.operationType))
       return false;
-    if (__isset.isAcid != rhs.__isset.isAcid)
+    if (__isset.isTransactional != rhs.__isset.isTransactional)
       return false;
-    else if (__isset.isAcid && !(isAcid == rhs.isAcid))
+    else if (__isset.isTransactional && !(isTransactional == rhs.isTransactional))
       return false;
     if (__isset.isDynamicPartitionWrite != rhs.__isset.isDynamicPartitionWrite)
       return false;

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
index 0307540..77de5c9 100644
--- a/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
+++ b/standalone-metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/LockComponent.java
@@ -44,7 +44,7 @@ import org.slf4j.LoggerFactory;
   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 org.apache.thrift.protocol.TField IS_TRANSACTIONAL_FIELD_DESC = new org.apache.thrift.protocol.TField("isTransactional", org.apache.thrift.protocol.TType.BOOL, (short)7);
   private static final org.apache.thrift.protocol.TField IS_DYNAMIC_PARTITION_WRITE_FIELD_DESC = new org.apache.thrift.protocol.TField("isDynamicPartitionWrite", org.apache.thrift.protocol.TType.BOOL, (short)8);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
@@ -59,7 +59,7 @@ import org.slf4j.LoggerFactory;
   private String tablename; // optional
   private String partitionname; // optional
   private DataOperationType operationType; // optional
-  private boolean isAcid; // optional
+  private boolean isTransactional; // optional
   private boolean isDynamicPartitionWrite; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -82,7 +82,7 @@ import org.slf4j.LoggerFactory;
      * @see DataOperationType
      */
     OPERATION_TYPE((short)6, "operationType"),
-    IS_ACID((short)7, "isAcid"),
+    IS_TRANSACTIONAL((short)7, "isTransactional"),
     IS_DYNAMIC_PARTITION_WRITE((short)8, "isDynamicPartitionWrite");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
@@ -110,8 +110,8 @@ import org.slf4j.LoggerFactory;
           return PARTITIONNAME;
         case 6: // OPERATION_TYPE
           return OPERATION_TYPE;
-        case 7: // IS_ACID
-          return IS_ACID;
+        case 7: // IS_TRANSACTIONAL
+          return IS_TRANSACTIONAL;
         case 8: // IS_DYNAMIC_PARTITION_WRITE
           return IS_DYNAMIC_PARTITION_WRITE;
         default:
@@ -154,10 +154,10 @@ import org.slf4j.LoggerFactory;
   }
 
   // isset id assignments
-  private static final int __ISACID_ISSET_ID = 0;
+  private static final int __ISTRANSACTIONAL_ISSET_ID = 0;
   private static final int __ISDYNAMICPARTITIONWRITE_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
-  private static final _Fields optionals[] = {_Fields.TABLENAME,_Fields.PARTITIONNAME,_Fields.OPERATION_TYPE,_Fields.IS_ACID,_Fields.IS_DYNAMIC_PARTITION_WRITE};
+  private static final _Fields optionals[] = {_Fields.TABLENAME,_Fields.PARTITIONNAME,_Fields.OPERATION_TYPE,_Fields.IS_TRANSACTIONAL,_Fields.IS_DYNAMIC_PARTITION_WRITE};
   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);
@@ -173,7 +173,7 @@ import org.slf4j.LoggerFactory;
         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, 
+    tmpMap.put(_Fields.IS_TRANSACTIONAL, new org.apache.thrift.meta_data.FieldMetaData("isTransactional", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
     tmpMap.put(_Fields.IS_DYNAMIC_PARTITION_WRITE, new org.apache.thrift.meta_data.FieldMetaData("isDynamicPartitionWrite", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
@@ -184,7 +184,7 @@ import org.slf4j.LoggerFactory;
   public LockComponent() {
     this.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.UNSET;
 
-    this.isAcid = false;
+    this.isTransactional = false;
 
     this.isDynamicPartitionWrite = false;
 
@@ -224,7 +224,7 @@ import org.slf4j.LoggerFactory;
     if (other.isSetOperationType()) {
       this.operationType = other.operationType;
     }
-    this.isAcid = other.isAcid;
+    this.isTransactional = other.isTransactional;
     this.isDynamicPartitionWrite = other.isDynamicPartitionWrite;
   }
 
@@ -241,7 +241,7 @@ import org.slf4j.LoggerFactory;
     this.partitionname = null;
     this.operationType = org.apache.hadoop.hive.metastore.api.DataOperationType.UNSET;
 
-    this.isAcid = false;
+    this.isTransactional = false;
 
     this.isDynamicPartitionWrite = false;
 
@@ -409,26 +409,26 @@ import org.slf4j.LoggerFactory;
     }
   }
 
-  public boolean isIsAcid() {
-    return this.isAcid;
+  public boolean isIsTransactional() {
+    return this.isTransactional;
   }
 
-  public void setIsAcid(boolean isAcid) {
-    this.isAcid = isAcid;
-    setIsAcidIsSet(true);
+  public void setIsTransactional(boolean isTransactional) {
+    this.isTransactional = isTransactional;
+    setIsTransactionalIsSet(true);
   }
 
-  public void unsetIsAcid() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISACID_ISSET_ID);
+  public void unsetIsTransactional() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ISTRANSACTIONAL_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);
+  /** Returns true if field isTransactional is set (has been assigned a value) and false otherwise */
+  public boolean isSetIsTransactional() {
+    return EncodingUtils.testBit(__isset_bitfield, __ISTRANSACTIONAL_ISSET_ID);
   }
 
-  public void setIsAcidIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISACID_ISSET_ID, value);
+  public void setIsTransactionalIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ISTRANSACTIONAL_ISSET_ID, value);
   }
 
   public boolean isIsDynamicPartitionWrite() {
@@ -503,11 +503,11 @@ import org.slf4j.LoggerFactory;
       }
       break;
 
-    case IS_ACID:
+    case IS_TRANSACTIONAL:
       if (value == null) {
-        unsetIsAcid();
+        unsetIsTransactional();
       } else {
-        setIsAcid((Boolean)value);
+        setIsTransactional((Boolean)value);
       }
       break;
 
@@ -542,8 +542,8 @@ import org.slf4j.LoggerFactory;
     case OPERATION_TYPE:
       return getOperationType();
 
-    case IS_ACID:
-      return isIsAcid();
+    case IS_TRANSACTIONAL:
+      return isIsTransactional();
 
     case IS_DYNAMIC_PARTITION_WRITE:
       return isIsDynamicPartitionWrite();
@@ -571,8 +571,8 @@ import org.slf4j.LoggerFactory;
       return isSetPartitionname();
     case OPERATION_TYPE:
       return isSetOperationType();
-    case IS_ACID:
-      return isSetIsAcid();
+    case IS_TRANSACTIONAL:
+      return isSetIsTransactional();
     case IS_DYNAMIC_PARTITION_WRITE:
       return isSetIsDynamicPartitionWrite();
     }
@@ -646,12 +646,12 @@ import org.slf4j.LoggerFactory;
         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))
+    boolean this_present_isTransactional = true && this.isSetIsTransactional();
+    boolean that_present_isTransactional = true && that.isSetIsTransactional();
+    if (this_present_isTransactional || that_present_isTransactional) {
+      if (!(this_present_isTransactional && that_present_isTransactional))
         return false;
-      if (this.isAcid != that.isAcid)
+      if (this.isTransactional != that.isTransactional)
         return false;
     }
 
@@ -701,10 +701,10 @@ import org.slf4j.LoggerFactory;
     if (present_operationType)
       list.add(operationType.getValue());
 
-    boolean present_isAcid = true && (isSetIsAcid());
-    list.add(present_isAcid);
-    if (present_isAcid)
-      list.add(isAcid);
+    boolean present_isTransactional = true && (isSetIsTransactional());
+    list.add(present_isTransactional);
+    if (present_isTransactional)
+      list.add(isTransactional);
 
     boolean present_isDynamicPartitionWrite = true && (isSetIsDynamicPartitionWrite());
     list.add(present_isDynamicPartitionWrite);
@@ -782,12 +782,12 @@ import org.slf4j.LoggerFactory;
         return lastComparison;
       }
     }
-    lastComparison = Boolean.valueOf(isSetIsAcid()).compareTo(other.isSetIsAcid());
+    lastComparison = Boolean.valueOf(isSetIsTransactional()).compareTo(other.isSetIsTransactional());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    if (isSetIsAcid()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isAcid, other.isAcid);
+    if (isSetIsTransactional()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.isTransactional, other.isTransactional);
       if (lastComparison != 0) {
         return lastComparison;
       }
@@ -875,10 +875,10 @@ import org.slf4j.LoggerFactory;
       }
       first = false;
     }
-    if (isSetIsAcid()) {
+    if (isSetIsTransactional()) {
       if (!first) sb.append(", ");
-      sb.append("isAcid:");
-      sb.append(this.isAcid);
+      sb.append("isTransactional:");
+      sb.append(this.isTransactional);
       first = false;
     }
     if (isSetIsDynamicPartitionWrite()) {
@@ -992,10 +992,10 @@ import org.slf4j.LoggerFactory;
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 7: // IS_ACID
+          case 7: // IS_TRANSACTIONAL
             if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) {
-              struct.isAcid = iprot.readBool();
-              struct.setIsAcidIsSet(true);
+              struct.isTransactional = iprot.readBool();
+              struct.setIsTransactionalIsSet(true);
             } else { 
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
@@ -1057,9 +1057,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldEnd();
         }
       }
-      if (struct.isSetIsAcid()) {
-        oprot.writeFieldBegin(IS_ACID_FIELD_DESC);
-        oprot.writeBool(struct.isAcid);
+      if (struct.isSetIsTransactional()) {
+        oprot.writeFieldBegin(IS_TRANSACTIONAL_FIELD_DESC);
+        oprot.writeBool(struct.isTransactional);
         oprot.writeFieldEnd();
       }
       if (struct.isSetIsDynamicPartitionWrite()) {
@@ -1097,7 +1097,7 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetOperationType()) {
         optionals.set(2);
       }
-      if (struct.isSetIsAcid()) {
+      if (struct.isSetIsTransactional()) {
         optionals.set(3);
       }
       if (struct.isSetIsDynamicPartitionWrite()) {
@@ -1113,8 +1113,8 @@ import org.slf4j.LoggerFactory;
       if (struct.isSetOperationType()) {
         oprot.writeI32(struct.operationType.getValue());
       }
-      if (struct.isSetIsAcid()) {
-        oprot.writeBool(struct.isAcid);
+      if (struct.isSetIsTransactional()) {
+        oprot.writeBool(struct.isTransactional);
       }
       if (struct.isSetIsDynamicPartitionWrite()) {
         oprot.writeBool(struct.isDynamicPartitionWrite);
@@ -1144,8 +1144,8 @@ import org.slf4j.LoggerFactory;
         struct.setOperationTypeIsSet(true);
       }
       if (incoming.get(3)) {
-        struct.isAcid = iprot.readBool();
-        struct.setIsAcidIsSet(true);
+        struct.isTransactional = iprot.readBool();
+        struct.setIsTransactionalIsSet(true);
       }
       if (incoming.get(4)) {
         struct.isDynamicPartitionWrite = iprot.readBool();

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
index 3be16de..6e3ec62 100644
--- a/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/standalone-metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -15506,7 +15506,7 @@ class LockComponent {
   /**
    * @var bool
    */
-  public $isAcid = false;
+  public $isTransactional = false;
   /**
    * @var bool
    */
@@ -15540,7 +15540,7 @@ class LockComponent {
           'type' => TType::I32,
           ),
         7 => array(
-          'var' => 'isAcid',
+          'var' => 'isTransactional',
           'type' => TType::BOOL,
           ),
         8 => array(
@@ -15568,8 +15568,8 @@ class LockComponent {
       if (isset($vals['operationType'])) {
         $this->operationType = $vals['operationType'];
       }
-      if (isset($vals['isAcid'])) {
-        $this->isAcid = $vals['isAcid'];
+      if (isset($vals['isTransactional'])) {
+        $this->isTransactional = $vals['isTransactional'];
       }
       if (isset($vals['isDynamicPartitionWrite'])) {
         $this->isDynamicPartitionWrite = $vals['isDynamicPartitionWrite'];
@@ -15640,7 +15640,7 @@ class LockComponent {
           break;
         case 7:
           if ($ftype == TType::BOOL) {
-            $xfer += $input->readBool($this->isAcid);
+            $xfer += $input->readBool($this->isTransactional);
           } else {
             $xfer += $input->skip($ftype);
           }
@@ -15695,9 +15695,9 @@ class LockComponent {
       $xfer += $output->writeI32($this->operationType);
       $xfer += $output->writeFieldEnd();
     }
-    if ($this->isAcid !== null) {
-      $xfer += $output->writeFieldBegin('isAcid', TType::BOOL, 7);
-      $xfer += $output->writeBool($this->isAcid);
+    if ($this->isTransactional !== null) {
+      $xfer += $output->writeFieldBegin('isTransactional', TType::BOOL, 7);
+      $xfer += $output->writeBool($this->isTransactional);
       $xfer += $output->writeFieldEnd();
     }
     if ($this->isDynamicPartitionWrite !== null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 4d4429f..486f061 100644
--- a/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/standalone-metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -10881,7 +10881,7 @@ class LockComponent:
    - tablename
    - partitionname
    - operationType
-   - isAcid
+   - isTransactional
    - isDynamicPartitionWrite
   """
 
@@ -10893,18 +10893,18 @@ class LockComponent:
     (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
+    (7, TType.BOOL, 'isTransactional', None, False, ), # 7
     (8, TType.BOOL, 'isDynamicPartitionWrite', None, False, ), # 8
   )
 
-  def __init__(self, type=None, level=None, dbname=None, tablename=None, partitionname=None, operationType=thrift_spec[6][4], isAcid=thrift_spec[7][4], isDynamicPartitionWrite=thrift_spec[8][4],):
+  def __init__(self, type=None, level=None, dbname=None, tablename=None, partitionname=None, operationType=thrift_spec[6][4], isTransactional=thrift_spec[7][4], isDynamicPartitionWrite=thrift_spec[8][4],):
     self.type = type
     self.level = level
     self.dbname = dbname
     self.tablename = tablename
     self.partitionname = partitionname
     self.operationType = operationType
-    self.isAcid = isAcid
+    self.isTransactional = isTransactional
     self.isDynamicPartitionWrite = isDynamicPartitionWrite
 
   def read(self, iprot):
@@ -10948,7 +10948,7 @@ class LockComponent:
           iprot.skip(ftype)
       elif fid == 7:
         if ftype == TType.BOOL:
-          self.isAcid = iprot.readBool()
+          self.isTransactional = iprot.readBool()
         else:
           iprot.skip(ftype)
       elif fid == 8:
@@ -10990,9 +10990,9 @@ class LockComponent:
       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)
+    if self.isTransactional is not None:
+      oprot.writeFieldBegin('isTransactional', TType.BOOL, 7)
+      oprot.writeBool(self.isTransactional)
       oprot.writeFieldEnd()
     if self.isDynamicPartitionWrite is not None:
       oprot.writeFieldBegin('isDynamicPartitionWrite', TType.BOOL, 8)
@@ -11019,7 +11019,7 @@ class LockComponent:
     value = (value * 31) ^ hash(self.tablename)
     value = (value * 31) ^ hash(self.partitionname)
     value = (value * 31) ^ hash(self.operationType)
-    value = (value * 31) ^ hash(self.isAcid)
+    value = (value * 31) ^ hash(self.isTransactional)
     value = (value * 31) ^ hash(self.isDynamicPartitionWrite)
     return value
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 6b6746f..dd7467c 100644
--- a/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/standalone-metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -2457,7 +2457,7 @@ class LockComponent
   TABLENAME = 4
   PARTITIONNAME = 5
   OPERATIONTYPE = 6
-  ISACID = 7
+  ISTRANSACTIONAL = 7
   ISDYNAMICPARTITIONWRITE = 8
 
   FIELDS = {
@@ -2467,7 +2467,7 @@ class LockComponent
     TABLENAME => {:type => ::Thrift::Types::STRING, :name => 'tablename', :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},
+    ISTRANSACTIONAL => {:type => ::Thrift::Types::BOOL, :name => 'isTransactional', :default => false, :optional => true},
     ISDYNAMICPARTITIONWRITE => {:type => ::Thrift::Types::BOOL, :name => 'isDynamicPartitionWrite', :default => false, :optional => true}
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
index 6e45187..1ad0638 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/LockComponentBuilder.java
@@ -77,8 +77,8 @@ public class LockComponentBuilder {
     return this;
   }
 
-  public LockComponentBuilder setIsFullAcid(boolean t) {
-    component.setIsAcid(t);
+  public LockComponentBuilder setIsTransactional(boolean t) {
+    component.setIsTransactional(t);
     return this;
   }
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
index cb5b853..e453e5a 100644
--- a/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
+++ b/standalone-metastore/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
@@ -1332,7 +1332,7 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
           // For each component in this lock request,
           // add an entry to the txn_components table
           for (LockComponent lc : rqst.getComponent()) {
-            if(lc.isSetIsAcid() && !lc.isIsAcid()) {
+            if(lc.isSetIsTransactional() && !lc.isIsTransactional()) {
               //we don't prevent using non-acid resources in a txn but we do lock them
               continue;
             }
@@ -1418,7 +1418,8 @@ abstract class TxnHandler implements TxnStore, TxnStore.MutexAPI {
             (MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST) || MetastoreConf.getBoolVar(conf, ConfVars.HIVE_IN_TEZ_TEST))) {
             //old version of thrift client should have (lc.isSetOperationType() == false) but they do not
             //If you add a default value to a variable, isSet() for that variable is true regardless of the where the
-            //message was created (for object variables.  It works correctly for boolean vars, e.g. LockComponent.isAcid).
+            //message was created (for object variables.
+            // It works correctly for boolean vars, e.g. LockComponent.isTransactional).
             //in test mode, upgrades are not tested, so client version and server version of thrift always matches so
             //we see UNSET here it means something didn't set the appropriate value.
             throw new IllegalStateException("Bug: operationType=" + lc.getOperationType() + " for component "

http://git-wip-us.apache.org/repos/asf/hive/blob/52c6e893/standalone-metastore/src/main/thrift/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/standalone-metastore/src/main/thrift/hive_metastore.thrift b/standalone-metastore/src/main/thrift/hive_metastore.thrift
index b816eb6..ef63eab 100644
--- a/standalone-metastore/src/main/thrift/hive_metastore.thrift
+++ b/standalone-metastore/src/main/thrift/hive_metastore.thrift
@@ -824,7 +824,7 @@ struct LockComponent {
     4: optional string tablename,
     5: optional string partitionname,
     6: optional DataOperationType operationType = DataOperationType.UNSET,
-    7: optional bool isAcid = false,
+    7: optional bool isTransactional = false,
     8: optional bool isDynamicPartitionWrite = false
 }