Skip to content
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

Make snapshots expiration job leaner #131

Merged
merged 3 commits into from
Jul 8, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.Transaction;
import org.apache.iceberg.actions.DeleteOrphanFiles;
import org.apache.iceberg.actions.ExpireSnapshots;
import org.apache.iceberg.actions.RewriteDataFiles;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.TableIdentifier;
Expand Down Expand Up @@ -201,22 +200,18 @@ public void deleteStagedOrphanDirectory(
}
}

/** Run ExpireSnapshots operation on a given fully-qualified table name. */
public ExpireSnapshots.Result expireSnapshots(String fqtn, long expireBeforeTimestampMs) {
return expireSnapshots(getTable(fqtn), expireBeforeTimestampMs);
/** Expire snapshots on a given fully-qualified table name. */
public void expireSnapshots(String fqtn, long expireBeforeTimestampMs) {
expireSnapshots(getTable(fqtn), expireBeforeTimestampMs);
}

/** Run ExpireSnapshots operation on a given {@link Table}. */
public ExpireSnapshots.Result expireSnapshots(Table table, long expireBeforeTimestampMs) {
return SparkActions.get(spark)
.expireSnapshots(table)
/** Expire snapshots on a given {@link Table}. */
public void expireSnapshots(Table table, long expireBeforeTimestampMs) {
table
.expireSnapshots()
.cleanExpiredFiles(false)
.expireOlderThan(expireBeforeTimestampMs)
.deleteWith(
teamurko marked this conversation as resolved.
Show resolved Hide resolved
(file) -> {
// skip deletion
log.info("Detected file {} that is not part of survived snapshots", file);
})
.execute();
.commit();
teamurko marked this conversation as resolved.
Show resolved Hide resolved
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -1,16 +1,12 @@
package com.linkedin.openhouse.jobs.spark;

import com.linkedin.openhouse.jobs.spark.state.StateManager;
import com.linkedin.openhouse.jobs.util.AppConstants;
import io.opentelemetry.api.common.AttributeKey;
import io.opentelemetry.api.common.Attributes;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.Option;
import org.apache.iceberg.actions.ExpireSnapshots;

/**
* Class with main entry point to run as a table snapshot expiration job. Snapshots for table which
Expand Down Expand Up @@ -40,42 +36,7 @@ protected void runInner(Operations ops) {
granularity);
long expireBeforeTimestampMs = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(count);
log.info("Expire snapshots before timestamp ms {}", expireBeforeTimestampMs);
ExpireSnapshots.Result result = ops.expireSnapshots(fqtn, expireBeforeTimestampMs);
log.info(
"Detected {} data files, {} manifest files, {} manifest list files that become orphaned as the result of snapshots expiration",
result.deletedDataFilesCount(),
result.deletedManifestsCount(),
result.deletedManifestListsCount());
METER
.counterBuilder(AppConstants.EXPIRED_FILE_COUNT)
teamurko marked this conversation as resolved.
Show resolved Hide resolved
.build()
.add(
result.deletedDataFilesCount(),
Attributes.of(
AttributeKey.stringKey(AppConstants.TABLE_NAME),
fqtn,
AttributeKey.stringKey(AppConstants.TYPE),
AppConstants.DATA_FILES));
METER
.counterBuilder(AppConstants.EXPIRED_FILE_COUNT)
.build()
.add(
result.deletedManifestsCount(),
Attributes.of(
AttributeKey.stringKey(AppConstants.TABLE_NAME),
fqtn,
AttributeKey.stringKey(AppConstants.TYPE),
AppConstants.MANIFEST_FILES));
METER
.counterBuilder(AppConstants.EXPIRED_FILE_COUNT)
.build()
.add(
result.deletedManifestListsCount(),
Attributes.of(
AttributeKey.stringKey(AppConstants.TABLE_NAME),
fqtn,
AttributeKey.stringKey(AppConstants.TYPE),
AppConstants.MANIFEST_LIST_FILES));
ops.expireSnapshots(fqtn, expireBeforeTimestampMs);
}

public static void main(String[] args) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,6 @@ public final class AppConstants {
// Spark App observability constants
public static final String TYPE = "type";
public static final String JOB_TYPE = "job_type";
public static final String DATA_FILES = "data_files";
public static final String MANIFEST_FILES = "manifest_files";
public static final String MANIFEST_LIST_FILES = "manifest_list_files";
public static final String EXPIRED_FILE_COUNT = "expired_file_count";
public static final String ORPHAN_FILE_COUNT = "orphan_file_count";
public static final String STAGED_FILE_COUNT = "staged_file_count";
public static final String ORPHAN_DIRECTORY_COUNT = "orphan_directory_count";
Expand Down
Loading