-
Notifications
You must be signed in to change notification settings - Fork 4.7k
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 |
private HiveIcebergUtil() { | ||
} | ||
|
||
public static ExecutorService getDeleteExecutorService(String completeName, int numThreads) { |
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.
Could this be moved to IcebergTableUtil
? Also maybe rename to newDeleteThreadPool
...tore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/TableFetcher.java
Show resolved
Hide resolved
* @param icebergTable the iceberg Table reference | ||
*/ | ||
private void expireSnapshotsForTable(Table icebergTable) { | ||
LOG.info("Expire snapshots for: {}", icebergTable); |
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.
That seems redundant as we log the same thing a bit lower
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
Show resolved
Hide resolved
String tablePattern = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.ICEBERG_TABLE_EXPIRY_TABLE_PATTERN); | ||
|
||
try (IMetaStoreClient msc = new HiveMetaStoreClient(conf)) { | ||
// TODO: Future improvement – modify TableFetcher to return HMS Table API objects directly, |
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 sence
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.
created HIVE-28952 and changing the comment to
// TODO: HIVE-28952 – modify ...
try (IMetaStoreClient msc = new HiveMetaStoreClient(conf)) { | ||
// TODO: Future improvement – modify TableFetcher to return HMS Table API objects directly, | ||
// avoiding the need for subsequent msc.getTable calls to fetch each matched table individually | ||
List<TableName> tables = getTableFetcher(msc, catalogName, dbPattern, tablePattern).getTables(); |
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 we add a cache for iceberg Table
obj to avoid frequent requests?
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.
Maybe I could introduce a simple time-based cache, where a table record is evicted once a day. This might be a good trade-off between cache efficiency and stale object risk.
I’d prefer not to make this period configurable, as it feels like too much of an implementation detail leaking into the configuration surface.
Are you okay with that?
public void run() { | ||
LOG.debug("Running IcebergHouseKeeperService..."); | ||
|
||
String catalogName = MetastoreConf.getVar(conf, MetastoreConf.ConfVars.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.
please add enforceMutex
@Override
public void enforceMutex(boolean enableMutex) {
this.shouldUseMutex = enableMutex;
}
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
…snapshots periodically
thanks for the review so far, addressed your comments in dd76d54 |
|
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: