Skip to content

HIVE-29028: Iceberg: Implement auto compaction #5886

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

Merged
merged 1 commit into from
Jul 3, 2025

Conversation

difin
Copy link
Contributor

@difin difin commented Jun 21, 2025

What changes were proposed in this pull request?

Iceberg auto compaction. In this initial version, IcebergInitiator remembers the last checked snapshot id of all Iceberg tables and it compares the current snapshot id to the cached snapshot id to determine if further evaluation is needed.

Why are the changes needed?

Currently, Iceberg compaction can only be triggered manually.
This PR introduces automatic Iceberg compaction which periodically checks all Iceberg tables and schedules compactions when necessary.

Does this PR introduce any user-facing change?

No

How was this patch tested?

New unit tests, precommit tests.

@deniskuzZ
Copy link
Member

plenty of reactions, but no reviews :)

@@ -2381,4 +2384,15 @@ private static List<FieldSchema> schema(List<VirtualColumn> exprs) {
private static List<FieldSchema> orderBy(VirtualColumn... exprs) {
return schema(Arrays.asList(exprs));
}

@Override
public CompactionType determineCompactionType(org.apache.hadoop.hive.metastore.api.Table table, CompactionInfo ci)
Copy link
Member

Choose a reason for hiding this comment

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

it doesn't belong here. maybe CompactionEvaluator?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed from HiveIcebergStorageHandler

* @throws IOException If an I/O error occurs during the compaction evaluation process.
* @throws UnsupportedOperationException if the implementing storage handler does not support this operation.
*/
default CompactionType determineCompactionType(Table table, CompactionInfo ci) throws IOException {
Copy link
Member

Choose a reason for hiding this comment

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

that is not responsibility of StorageHandler

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed

* A class to initiate Iceberg compactions. This will run in a separate thread.
* It's critical that there exactly 1 of these in a given warehouse.
*/
public class IcebergInitiator extends InitiatorBase {
Copy link
Member

Choose a reason for hiding this comment

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

please move it under iceberg-handler, org.apache.iceberg.mr.hive.compaction.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done

*/
public abstract class InitiatorBase extends MetaStoreCompactorThread {

protected ExecutorService compactionExecutor;
Copy link
Member

@deniskuzZ deniskuzZ Jun 26, 2025

Choose a reason for hiding this comment

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

Is there a way to disable certain Initiators individually? I don't think we need 2 processes, but 1 running multiple tasks:

  • AcidTableOptimizer
  • IcebergTableOptimizer

Copy link
Member

Choose a reason for hiding this comment

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

both should return a list of tasks the Initiator needs to schedule

Copy link
Contributor Author

@difin difin Jun 27, 2025

Choose a reason for hiding this comment

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

Added a way to disable certain initiators individually by introducing a new Hive conf that contains list of active initiator classes and refactored it to be one process instead of 2.

Copy link
Contributor

@okumin okumin left a comment

Choose a reason for hiding this comment

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

Mostly +1. I left some minor comments

Copy link
Contributor

@okumin okumin left a comment

Choose a reason for hiding this comment

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

Thanks for addressing the remaining issues!

Copy link

sonarqubecloud bot commented Jul 2, 2025

@difin
Copy link
Contributor Author

difin commented Jul 3, 2025

Thanks, @deniskuzZ and @okumin for code reviews!

@difin difin merged commit 279996b into apache:master Jul 3, 2025
4 checks passed
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants