-
Notifications
You must be signed in to change notification settings - Fork 4.8k
HIVE-28930: Implement a metastore service that expires iceberg table snapshots periodically #5786
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Conversation
@deniskuzZ : this is the reusable, general part of the iceberg table maintenance service (no query history bits can be found here), I would appreciate a review in the future once you have time for that |
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergUtil.java
Outdated
Show resolved
Hide resolved
...tore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/TableFetcher.java
Show resolved
Hide resolved
...ndler/src/main/java/org/apache/iceberg/mr/hive/metastore/task/IcebergHouseKeeperService.java
Outdated
Show resolved
Hide resolved
iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergUtil.java
Outdated
Show resolved
Hide resolved
...ndler/src/main/java/org/apache/iceberg/mr/hive/metastore/task/IcebergHouseKeeperService.java
Outdated
Show resolved
Hide resolved
...ndler/src/main/java/org/apache/iceberg/mr/hive/metastore/task/IcebergHouseKeeperService.java
Outdated
Show resolved
Hide resolved
...ndler/src/main/java/org/apache/iceberg/mr/hive/metastore/task/IcebergHouseKeeperService.java
Show resolved
Hide resolved
...ndler/src/main/java/org/apache/iceberg/mr/hive/metastore/task/IcebergHouseKeeperService.java
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In general LGTM, minor comments
ICEBERG_TABLE_EXPIRY_INTERVAL("metastore.iceberg.table.expiry.interval", | ||
"hive.metastore.iceberg.table.expiry.interval", 3600, TimeUnit.SECONDS, | ||
"Time interval describing how often the iceberg table expiry service runs."), | ||
ICEBERG_TABLE_EXPIRY_CATALOG_NAME("metastore.iceberg.table.expiry.catalog.name", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, I am, of course.
"By default, the pattern is set to empty string, which results in no matches (this is intentional" + | ||
"to avoid expensive metastore calls unless explicitly configured by the user)."), | ||
ICEBERG_TABLE_EXPIRY_TABLE_PATTERN("metastore.iceberg.table.expiry.table.pattern", | ||
"hive.metastore.iceberg.table.expiry.table.pattern", "none", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can this be the empty string, too?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah
if (deleteExecutorService != null) { | ||
expireSnapshots.executeDeleteWith(deleteExecutorService); | ||
} | ||
expireSnapshots.commit(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would skip not only ExpireSnapshots#executeDeleteWith
but also icebergTable.expireSnapshots()
and ExpireSnapshots#commit
. That's because the Iceberg library might implement some side effects in the future, especially on commit.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do you mean something like this?
if (deleteExecutorService != null) {
ExpireSnapshots expireSnapshots = icebergTable.expireSnapshots();
expireSnapshots.executeDeleteWith(deleteExecutorService);
expireSnapshots.commit();
}
so, basically, setting hive.iceberg.expire.snapshot.numthreads=0 to completely turn this housekeeper service off?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, I thought executeDeleteWith
would perform the side effect. Now, I understand it just configures the executor. We don't need to make any changes. My bad
https://github.com/apache/iceberg/blob/apache-iceberg-1.9.1/api/src/main/java/org/apache/iceberg/ExpireSnapshots.java#L87-L100
} | ||
} catch (Exception e) { | ||
LOG.error("Exception while running iceberg expiry service on catalog/db/table: {}/{}/{}", | ||
catalogName, dbPattern, tablePattern, e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We may move the try-catch block within the for-loop because ExpireSnapshots#commit
can fail due to a conflict.
https://github.com/apache/iceberg/blob/apache-iceberg-1.9.1/api/src/main/java/org/apache/iceberg/PendingUpdate.java#L43-L55
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
makes sense, currently, if a table fails to be committed, none of the remaining tables will be attempted, which is not the desired behavior
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM 👍
@abstractdog It sounds fine to me. The build failed. Please take a look and fix it.
|
eh, sorry, committing one more |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good to me if CI is successful
|
What changes were proposed in this pull request?
This patch introduces a metastore task as a MetastoreTaskThread that can expire snapshots of iceberg tables periodically according to configuration: catalog name, database pattern, table pattern. The configuration was inspired by the partition management task.
Patch contents:
Why are the changes needed?
This service could act as a convenient helper to maintain iceberg tables, which otherwise need explicit hive ql statements by the user.
Does this PR introduce any user-facing change?
No.
How was this patch tested?
Unit tests added.
Manual testing is also possible, as the patch adds MiniHS2 capability and fixes to run metastore tasks in remote mode, example command: