You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dk...@apache.org on 2022/07/13 10:55:54 UTC
[hive] branch master updated: HIVE-26384: Compactor worker should not trigger stop heartbeat for an uninitialized TXN (Yu-Wen Lai, reviewed by Denys Kuzmenko)
This is an automated email from the ASF dual-hosted git repository.
dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new fe99d76adb9 HIVE-26384: Compactor worker should not trigger stop heartbeat for an uninitialized TXN (Yu-Wen Lai, reviewed by Denys Kuzmenko)
fe99d76adb9 is described below
commit fe99d76adb9ccf3f95efc8e78dc94b65b530c366
Author: Yu-Wen <hs...@users.noreply.github.com>
AuthorDate: Wed Jul 13 03:55:48 2022 -0700
HIVE-26384: Compactor worker should not trigger stop heartbeat for an uninitialized TXN (Yu-Wen Lai, reviewed by Denys Kuzmenko)
Closes #3428
---
.../org/apache/hadoop/hive/ql/txn/compactor/Worker.java | 17 +++++++++--------
1 file changed, 9 insertions(+), 8 deletions(-)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 707ebff393c..26f7f9c0eba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -640,7 +640,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
private long lockId = 0;
private TxnStatus status = TxnStatus.UNKNOWN;
- private boolean succeessfulCompaction = false;
+ private boolean successfulCompaction = false;
/**
* Try to open a new txn.
@@ -664,7 +664,7 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
* Mark compaction as successful. This means the txn will be committed; otherwise it will be aborted.
*/
void wasSuccessful() {
- this.succeessfulCompaction = true;
+ this.successfulCompaction = true;
}
/**
@@ -672,16 +672,17 @@ public class Worker extends RemoteCompactorThread implements MetaStoreThread {
* @throws Exception
*/
@Override public void close() throws Exception {
+ if (status == TxnStatus.UNKNOWN) {
+ return;
+ }
try {
//the transaction is about to close, we can stop heartbeating regardless of it's state
CompactionHeartbeatService.getInstance(conf).stopHeartbeat(txnId);
} finally {
- if (status != TxnStatus.UNKNOWN) {
- if (succeessfulCompaction) {
- commit();
- } else {
- abort();
- }
+ if (successfulCompaction) {
+ commit();
+ } else {
+ abort();
}
}
}