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 2020/09/28 16:58:07 UTC

[GitHub] [hive] deniskuzZ opened a new pull request #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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


   …tection
   
   <!--
   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.
   -->
   
   
   ### 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.
   -->
   
   
   ### 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'.
   -->
   
   
   ### 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.
   -->
   


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

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] pvargacl commented on a change in pull request #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/Driver.java
##########
@@ -497,38 +497,41 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command
         HiveConf.ConfVars.HIVE_TXN_MAX_RETRYSNAPSHOT_COUNT);
 
       try {
-        while (!driverTxnHandler.isValidTxnListState() && ++retryShapshotCnt <= maxRetrySnapshotCnt) {
-          LOG.info("Re-compiling after acquiring locks, attempt #" + retryShapshotCnt);
-          // Snapshot was outdated when locks were acquired, hence regenerate context, txn list and retry.
-          // TODO: Lock acquisition should be moved before analyze, this is a bit hackish.
-          // Currently, we acquire a snapshot, compile the query with that snapshot, and then - acquire locks.
-          // If snapshot is still valid, we continue as usual.
-          // But if snapshot is not valid, we recompile the query.
-          if (driverContext.isOutdatedTxn()) {
-            LOG.info("Snapshot is outdated, re-initiating transaction ...");
-            driverContext.getTxnManager().rollbackTxn();
-
-            String userFromUGI = DriverUtils.getUserFromUGI(driverContext);
-            driverContext.getTxnManager().openTxn(context, userFromUGI, driverContext.getTxnType());
-            lockAndRespond();
+        do {
+          driverContext.setOutdatedTxn(false);
+
+          if (!driverTxnHandler.isValidTxnListState()) {
+            LOG.info("Re-compiling after acquiring locks, attempt #" + retryShapshotCnt);
+            // Snapshot was outdated when locks were acquired, hence regenerate context, txn list and retry.
+            // TODO: Lock acquisition should be moved before analyze, this is a bit hackish.
+            // Currently, we acquire a snapshot, compile the query with that snapshot, and then - acquire locks.
+            // If snapshot is still valid, we continue as usual.
+            // But if snapshot is not valid, we recompile the query.
+            if (driverContext.isOutdatedTxn()) {
+              LOG.info("Snapshot is outdated, re-initiating transaction ...");
+              driverContext.getTxnManager().rollbackTxn();
+
+              String userFromUGI = DriverUtils.getUserFromUGI(driverContext);
+              driverContext.getTxnManager().openTxn(context, userFromUGI, driverContext.getTxnType());
+              lockAndRespond();
+            }
+            driverContext.setRetrial(true);
+            driverContext.getBackupContext().addSubContext(context);
+            driverContext.getBackupContext().setHiveLocks(context.getHiveLocks());
+            context = driverContext.getBackupContext();
+
+            driverContext.getConf().set(ValidTxnList.VALID_TXNS_KEY,
+              driverContext.getTxnManager().getValidTxns().toString());
+
+            if (driverContext.getPlan().hasAcidResourcesInQuery()) {
+              compileInternal(context.getCmd(), true);
+              driverTxnHandler.recordValidWriteIds();
+              driverTxnHandler.setWriteIdForAcidFileSinks();
+            }
+            // Since we're reusing the compiled plan, we need to update its start time for current run
+            driverContext.getPlan().setQueryStartTime(driverContext.getQueryDisplay().getQueryStartTime());
           }
-
-          driverContext.setRetrial(true);
-          driverContext.getBackupContext().addSubContext(context);
-          driverContext.getBackupContext().setHiveLocks(context.getHiveLocks());
-          context = driverContext.getBackupContext();
-
-          driverContext.getConf().set(ValidTxnList.VALID_TXNS_KEY,
-            driverContext.getTxnManager().getValidTxns().toString());
-
-          if (driverContext.getPlan().hasAcidResourcesInQuery()) {
-            compileInternal(context.getCmd(), true);
-            driverTxnHandler.recordValidWriteIds();
-            driverTxnHandler.setWriteIdForAcidFileSinks();
-          }
-          // Since we're reusing the compiled plan, we need to update its start time for current run
-          driverContext.getPlan().setQueryStartTime(driverContext.getQueryDisplay().getQueryStartTime());
-        }
+        } while (driverContext.isOutdatedTxn() && ++retryShapshotCnt <= maxRetrySnapshotCnt);

Review comment:
       That is an interesting change, do I understand correctly, that this says we only need to do a follow up check only, if the txn was outdated, because in the other cases the exclusive lock + the partition based writeset checking guarantees, that writeIds will be always valid at the second time? Could you add some comments 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.

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 #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1408,6 +1410,43 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw
     }
   }
 
+  public long getLatestTxnInConflict(long txnid) throws MetaException {
+    Connection dbConn = null;
+    Statement stmt = null;
+
+    try {
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      stmt = dbConn.createStatement();
+
+      String writeConflictQuery = "SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" +

Review comment:
       In most of the cases it will just decrease the performance. It could only be useful if there is a conflict with already registered in snapshot txn. Also won't work with dynamic partitioning as we need to re-compile and get a fresh validWriteIdList list.




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

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 #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1408,6 +1410,43 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw
     }
   }
 
+  public long getLatestTxnInConflict(long txnid) throws MetaException {
+    Connection dbConn = null;
+    Statement stmt = null;
+
+    try {
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      stmt = dbConn.createStatement();
+
+      String writeConflictQuery = "SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" +

Review comment:
       In most of the cases it will just decrease the performance. It could only be useful if there is a conflict with already registered in snapshot txn. Also won't work with dynamic partitioning as we need to recompile and get a fresh validWriteIdList list.




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

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 #1533: HIVE-24211: Replace Snapshot invalidate logic with WriteSet check for txn conflict detection

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



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/Driver.java
##########
@@ -497,38 +497,41 @@ private void runInternal(String command, boolean alreadyCompiled) throws Command
         HiveConf.ConfVars.HIVE_TXN_MAX_RETRYSNAPSHOT_COUNT);
 
       try {
-        while (!driverTxnHandler.isValidTxnListState() && ++retryShapshotCnt <= maxRetrySnapshotCnt) {
-          LOG.info("Re-compiling after acquiring locks, attempt #" + retryShapshotCnt);
-          // Snapshot was outdated when locks were acquired, hence regenerate context, txn list and retry.
-          // TODO: Lock acquisition should be moved before analyze, this is a bit hackish.
-          // Currently, we acquire a snapshot, compile the query with that snapshot, and then - acquire locks.
-          // If snapshot is still valid, we continue as usual.
-          // But if snapshot is not valid, we recompile the query.
-          if (driverContext.isOutdatedTxn()) {
-            LOG.info("Snapshot is outdated, re-initiating transaction ...");
-            driverContext.getTxnManager().rollbackTxn();
-
-            String userFromUGI = DriverUtils.getUserFromUGI(driverContext);
-            driverContext.getTxnManager().openTxn(context, userFromUGI, driverContext.getTxnType());
-            lockAndRespond();
+        do {
+          driverContext.setOutdatedTxn(false);
+
+          if (!driverTxnHandler.isValidTxnListState()) {
+            LOG.info("Re-compiling after acquiring locks, attempt #" + retryShapshotCnt);
+            // Snapshot was outdated when locks were acquired, hence regenerate context, txn list and retry.
+            // TODO: Lock acquisition should be moved before analyze, this is a bit hackish.
+            // Currently, we acquire a snapshot, compile the query with that snapshot, and then - acquire locks.
+            // If snapshot is still valid, we continue as usual.
+            // But if snapshot is not valid, we recompile the query.
+            if (driverContext.isOutdatedTxn()) {
+              LOG.info("Snapshot is outdated, re-initiating transaction ...");
+              driverContext.getTxnManager().rollbackTxn();
+
+              String userFromUGI = DriverUtils.getUserFromUGI(driverContext);
+              driverContext.getTxnManager().openTxn(context, userFromUGI, driverContext.getTxnType());
+              lockAndRespond();
+            }
+            driverContext.setRetrial(true);
+            driverContext.getBackupContext().addSubContext(context);
+            driverContext.getBackupContext().setHiveLocks(context.getHiveLocks());
+            context = driverContext.getBackupContext();
+
+            driverContext.getConf().set(ValidTxnList.VALID_TXNS_KEY,
+              driverContext.getTxnManager().getValidTxns().toString());
+
+            if (driverContext.getPlan().hasAcidResourcesInQuery()) {
+              compileInternal(context.getCmd(), true);
+              driverTxnHandler.recordValidWriteIds();
+              driverTxnHandler.setWriteIdForAcidFileSinks();
+            }
+            // Since we're reusing the compiled plan, we need to update its start time for current run
+            driverContext.getPlan().setQueryStartTime(driverContext.getQueryDisplay().getQueryStartTime());
           }
-
-          driverContext.setRetrial(true);
-          driverContext.getBackupContext().addSubContext(context);
-          driverContext.getBackupContext().setHiveLocks(context.getHiveLocks());
-          context = driverContext.getBackupContext();
-
-          driverContext.getConf().set(ValidTxnList.VALID_TXNS_KEY,
-            driverContext.getTxnManager().getValidTxns().toString());
-
-          if (driverContext.getPlan().hasAcidResourcesInQuery()) {
-            compileInternal(context.getCmd(), true);
-            driverTxnHandler.recordValidWriteIds();
-            driverTxnHandler.setWriteIdForAcidFileSinks();
-          }
-          // Since we're reusing the compiled plan, we need to update its start time for current run
-          driverContext.getPlan().setQueryStartTime(driverContext.getQueryDisplay().getQueryStartTime());
-        }
+        } while (driverContext.isOutdatedTxn() && ++retryShapshotCnt <= maxRetrySnapshotCnt);

Review comment:
       added comments




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

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] pvargacl commented on pull request #1533: HIVE-24211: Replace Snapshot invalidate logic with WriteSet check for txn conflict detection

Posted by GitBox <gi...@apache.org>.
pvargacl commented on pull request #1533:
URL: https://github.com/apache/hive/pull/1533#issuecomment-701375986


   LGTM +1


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

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] pvargacl commented on a change in pull request #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1408,6 +1410,43 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw
     }
   }
 
+  public long getLatestTxnInConflict(long txnid) throws MetaException {
+    Connection dbConn = null;
+    Statement stmt = null;
+
+    try {
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      stmt = dbConn.createStatement();
+
+      String writeConflictQuery = "SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" +

Review comment:
       This is probably a big enough change, but I was wondering if you could return all the conflicting txnIds and writeIds. In that case if the max conflicting txnId is less than the current one, you could skip two other call to the HMS for validTxnList and validWriteIdList and just remove the exceptions from the list.




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

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] pvargacl commented on a change in pull request #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1408,6 +1410,43 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw
     }
   }
 
+  public long getLatestTxnInConflict(long txnid) throws MetaException {

Review comment:
       Maybe add some javadoc, what does it do. I would emphasise that this method call only makes sense if the caller holds an exclusive lock, that blocks other txns to commit writes. And also, that this deliberately ignores inserts, or maybe this should be added to the DriverTxnHandler, that inserts will not invalidate the snapshot, that can cause duplicates 




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

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 #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1408,6 +1410,43 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw
     }
   }
 
+  public long getLatestTxnInConflict(long txnid) throws MetaException {
+    Connection dbConn = null;
+    Statement stmt = null;
+
+    try {
+      dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+      stmt = dbConn.createStatement();
+
+      String writeConflictQuery = "SELECT MAX(\"COMMITTED\".\"WS_TXNID\")" +

Review comment:
       In most of the cases it will just decrease the performance. It could only be useful if there is a conflict with already registered in snapshot txn.




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

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 #1533: HIVE-24211: Replace Snapshot invalidate logic with WriteSet check for txn conflict detection

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


   


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

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 pull request #1533: HIVE-24211: Replace Snapshot invalidate logic with WriteSet check for txn conflict detection

Posted by GitBox <gi...@apache.org>.
deniskuzZ commented on pull request #1533:
URL: https://github.com/apache/hive/pull/1533#issuecomment-701956664


   @jcamachor , could you please review 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.

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 #1533: HIVE-24211: Replace Snapshot invalidate logic with WriteSet check for txn conflict detection

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -1408,6 +1410,43 @@ private boolean isUpdateOrDelete(Statement stmt, String conflictSQLSuffix) throw
     }
   }
 
+  public long getLatestTxnInConflict(long txnid) throws MetaException {

Review comment:
       added javadoc




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

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 #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3501,6 +3541,10 @@ public void addDynamicPartitions(AddDynamicPartitions rqst)
             pstmt.executeBatch();
           }
         }
+        try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_DP_DELETE_QUERY)) {

Review comment:
       It's actually related. Operation that involve dynamic partitioning doesn't generate TXN_COMPONENTS at the locking stage, making it impossible to check for write conflict.




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

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] pvargacl commented on a change in pull request #1533: Snapshot invalidate logic replace with writeset check for conflict de…

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



##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3501,6 +3541,10 @@ public void addDynamicPartitions(AddDynamicPartitions rqst)
             pstmt.executeBatch();
           }
         }
+        try (PreparedStatement pstmt = dbConn.prepareStatement(TXN_COMPONENTS_DP_DELETE_QUERY)) {

Review comment:
       This fix is not related to the original problem, shouldn't it be committed separately?




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

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