You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/11/11 16:34:00 UTC

[jira] [Work logged] (HIVE-24445) Non blocking DROP TABLE implementation

     [ https://issues.apache.org/jira/browse/HIVE-24445?focusedWorklogId=680340&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-680340 ]

ASF GitHub Bot logged work on HIVE-24445:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 11/Nov/21 16:33
            Start Date: 11/Nov/21 16:33
    Worklog Time Spent: 10m 
      Work Description: klcopp commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r747583654



##########
File path: ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
##########
@@ -3359,4 +3366,59 @@ public void testUpdateSnapshotIsolation() throws Exception {
     Assert.assertEquals("1\t2", res.get(0));
   }
 
-}
+  @Test
+  public void testDropTable() throws Exception {
+    dropTable(new String[] {"tab_acid"});
+
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_TXN_LOCKLESS_READS_ENABLED, true);
+    driver2.getConf().setBoolVar( HiveConf.ConfVars.HIVE_TXN_NON_BLOCKING_DROP_TABLE, true);
+
+    driver.run("create table if not exists tab_acid (a int, b int) partitioned by (p string) " +
+      "stored as orc TBLPROPERTIES ('transactional'='true')");
+    driver.run("insert into tab_acid partition(p) (a,b,p) values(1,2,'foo'),(3,4,'bar')");
+
+    driver.compileAndRespond("select * from tab_acid");
+
+    DbTxnManager txnMgr2 = (DbTxnManager) TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    swapTxnManager(txnMgr2);
+    driver2.run("drop table if exists tab_acid");
+
+    swapTxnManager(txnMgr);
+    driver.run();
+
+    FileSystem fs = FileSystem.get(conf);
+    FileStatus[] stat = fs.listStatus(new Path(Paths.get("target/warehouse").toUri()),
+      p -> p.getName().matches("tab_acid" + VISIBILITY_PATTERN));
+    if (1 != stat.length) {
+      Assert.fail("Table data was removed from FS");
+    }
+
+    List<String> res = new ArrayList<>();
+    driver.getFetchTask().fetch(res);
+    Assert.assertEquals("No records found", 2, res.size());

Review comment:
       Nit: Well, "No records found" isn't necessarily true :) 

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3991,7 +3991,7 @@ public void addDynamicPartitions(AddDynamicPartitions rqst)
   @Override
   @RetrySemantics.Idempotent
   public void cleanupRecords(HiveObjectType type, Database db, Table table,
-                             Iterator<Partition> partitionIterator) throws MetaException {
+      Iterator<Partition> partitionIterator, boolean keepTxnToWriteIdMetaData) throws MetaException {

Review comment:
       keepTxnToWriteIdMetaData: this would only get cleaned for tables – so maybe it would be best to reflect that in the variable name? maybe something like: keepWriteIdMetaDataForTable?

##########
File path: ql/src/test/org/apache/hadoop/hive/ql/lockmgr/TestDbTxnManager2.java
##########
@@ -3359,4 +3366,59 @@ public void testUpdateSnapshotIsolation() throws Exception {
     Assert.assertEquals("1\t2", res.get(0));
   }
 
-}
+  @Test
+  public void testDropTable() throws Exception {
+    dropTable(new String[] {"tab_acid"});
+
+    conf.setBoolVar(HiveConf.ConfVars.HIVE_TXN_LOCKLESS_READS_ENABLED, true);
+    driver2.getConf().setBoolVar( HiveConf.ConfVars.HIVE_TXN_NON_BLOCKING_DROP_TABLE, true);
+
+    driver.run("create table if not exists tab_acid (a int, b int) partitioned by (p string) " +
+      "stored as orc TBLPROPERTIES ('transactional'='true')");
+    driver.run("insert into tab_acid partition(p) (a,b,p) values(1,2,'foo'),(3,4,'bar')");
+
+    driver.compileAndRespond("select * from tab_acid");
+
+    DbTxnManager txnMgr2 = (DbTxnManager) TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    swapTxnManager(txnMgr2);
+    driver2.run("drop table if exists tab_acid");
+
+    swapTxnManager(txnMgr);
+    driver.run();
+
+    FileSystem fs = FileSystem.get(conf);
+    FileStatus[] stat = fs.listStatus(new Path(Paths.get("target/warehouse").toUri()),
+      p -> p.getName().matches("tab_acid" + VISIBILITY_PATTERN));
+    if (1 != stat.length) {
+      Assert.fail("Table data was removed from FS");
+    }
+
+    List<String> res = new ArrayList<>();
+    driver.getFetchTask().fetch(res);
+    Assert.assertEquals("No records found", 2, res.size());
+
+    try {
+      driver.run("select * from tab_acid");
+    } catch (CommandProcessorException ex) {
+      Assert.assertEquals(ErrorMsg.INVALID_TABLE.getErrorCode(), ex.getResponseCode());
+      Assert.assertTrue(ex.getMessage().contains(ErrorMsg.INVALID_TABLE.getMsg("'tab_acid'")));
+    }
+
+    //re-create table with the same name
+    driver.run("create table if not exists tab_acid (a int, b int) partitioned by (p string) " +
+      "stored as orc TBLPROPERTIES ('transactional'='true')");
+    driver.run("insert into tab_acid partition(p) (a,b,p) values(1,2,'foo'),(3,4,'bar')");
+
+    driver.run("select * from tab_acid ");
+    res = new ArrayList<>();
+    driver.getFetchTask().fetch(res);
+    Assert.assertEquals("No records found", 2, res.size());
+
+    stat = fs.listStatus(new Path(Paths.get("target/warehouse").toUri()),

Review comment:
       Is this bit of cleanup necessary? Just wondering

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1331,6 +1340,15 @@ public void dropTable(String tableName, boolean ifPurge) throws HiveException {
     dropTable(names[0], names[1], true, true, ifPurge);
   }
 
+  public void dropTable(Table table, boolean ifPurge) throws HiveException {
+    boolean deleteData = !((HiveConf.getBoolVar(conf, ConfVars.HIVE_TXN_NON_BLOCKING_DROP_TABLE)
+        || HiveConf.getBoolVar(conf, ConfVars.HIVE_TXN_LOCKLESS_READS_ENABLED))
+      && AcidUtils.isTransactionalTable(table)
+      && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID));

Review comment:
       Idk if there's any getting around this, but what if the actual table name matches ".*_v\d" ?
   
   Also if a table was created with HIVE_TXN_NON_BLOCKING_DROP_TABLE=false, we can still keep the data right? so we can get rid of : && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID)

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java
##########
@@ -3034,6 +3040,10 @@ private boolean drop_table_core(final RawStore ms, final String catName, final S
         deletePartitionData(partPaths, ifPurge, ReplChangeManager.shouldEnableCm(db, tbl));
         // Delete the data in the table
         deleteTableData(tblPath, ifPurge, ReplChangeManager.shouldEnableCm(db, tbl));
+      } else if (TxnUtils.isTransactionalTable(tbl)) {

Review comment:
       Why compact when dropping?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 680340)
    Time Spent: 4h 40m  (was: 4.5h)

> Non blocking DROP TABLE implementation
> --------------------------------------
>
>                 Key: HIVE-24445
>                 URL: https://issues.apache.org/jira/browse/HIVE-24445
>             Project: Hive
>          Issue Type: New Feature
>          Components: Hive
>            Reporter: Zoltan Chovan
>            Assignee: Denys Kuzmenko
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 4h 40m
>  Remaining Estimate: 0h
>
> Implement a way to execute drop table operations in a way that doesn't have to wait for currently running read operations to be finished.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)