Skip to content

Conversation

@krisdas
Copy link
Contributor

@krisdas krisdas commented Jun 14, 2022

While acquiring hive-metastore lock on an Iceberg table, after every timeout, below log line is printed, which doesn't have database and table name information.

org.apache.iceberg.hive.HiveTableOperations$WaitingForLockException: Waiting for lock.

After exhausting all the re-try attempt, finally below log line prints database and table name.

Retrying task after failure: Timed out after 180133 ms waiting for lock on database.table

Here we are adding the database and table name in first log line to speed up investigation related with locking.

cc : @szehon-ho @dramaticlly

@github-actions github-actions bot added the hive label Jun 14, 2022
@krisdas krisdas marked this pull request as ready for review June 14, 2022 23:54
Copy link
Contributor

@dramaticlly dramaticlly left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@abmo-x
Copy link
Contributor

abmo-x commented Jun 15, 2022

Question:

throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",

And
throw new CommitFailedException("Could not acquire the lock on %s.%s, " +

These should be logging the database.tablename if lock is not acquired in a timely manner.

is that exception getting swallowed somewhere or the msg not logged from the exception, which includes these details along with the state of the lock.

@krisdas
Copy link
Contributor Author

krisdas commented Jun 15, 2022

Question:

throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",

And

throw new CommitFailedException("Could not acquire the lock on %s.%s, " +

These should be logging the database.tablename if lock is not acquired in a timely manner.

is that exception getting swallowed somewhere or the msg not logged from the exception, which includes these details along with the state of the lock.

These logs gets printed, just that, they are printed at the very end, after exhausting all the retries. Here we are trying to add db.table details after each re-try and get the information up in the log file. Currently with just "Waiting for lock" message, we don't get details about, which db.table, the lock is requested.

@abmo-x
Copy link
Contributor

abmo-x commented Jun 15, 2022

Question:

throw new CommitFailedException("Timed out after %s ms waiting for lock on %s.%s",

And

throw new CommitFailedException("Could not acquire the lock on %s.%s, " +

These should be logging the database.tablename if lock is not acquired in a timely manner.
is that exception getting swallowed somewhere or the msg not logged from the exception, which includes these details along with the state of the lock.

These logs gets printed, just that, they are printed at the very end, after exhausting all the retries. Here we are trying to add db.table details after each re-try and get the information up in the log file. Currently with just "Waiting for lock" message, we don't get details about, which db.table, the lock is requested.

Ah! That makes sense, thanks for clarifying.

LockResponse lockResponse = metaClients.run(client -> client.lock(lockRequest));
AtomicReference<LockState> state = new AtomicReference<>(lockResponse.getState());
long lockId = lockResponse.getLockid();
final Pair<Long, String> lockDetails = Pair.of(lockId, String.format("%s.%s", database, tableName));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do not think we need this Pair object here. Could we just use database/tableName down below?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In current implementation, only a single lock is acquired, so we can use database/tableName directly inside the task/lambda. Since we are using Tasks.foreach, where multiple tasks can run in future, grouped lock id and table info in a pair (to avoid confusion). thoughts?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the Tasks.foreach(lockDetails) is only used to reuse the retry functionality of Tasks. So I think it is perfectly fine to keep the simpler implementation and using database/tableName.

Copy link
Contributor

@pvary pvary left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 pending tests

@pvary pvary changed the title Print db and table name while acquiring hive meta-store lock Hive: Print db and table name while acquiring hive meta-store lock Jun 22, 2022
@pvary pvary merged commit 08c8764 into apache:master Jun 22, 2022
@pvary
Copy link
Contributor

pvary commented Jun 22, 2022

Thanks @krisdas for the PR, and for the team for the review comments!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants