You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/02/22 02:41:40 UTC

[GitHub] [iceberg] ZorTsou opened a new pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

ZorTsou opened a new pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263


   Currently, there is no way to call `unlock` if `HiveTableOperations.acquireLock` fails at waiting for lock on hive table. This PR aims to try to invoke `unlock` in the finally block.


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ZorTsou commented on pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
ZorTsou commented on pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#issuecomment-783912599


   > Actually, isn't the unlock already called within `cleanupMetadataAndUnlock` at the moment?
   
   In `doCommit`, lockId will be lost when an exception is thrown from `acquireLock`.
   


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#discussion_r582187686



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));

Review comment:
       Okay, and passing the lock ID into unlock ensures that this is only for the lock request that was created when trying to acquire. Sounds good to me. Thanks for the explanation, @ZorTsou!




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));
+      }

Review comment:
       It is a little bit hard to see the final state of the code based on the PR, but I think it is this:
   ```
       } catch (WaitingForLockException waitingForLockException) {
         // timeout and do not have lock acquired
         throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
             System.currentTimeMillis() - start, database, tableName);
       } finally {
         if (!state.get().equals(LockState.ACQUIRED)) {
           unlock(Optional.of(lockId));
         }
       }
   
       if (!state.get().equals(LockState.ACQUIRED)) {
         throw new CommitFailedException("Could not acquire the lock on %s.%s, " +
             "lock request ended in state %s", database, tableName, state);
       }
       return lockId;
     }
   ```
   
   I have preferred the original wording where the exceptions are handled in one place. Maybe something like this:
   ```
       } catch (WaitingForLockException waitingForLockException) {
         timeout = true;
         duration = System.currentTimeMillis() - start;
       } finally {
         if (!state.get().equals(LockState.ACQUIRED)) {
           unlock(Optional.of(lockId));
         }
       }
   
       // timeout and do not have lock acquired
       if (timeout && !state.get().equals(LockState.ACQUIRED)) {
         throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
             duration, database, tableName);
       }
   
       if (!state.get().equals(LockState.ACQUIRED)) {
         throw new CommitFailedException("Could not acquire the lock on %s.%s, " +
             "lock request ended in state %s", database, tableName, state);
       }
   
       return lockId;
     }
   ```
   
   What do you think @ZorTsou?
   
   Oh, and nice catch BTW! 👍 




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));
+      }

Review comment:
       It is a little bit hard to see the final state of the code based on the PR, but I think it is this:
   ```
       } catch (WaitingForLockException waitingForLockException) {
         // timeout and do not have lock acquired
         throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
             System.currentTimeMillis() - start, database, tableName);
       } finally {
         if (!state.get().equals(LockState.ACQUIRED)) {
           unlock(Optional.of(lockId));
         }
       }
   
       if (!state.get().equals(LockState.ACQUIRED)) {
         throw new CommitFailedException("Could not acquire the lock on %s.%s, " +
             "lock request ended in state %s", database, tableName, state);
       }
       return lockId;
     }
   ```
   
   I have preferred the original wording where the exceptions are handled in one place. Maybe something like this:
   ```
       } catch (WaitingForLockException waitingForLockException) {
         timeout = true;
         duration = System.currentTimeMillis() - start;
       } finally {
         if (!state.get().equals(LockState.ACQUIRED)) {
           unlock(Optional.of(lockId));
         }
       }
   
       // timeout and do not have lock acquired
       if (timeout && !state.get().equals(LockState.ACQUIRED)) {
         throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
             duration, database, tableName);
       }
   
       if (!state.get().equals(LockState.ACQUIRED)) {
         throw new CommitFailedException("Could not acquire the lock on %s.%s, " +
             "lock request ended in state %s", database, tableName, state);
       }
   
       return lockId;
     }
   ```
   
   What do you think @ZorTsou?




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#discussion_r580338086



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired

Review comment:
       nit: I think this is a 'legacy' comment from an earlier implementation of the timeout/retry logic. Now if you get here, the lock state is always WAITING, so it's sufficient to say it's due to timeout and the 'do not have lock acquired' part can be removed




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#issuecomment-785271513


   Good catch, and thank you for fixing this, @ZorTsou!
   
   Thanks for the review, @pvary!


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue edited a comment on pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
rdblue edited a comment on pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#issuecomment-785271513


   Good catch, and thank you for fixing this, @ZorTsou!
   
   Thanks for the reviews, @pvary and @marton-bod!


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#discussion_r580698463



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));

Review comment:
       Is this correct? If the lock wasn't acquired, why would it be correct to unlock it?




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

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



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));

Review comment:
       The cleanup process will remove these locks when the timeout is reached, but until that time this can block other lock requests. So it is good to clean these locks 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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ZorTsou commented on pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
ZorTsou commented on pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#issuecomment-785526677


   Thanks for reviewing, @marton-bod @pvary @rdblue.


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ZorTsou commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
ZorTsou commented on a change in pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#discussion_r581001076



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));
+      }

Review comment:
       I agree. It looks more clear. I updated the code, thanks.




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
marton-bod commented on pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#issuecomment-783469200


   Isn't the unlock already called within `cleanupMetadataAndUnlock` at the moment?


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263


   


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ZorTsou commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
ZorTsou commented on a change in pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#discussion_r580864911



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));

Review comment:
       While client calling lock, hive metastore creates an exclusive lock with `WAITING` state on the hive table first. If there is no other lock on the table, then metastore changes the state to `ACQUIRED`. Otherwise, updates `hl_blockedby_ext_id` to the latest lockId. No matter whether the lock state is acquired, the lock information will stored in HIVE_LOCKS as below.
   
   hl_lock_ext_id | hl_table | hl_lock_state | hl_lock_type | hl_last_heartbeat | hl_blockedby_ext_id
   -- | -- | -- | -- | -- | --
   1651514 | test_table | a | e | 1614062112285 |  
   1651516 | test_table | w | e | 1614062132342 | 1651514
   1651518 | test_table | w | e | 1614062152426 | 1651516
   
   In this case, a new lock will remain in the waiting state until the latest lock is deleted by client or txn cleanup thread.




----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod removed a comment on pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
marton-bod removed a comment on pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#issuecomment-783469200


   Isn't the unlock already called within `cleanupMetadataAndUnlock` at the moment?


----------------------------------------------------------------
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: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #2263: Hive: make sure unlock is called when HiveTableOperations can not acq…

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #2263:
URL: https://github.com/apache/iceberg/pull/2263#discussion_r580876984



##########
File path: hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -348,16 +345,15 @@ private long acquireLock() throws UnknownHostException, TException, InterruptedE
                 LOG.warn("Interrupted while waiting for lock.", e);
               }
             }, TException.class);
-      } catch (WaitingForLockException waitingForLockException) {
-        timeout = true;
-        duration = System.currentTimeMillis() - start;
       }
-    }
-
-    // timeout and do not have lock acquired
-    if (timeout && !state.get().equals(LockState.ACQUIRED)) {
+    } catch (WaitingForLockException waitingForLockException) {
+      // timeout and do not have lock acquired
       throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",
-          duration, database, tableName);
+          System.currentTimeMillis() - start, database, tableName);
+    } finally {
+      if (!state.get().equals(LockState.ACQUIRED)) {
+        unlock(Optional.of(lockId));

Review comment:
       Correct, `HiveMetaStoreClient.lock()` will enqueue a new lock request into the HMS `HIVE_LOCKS` table and if we throw an exception here upon timeout, our lock request will be stuck there in `WAITING` state (potentially blocking other subsequent requests) unless we call `unlock()` which makes sure to remove the lock from the HMS 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.

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org