rdblue commented on a change in pull request #1873:
URL: https://github.com/apache/iceberg/pull/1873#discussion_r538895183
##########
File path:
hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
##########
@@ -304,32 +330,57 @@ private long acquireLock() throws UnknownHostException,
TException, InterruptedE
System.getProperty("user.name"),
InetAddress.getLocalHost().getHostName());
LockResponse lockResponse = metaClients.run(client ->
client.lock(lockRequest));
- LockState state = lockResponse.getState();
+ AtomicReference<LockState> state = new
AtomicReference<>(lockResponse.getState());
long lockId = lockResponse.getLockid();
final long start = System.currentTimeMillis();
long duration = 0;
boolean timeout = false;
- while (!timeout && state.equals(LockState.WAITING)) {
- lockResponse = metaClients.run(client -> client.checkLock(lockId));
- state = lockResponse.getState();
- // check timeout
- duration = System.currentTimeMillis() - start;
- if (duration > lockAcquireTimeout) {
+ if (state.get().equals(LockState.WAITING)) {
+ try {
+ Tasks.foreach(lockId)
+ .retry(Integer.MAX_VALUE - 100) // Endless retries bound by
timeouts. Tasks.retry adds 1 for "first try".
+ .exponentialBackoff(
+ lockCheckMinWaitTime,
+ lockCheckMaxWaitTime,
+ lockAcquireTimeout,
+ lockCheckBackoffScaleFactor)
+ .throwFailureWhenFinished()
+ .onlyRetryOn(WaitingForLockException.class)
+ .run(id -> {
+ try {
+ LockResponse response = metaClients.run(client ->
client.checkLock(id));
+ LockState newState = response.getState();
+ state.set(newState);
+ if (newState.equals(LockState.WAITING)) {
+ throw WAITING_FOR_LOCK_EXCEPTION;
+ }
+ } catch (InterruptedException | TException e) {
Review comment:
I'd probably opt to suppress the interrupt and let the code carry on
after setting that the thread was interrupted. That results in a
`CommitFailedException`. I don't think that preserving the stack of the
`InterruptedException` is really needed, but I'm fin with it this way if you
prefer 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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]