You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/11/09 12:08:58 UTC

[GitHub] [hive] deniskuzZ opened a new pull request #2772: [WIP] HIVE-24445: Non blocking DROP table implementation

deniskuzZ opened a new pull request #2772:
URL: https://github.com/apache/hive/pull/2772


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   This changes the drop table operation to asynchronous. The data files of dropped table will not be removed right away, but they are going to be cleaned up later by the Compactor/Cleaner.
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   This along with a few other changes will enable us to implement lockless reads, which provides perf boost to the transactional tables.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   No
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Unit tests


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773112203



##########
File path: storage-api/src/java/org/apache/hadoop/hive/common/AcidConstants.java
##########
@@ -40,6 +40,10 @@
   public static final String VISIBILITY_PREFIX = "_v";
   public static final Pattern VISIBILITY_PATTERN = Pattern.compile(VISIBILITY_PREFIX + "\\d+");
 
+  public static final String V_TABLE_SUFFIX = ".v";

Review comment:
       Could we use a variable name which describes these tables better?
   Like: SOFT_DELETE_PATH_SUFFIX, SOFT_DELETE_TABLE_PATTERN, SOFT_DELETE_TABLE?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760047913



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1451,11 +1453,11 @@ public void commitTxn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbort
                 OperationType.UPDATE + "," + OperationType.DELETE + ")";
         long tempCommitId = generateTemporaryId();
 
-        if (txnType.get() == TxnType.COMPACTION) {
+        if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) {
           acquireTxnLock(stmt, false);
           commitId = getHighWaterMark(stmt);
 
-        } else if (txnType.get() != TxnType.READ_ONLY && !isReplayedReplTxn) {
+        } else if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn) {

Review comment:
       What will happen wrt replication of these soft delete-s? Do they know about the changes we propose?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760229198



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1273,14 +1276,21 @@ public void createTable(Table tbl, boolean ifNotExists,
           principalPrivs.setRolePrivileges(grants.getRoleGrants());
           tTbl.setPrivileges(principalPrivs);
         }
+        boolean createTableUseSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+            || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+          && AcidUtils.isTransactionalTable(tbl);
+
+        if (createTableUseSuffix) {
+          tTbl.setTxnId(ss.getTxnMgr().getCurrentTxnId());

Review comment:
       I am relying on txnId a flag that shows if the non-blocking drop is supported. if not set, we won't create a versioned table.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r771284378



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -172,22 +174,29 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       if (metricsEnabled) {
         perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
       }
-      Table t = resolveTable(ci);
-      if (t == null) {
-        // The table was dropped before we got around to cleaning it.
-        LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
+      String location = Optional.ofNullable(ci.properties).map(StringableMap::new)

Review comment:
       Maybe one more check - if the location is equals what we would have created -, so creating table with `show create table` would work as well.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r749481884



##########
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:
       fixed




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760170333



##########
File path: standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
##########
@@ -615,16 +615,18 @@ int _kTxnTypeValues[] = {
   TxnType::REPL_CREATED,
   TxnType::READ_ONLY,
   TxnType::COMPACTION,
-  TxnType::MATER_VIEW_REBUILD
+  TxnType::MATER_VIEW_REBUILD,
+  TxnType::SOFT_DELETE

Review comment:
       however, old HMS won't have support for lockless read, so we should be good




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760218118



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/table/drop/DropTableAnalyzer.java
##########
@@ -51,7 +55,14 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
     Table table = getTable(tableName, throwException);
     if (table != null) {
       inputs.add(new ReadEntity(table));
-      outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_EXCLUSIVE));
+
+      boolean tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+          || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+        && AcidUtils.isTransactionalTable(table)
+        && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID);
+
+      outputs.add(new WriteEntity(table, 

Review comment:
       it's already logged by DbTxnManager when calling AcidUtils.makeLockComponents




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760033191



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -279,10 +292,14 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       }
     }
   }
+  
+  private interface CleanerTask<T> {

Review comment:
       Why do we need this specific interface instead of `Callable`?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r761090920



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -1772,6 +1773,22 @@ public void dropTable(String dbname, String name, boolean deleteData,
     dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, ifPurge);
   }
 
+  @Override
+  public void dropTable(Table tbl, boolean deleteData, boolean ignoreUnknownTbl, boolean ifPurge) throws TException {
+    EnvironmentContext context = null;
+    if (ifPurge) {

Review comment:
       fixed, see Cleaner




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r748460586



##########
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:
       if table location doesn't contain the versioning - we won't support lockless reads




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773273572



##########
File path: storage-api/src/java/org/apache/hadoop/hive/common/AcidConstants.java
##########
@@ -40,6 +40,10 @@
   public static final String VISIBILITY_PREFIX = "_v";
   public static final Pattern VISIBILITY_PATTERN = Pattern.compile(VISIBILITY_PREFIX + "\\d+");
 
+  public static final String V_TABLE_SUFFIX = ".v";

Review comment:
       @pvary, should we keep`versioned=true` as tableProperty or you prefer `soft_delete=true` ?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773110665



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -4238,22 +4307,23 @@ public void cleanupRecords(HiveObjectType type, Database db, Table table,
             buff.append("'");
             queries.add(buff.toString());
 
-            buff.setLength(0);
-            buff.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\"='");
-            buff.append(dbName.toLowerCase());
-            buff.append("' AND \"T2W_TABLE\"='");
-            buff.append(tblName.toLowerCase());
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\"='");
-            buff.append(dbName.toLowerCase());
-            buff.append("' AND \"NWI_TABLE\"='");
-            buff.append(tblName.toLowerCase());
-            buff.append("'");
-            queries.add(buff.toString());
+            if (!keepTxnToWriteIdMetaData) {

Review comment:
       When will we remove this data for tables which are soft deleted?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760027429



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1334,6 +1344,20 @@ 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 tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+        || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+      && AcidUtils.isTransactionalTable(table)
+      && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID);
+    
+    if (tableWithSuffix) {
+      long txnId = Optional.ofNullable(SessionState.get())
+        .map(ss -> ss.getTxnMgr().getCurrentTxnId()).orElse(0L);
+      table.getTTable().setTxnId(txnId);
+    }

Review comment:
       if the location is not PATH_WITH_VERSION_ID, we might have a situation were we are trying to do a softdelete, but could not do it. Do we need to handle this?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760026327



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1273,14 +1276,21 @@ public void createTable(Table tbl, boolean ifNotExists,
           principalPrivs.setRolePrivileges(grants.getRoleGrants());
           tTbl.setPrivileges(principalPrivs);
         }
+        boolean createTableUseSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+            || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+          && AcidUtils.isTransactionalTable(tbl);
+
+        if (createTableUseSuffix) {
+          tTbl.setTxnId(ss.getTxnMgr().getCurrentTxnId());

Review comment:
       Shall we just push this transactionId for every transactional table?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760246130



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -279,10 +292,14 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       }
     }
   }
+  
+  private interface CleanerTask<T> {
+    T execute() throws Exception;
+  }
 
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, Table t, ValidTxnList validTxnList)
+  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)

Review comment:
       it's needed for `safe_delete` use-case as we won't be able to resolve the table object - metadata would be already removed, however, we could get full tableName from the ci object.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760214887



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
##########
@@ -3109,10 +3109,16 @@ public static TxnType getTxnType(Configuration conf, ASTNode tree) {
     if (tree.getFirstChildWithType(HiveParser.TOK_ALTERTABLE_COMPACT) != null){
       return TxnType.COMPACTION;
     }
+    // check if soft delete
+    if (tree.getToken().getType() == HiveParser.TOK_DROPTABLE 
+      && (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+        || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))){
+      return TxnType.SOFT_DELETE;

Review comment:
       I don't think so as we still should be able to drop tables created without that option (not versioned)




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760253006



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -1772,6 +1773,22 @@ public void dropTable(String dbname, String name, boolean deleteData,
     dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, ifPurge);
   }
 
+  @Override
+  public void dropTable(Table tbl, boolean deleteData, boolean ignoreUnknownTbl, boolean ifPurge) throws TException {
+    EnvironmentContext context = null;
+    if (ifPurge) {

Review comment:
       good catch, it's not handled yet, as we don't support purge in Cleaner, however, we could pass `purge` attribute via the queue tblproperties.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760043808



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
##########
@@ -1772,6 +1773,22 @@ public void dropTable(String dbname, String name, boolean deleteData,
     dropTable(getDefaultCatalog(conf), dbname, name, deleteData, ignoreUnknownTab, ifPurge);
   }
 
+  @Override
+  public void dropTable(Table tbl, boolean deleteData, boolean ignoreUnknownTbl, boolean ifPurge) throws TException {
+    EnvironmentContext context = null;
+    if (ifPurge) {

Review comment:
       How will we handle purge with soft delete?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r772455283



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/table/drop/DropTableAnalyzer.java
##########
@@ -51,7 +55,14 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
     Table table = getTable(tableName, throwException);
     if (table != null) {
       inputs.add(new ReadEntity(table));
-      outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_EXCLUSIVE));
+
+      boolean tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+          || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+        && AcidUtils.isTransactionalTable(table)
+        && table.getSd().getLocation().matches(V_TABLE_PATTERN);

Review comment:
       - suffix starts with `dot` that is not allowed for table names in create statement;
   - changed to table property `versioned` (added on create table)




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773107563



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java
##########
@@ -363,8 +366,14 @@ public Path getDefaultTablePath(Database db, String tableName, boolean isExterna
     } else {
       dbPath = getDatabaseManagedPath(db);
     }
-    return getDnsPath(
-        new Path(dbPath, MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
+    if (!isExternal && tableName.matches("(.*)" + V_TABLE_PATTERN)) {

Review comment:
       nit: Maybe a comment, or a different String value, since this is not a `tableName`. This is `tablePath`? Strings are always recreated anyway, so there is not performance loss, but I think it would be easier to understand the code




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] klcopp commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773103500



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1273,14 +1276,21 @@ public void createTable(Table tbl, boolean ifNotExists,
           principalPrivs.setRolePrivileges(grants.getRoleGrants());
           tTbl.setPrivileges(principalPrivs);
         }
+        boolean createTableUseSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+            || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+          && AcidUtils.isTransactionalTable(tbl);
+
+        if (createTableUseSuffix) {
+          tTbl.setTxnId(ss.getTxnMgr().getCurrentTxnId());

Review comment:
       Could it be that we always set the `txnId`, and only set the table property if needed?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r771280308



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/table/drop/DropTableAnalyzer.java
##########
@@ -51,7 +55,14 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
     Table table = getTable(tableName, throwException);
     if (table != null) {
       inputs.add(new ReadEntity(table));
-      outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_EXCLUSIVE));
+
+      boolean tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+          || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+        && AcidUtils.isTransactionalTable(table)
+        && table.getSd().getLocation().matches(V_TABLE_PATTERN);

Review comment:
       Might have issues with legacy tables with where the name (and location) ends with `_v1`.
   Would it cause problems?
   - Say I have an old table users_v2
   - Switch on the `HIVE_ACID_CREATE_TABLE_USE_SUFFIX`
   - Drop the table?
   




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760238917



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -172,22 +174,29 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       if (metricsEnabled) {
         perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
       }
-      Table t = resolveTable(ci);
-      if (t == null) {
-        // The table was dropped before we got around to cleaning it.
-        LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
+      String location = Optional.ofNullable(ci.properties).map(StringableMap::new)

Review comment:
       we only support this feature for acid tables with a non-customized location:
   ````
   if (!TableType.VIRTUAL_VIEW.toString().equals(tbl.getTableType())) {
       if (tbl.getSd().getLocation() == null || tbl.getSd().getLocation().isEmpty()) {
           tblPath = wh.getDefaultTablePath(db, getPathWithSuffix(tbl.getTableName(), tbl), isExternal(tbl));
   }
   
   String getPathWithSuffix(String path, Table tbl) {
       return path + (tbl.isSetTxnId() ?
           VISIBILITY_PREFIX + String.format(DELTA_DIGITS, tbl.getTxnId()) : "");
   }
   ````




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760156827



##########
File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
##########
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql;
 
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.*;

Review comment:
       import is required after the tests refactor, previous visibility was package-private.
   ps: moved tests refactor under https://github.com/apache/hive/pull/2831




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r749481709



##########
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:
       fixed




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773108622



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1053,8 +1055,8 @@ public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaExcept
           txnid = targetTxnIds.get(0);
         }
 
-        Optional<TxnType> txnType = getOpenTxnTypeAndLock(stmt, txnid);
-        if (!txnType.isPresent()) {
+        TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid);

Review comment:
       Why remove `Optional`?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760050115



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
##########
@@ -336,9 +336,9 @@ public void markCompacted(CompactionInfo info) throws MetaException {
          * By filtering on minOpenTxnWaterMark, we will only cleanup after every transaction is committed, that could see
          * the uncompacted deltas. This way the cleaner can clean up everything that was made obsolete by this compaction.
          */
-        String s = "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", "
-                + "\"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = '"
-                + READY_FOR_CLEANING + "'";
+        String s = "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", \"CQ_TYPE\", " 
+            + "\"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TBLPROPERTIES\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = '"

Review comment:
       Is this an already existing field?
   Do we show this in the sysdb tables?
   What happens with this compaction if an old VW picks this up?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760168937



##########
File path: standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
##########
@@ -615,16 +615,18 @@ int _kTxnTypeValues[] = {
   TxnType::REPL_CREATED,
   TxnType::READ_ONLY,
   TxnType::COMPACTION,
-  TxnType::MATER_VIEW_REBUILD
+  TxnType::MATER_VIEW_REBUILD,
+  TxnType::SOFT_DELETE

Review comment:
       Old versions of HMS would treat new values as null (default - null), in this case, we won't ever enter the if branch for SOFT_DELETE, meaning, we won't populate the highest write id required by cleaner and might clean up some deltas prematurely, see HIVE-24291. 




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760251029



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
##########
@@ -249,7 +249,7 @@ protected String resolveUserToRunAs(Map<String, String> cache, Table t, Partitio
 
     String user = cache.get(fullTableName);
     if (user == null) {
-      user = findUserToRunAs(sd.getLocation(), t);
+      user = TxnUtils.findUserToRunAs(sd.getLocation(), t, conf);

Review comment:
       nope, refactor needed to set runAs user for `safe-delete` in AcidEventListener.dropTable()




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760025563



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
##########
@@ -3109,10 +3109,16 @@ public static TxnType getTxnType(Configuration conf, ASTNode tree) {
     if (tree.getFirstChildWithType(HiveParser.TOK_ALTERTABLE_COMPACT) != null){
       return TxnType.COMPACTION;
     }
+    // check if soft delete
+    if (tree.getToken().getType() == HiveParser.TOK_DROPTABLE 
+      && (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+        || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))){
+      return TxnType.SOFT_DELETE;

Review comment:
       Shall we throw an exception if we try to drop a table but the location is not PATH_WITH_VERSION_ID?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760157455



##########
File path: ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
##########
@@ -202,8 +208,29 @@ public int compare(int[] row1, int[] row2) {
       return 0;
     }
   }
-  protected String makeValuesClause(int[][] rows) {
-    return TestTxnCommands2.makeValuesClause(rows);
+  public static String makeValuesClause(int[][] rows) {

Review comment:
       moved test refactor under: https://github.com/apache/hive/pull/2831




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760255174



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
##########
@@ -60,9 +65,30 @@ public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
 
   @Override
   public void onDropTable(DropTableEvent tableEvent)  throws MetaException {
-    if (TxnUtils.isTransactionalTable(tableEvent.getTable())) {
+    Table table = tableEvent.getTable();
+    
+    if (TxnUtils.isTransactionalTable(table)) {
       txnHandler = getTxnHandler();
-      txnHandler.cleanupRecords(HiveObjectType.TABLE, null, tableEvent.getTable(), null);
+      txnHandler.cleanupRecords(HiveObjectType.TABLE, null, table, null, !tableEvent.getDeleteData());
+      
+      if (!tableEvent.getDeleteData()) {
+        long currentTxn = Optional.ofNullable(tableEvent.getEnvironmentContext())
+          .map(EnvironmentContext::getProperties)
+          .map(prop -> prop.get("txnId"))
+          .map(Long::parseLong)
+          .orElse(0L);
+
+        if (currentTxn > 0) {
+          CompactionRequest rqst = new CompactionRequest(table.getDbName(), table.getTableName(), CompactionType.MAJOR);
+          rqst.putToProperties("location", table.getSd().getLocation());

Review comment:
       I didn't want to change the compaction queue schema + it kinda fits the definition of CQ_TBLPROPERTIES column




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r748461364



##########
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:
       we are not dropping, we are delegating cleanup to the Cleaner do remove when possible




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r748461539



##########
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:
       added default method
   




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760023405



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/table/drop/DropTableAnalyzer.java
##########
@@ -51,7 +55,14 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
     Table table = getTable(tableName, throwException);
     if (table != null) {
       inputs.add(new ReadEntity(table));
-      outputs.add(new WriteEntity(table, WriteEntity.WriteType.DDL_EXCLUSIVE));
+
+      boolean tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+          || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+        && AcidUtils.isTransactionalTable(table)
+        && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID);
+
+      outputs.add(new WriteEntity(table, 

Review comment:
       It would be good to see this in the log. Minimally with debug level




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760045866



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/AcidEventListener.java
##########
@@ -60,9 +65,30 @@ public void onDropDatabase (DropDatabaseEvent dbEvent) throws MetaException {
 
   @Override
   public void onDropTable(DropTableEvent tableEvent)  throws MetaException {
-    if (TxnUtils.isTransactionalTable(tableEvent.getTable())) {
+    Table table = tableEvent.getTable();
+    
+    if (TxnUtils.isTransactionalTable(table)) {
       txnHandler = getTxnHandler();
-      txnHandler.cleanupRecords(HiveObjectType.TABLE, null, tableEvent.getTable(), null);
+      txnHandler.cleanupRecords(HiveObjectType.TABLE, null, table, null, !tableEvent.getDeleteData());
+      
+      if (!tableEvent.getDeleteData()) {
+        long currentTxn = Optional.ofNullable(tableEvent.getEnvironmentContext())
+          .map(EnvironmentContext::getProperties)
+          .map(prop -> prop.get("txnId"))
+          .map(Long::parseLong)
+          .orElse(0L);
+
+        if (currentTxn > 0) {
+          CompactionRequest rqst = new CompactionRequest(table.getDbName(), table.getTableName(), CompactionType.MAJOR);
+          rqst.putToProperties("location", table.getSd().getLocation());

Review comment:
       Why properties, and not optional field?
   




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760170333



##########
File path: standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
##########
@@ -615,16 +615,18 @@ int _kTxnTypeValues[] = {
   TxnType::REPL_CREATED,
   TxnType::READ_ONLY,
   TxnType::COMPACTION,
-  TxnType::MATER_VIEW_REBUILD
+  TxnType::MATER_VIEW_REBUILD,
+  TxnType::SOFT_DELETE

Review comment:
       however, old HMS won't have support for lockless drop as well, so we should be good




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760233519



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1334,6 +1344,20 @@ 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 tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+        || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+      && AcidUtils.isTransactionalTable(table)
+      && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID);
+    
+    if (tableWithSuffix) {
+      long txnId = Optional.ofNullable(SessionState.get())
+        .map(ss -> ss.getTxnMgr().getCurrentTxnId()).orElse(0L);
+      table.getTTable().setTxnId(txnId);
+    }

Review comment:
       we could have old tables that were created without this feature being enabled, and we still should support regular drops for them.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760241861



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -279,10 +292,14 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       }
     }
   }
+  
+  private interface CleanerTask<T> {

Review comment:
       not really, good catch!




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773273572



##########
File path: storage-api/src/java/org/apache/hadoop/hive/common/AcidConstants.java
##########
@@ -40,6 +40,10 @@
   public static final String VISIBILITY_PREFIX = "_v";
   public static final Pattern VISIBILITY_PATTERN = Pattern.compile(VISIBILITY_PREFIX + "\\d+");
 
+  public static final String V_TABLE_SUFFIX = ".v";

Review comment:
       @pvary, should we keep`versioned=true` as tableProperty or you prefer `soft_delete=true` ?
   V_TABLE_ ... stands for a versioned table, I didn't want to limit this just to soft delete functionality. 




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r771281801



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1273,14 +1276,21 @@ public void createTable(Table tbl, boolean ifNotExists,
           principalPrivs.setRolePrivileges(grants.getRoleGrants());
           tTbl.setPrivileges(principalPrivs);
         }
+        boolean createTableUseSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+            || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+          && AcidUtils.isTransactionalTable(tbl);
+
+        if (createTableUseSuffix) {
+          tTbl.setTxnId(ss.getTxnMgr().getCurrentTxnId());

Review comment:
       Do we have a context object where we can push this info instead of a `magical` flag?
   It would be better if we explicitly say "non-blocking" create, instead of depending on the txnId




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773102565



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1403,8 +1426,14 @@ public void dropTable(String dbName, String tableName, boolean deleteData,
     }
   }
 
-
-
+  public void dropTable(org.apache.hadoop.hive.metastore.api.Table table, 
+      boolean deleteData, boolean ignoreUnknownTab, boolean ifPurge) throws HiveException {
+    try {
+      getMSC().dropTable(table, deleteData, ignoreUnknownTab, ifPurge);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }  

Review comment:
       nit: newline




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773270399



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -4238,22 +4307,23 @@ public void cleanupRecords(HiveObjectType type, Database db, Table table,
             buff.append("'");
             queries.add(buff.toString());
 
-            buff.setLength(0);
-            buff.append("DELETE FROM \"TXN_TO_WRITE_ID\" WHERE \"T2W_DATABASE\"='");
-            buff.append(dbName.toLowerCase());
-            buff.append("' AND \"T2W_TABLE\"='");
-            buff.append(tblName.toLowerCase());
-            buff.append("'");
-            queries.add(buff.toString());
-
-            buff.setLength(0);
-            buff.append("DELETE FROM \"NEXT_WRITE_ID\" WHERE \"NWI_DATABASE\"='");
-            buff.append(dbName.toLowerCase());
-            buff.append("' AND \"NWI_TABLE\"='");
-            buff.append(tblName.toLowerCase());
-            buff.append("'");
-            queries.add(buff.toString());
+            if (!keepTxnToWriteIdMetaData) {

Review comment:
       AcidHouseKeeperThread would do it in cleanTheHouse() once min_uncommitted_txnid which is currently seen by any open transactions is advanced.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760213666



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/table/drop/DropTableOperation.java
##########
@@ -109,7 +109,7 @@ public int execute() throws HiveException {
     }
 
     // TODO: API w/catalog name
-    context.getDb().dropTable(desc.getTableName(), desc.isPurge());
+    context.getDb().dropTable(table, desc.isPurge());

Review comment:
       I need to pass extra info like txnId through the tbl object




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760257691



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1451,11 +1453,11 @@ public void commitTxn(CommitTxnRequest rqst) throws NoSuchTxnException, TxnAbort
                 OperationType.UPDATE + "," + OperationType.DELETE + ")";
         long tempCommitId = generateTemporaryId();
 
-        if (txnType.get() == TxnType.COMPACTION) {
+        if (txnType == TxnType.SOFT_DELETE || txnType == TxnType.COMPACTION) {
           acquireTxnLock(stmt, false);
           commitId = getHighWaterMark(stmt);
 
-        } else if (txnType.get() != TxnType.READ_ONLY && !isReplayedReplTxn) {
+        } else if (txnType != TxnType.READ_ONLY && !isReplayedReplTxn) {

Review comment:
       I need to check if we send the metadata delete event for repl, however, for the data removal, we are doing this in Cleaner. 




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760030923



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -172,22 +174,29 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       if (metricsEnabled) {
         perfLogger.perfLogBegin(CLASS_NAME, cleanerMetric);
       }
-      Table t = resolveTable(ci);
-      if (t == null) {
-        // The table was dropped before we got around to cleaning it.
-        LOG.info("Unable to find table " + ci.getFullTableName() + ", assuming it was dropped." +
+      String location = Optional.ofNullable(ci.properties).map(StringableMap::new)

Review comment:
       How this behaves with non-standard locations, and transformer generated locations.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760021737



##########
File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
##########
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql;
 
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.*;

Review comment:
       Is there any other change in this file than the import change? If not I would revert this file




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760042920



##########
File path: standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
##########
@@ -615,16 +615,18 @@ int _kTxnTypeValues[] = {
   TxnType::REPL_CREATED,
   TxnType::READ_ONLY,
   TxnType::COMPACTION,
-  TxnType::MATER_VIEW_REBUILD
+  TxnType::MATER_VIEW_REBUILD,
+  TxnType::SOFT_DELETE

Review comment:
       Will this cause any compatibility issues?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773109849



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3814,6 +3815,74 @@ public CompactionResponse compact(CompactionRequest rqst) throws MetaException {
     }
   }
 
+  @Override
+  @RetrySemantics.Idempotent

Review comment:
       This means, we can resubmit without issues in case of a failure.
   Would it cause issues if we have multiple compactions scheduled for the same table?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773245672



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/Warehouse.java
##########
@@ -363,8 +366,14 @@ public Path getDefaultTablePath(Database db, String tableName, boolean isExterna
     } else {
       dbPath = getDatabaseManagedPath(db);
     }
-    return getDnsPath(
-        new Path(dbPath, MetaStoreUtils.encodeTableName(tableName.toLowerCase())));
+    if (!isExternal && tableName.matches("(.*)" + V_TABLE_PATTERN)) {

Review comment:
       sorry, don't follow,  tableName is actually a table name, like `acid_tbl.00005`. dbPath contains an absolute path.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773281383



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3814,6 +3815,74 @@ public CompactionResponse compact(CompactionRequest rqst) throws MetaException {
     }
   }
 
+  @Override
+  @RetrySemantics.Idempotent

Review comment:
       changed to safeToRetry, checked - no issues
   




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760037581



##########
File path: ql/src/test/org/apache/hadoop/hive/ql/TxnCommandsBaseForTests.java
##########
@@ -202,8 +208,29 @@ public int compare(int[] row1, int[] row2) {
       return 0;
     }
   }
-  protected String makeValuesClause(int[][] rows) {
-    return TestTxnCommands2.makeValuesClause(rows);
+  public static String makeValuesClause(int[][] rows) {

Review comment:
       nit: empty line




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760024012



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/ddl/table/drop/DropTableOperation.java
##########
@@ -109,7 +109,7 @@ public int execute() throws HiveException {
     }
 
     // TODO: API w/catalog name
-    context.getDb().dropTable(desc.getTableName(), desc.isPurge());
+    context.getDb().dropTable(table, desc.isPurge());

Review comment:
       nit: Why is this change needed?




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760021077



##########
File path: itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestAcidOnTez.java
##########
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql;
 
+import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.*;

Review comment:
       nit: do not use wildcard imports




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r761090236



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -279,10 +292,14 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       }
     }
   }
+  
+  private interface CleanerTask<T> {

Review comment:
       fixed




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773104253



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
##########
@@ -1334,6 +1344,20 @@ 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 tableWithSuffix = (HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_CREATE_TABLE_USE_SUFFIX)
+        || HiveConf.getBoolVar(conf, ConfVars.HIVE_ACID_LOCKLESS_READS_ENABLED))
+      && AcidUtils.isTransactionalTable(table)
+      && table.getSd().getLocation().matches(PATH_WITH_VERSION_ID);
+    
+    if (tableWithSuffix) {
+      long txnId = Optional.ofNullable(SessionState.get())
+        .map(ss -> ss.getTxnMgr().getCurrentTxnId()).orElse(0L);
+      table.getTTable().setTxnId(txnId);
+    }

Review comment:
       Like above - we can always set the txnId, and use the flag to decide how to drop (soft/hard)




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r773250362



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1053,8 +1055,8 @@ public void abortTxn(AbortTxnRequest rqst) throws NoSuchTxnException, MetaExcept
           txnid = targetTxnIds.get(0);
         }
 
-        Optional<TxnType> txnType = getOpenTxnTypeAndLock(stmt, txnid);
-        if (!txnType.isPresent()) {
+        TxnType txnType = getOpenTxnTypeAndLock(stmt, txnid);

Review comment:
       I think, with the current code design in TxnHandler, like throwing an Exception within a separate method (inspection won't work) this is a bit easier to read and interpret.   




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760249779



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -318,8 +335,8 @@ private static String idWatermark(CompactionInfo ci) {
   private boolean removeFiles(String location, ValidWriteIdList writeIdList, CompactionInfo ci)
       throws IOException, NoSuchObjectException, MetaException {
     Path locPath = new Path(location);
-    AcidDirectory dir = AcidUtils.getAcidState(locPath.getFileSystem(conf), locPath, conf, writeIdList, Ref.from(
-        false), false);
+    FileSystem fs = locPath.getFileSystem(conf);

Review comment:
       no, created a separate cleanup method to handle `safe-delete` + extracted the common part 




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760246130



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -279,10 +292,14 @@ private void clean(CompactionInfo ci, long minOpenTxnGLB, boolean metricsEnabled
       }
     }
   }
+  
+  private interface CleanerTask<T> {
+    T execute() throws Exception;
+  }
 
-  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, Table t, ValidTxnList validTxnList)
+  private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, ValidTxnList validTxnList)

Review comment:
       it's needed for `safe_delete` use case as we won't be able to resolve table object - metadata would be already removed, we should get full tableName from the ci object.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760247777



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
##########
@@ -304,7 +321,7 @@ private ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, Tab
   }
 
   private static boolean isDynPartAbort(Table t, CompactionInfo ci) {
-    return t.getPartitionKeys() != null && t.getPartitionKeys().size() > 0
+    return Optional.ofNullable(t).map(Table::getPartitionKeys).filter(pk -> pk.size() > 0).isPresent()

Review comment:
       `safe-delete` use case - table object would be null here 




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ commented on a change in pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on a change in pull request #2772:
URL: https://github.com/apache/hive/pull/2772#discussion_r760259374



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
##########
@@ -336,9 +336,9 @@ public void markCompacted(CompactionInfo info) throws MetaException {
          * By filtering on minOpenTxnWaterMark, we will only cleanup after every transaction is committed, that could see
          * the uncompacted deltas. This way the cleaner can clean up everything that was made obsolete by this compaction.
          */
-        String s = "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", "
-                + "\"CQ_TYPE\", \"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = '"
-                + READY_FOR_CLEANING + "'";
+        String s = "SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", \"CQ_TYPE\", " 
+            + "\"CQ_RUN_AS\", \"CQ_HIGHEST_WRITE_ID\", \"CQ_TBLPROPERTIES\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = '"

Review comment:
       it's an existing field, so should be in sysdb tables. This part only affects the Cleaner.




-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] deniskuzZ merged pull request #2772: HIVE-24445: Non blocking DROP TABLE implementation

Posted by GitBox <gi...@apache.org>.
deniskuzZ merged pull request #2772:
URL: https://github.com/apache/hive/pull/2772


   


-- 
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



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org