-
Notifications
You must be signed in to change notification settings - Fork 3
Added quartz scheduler to schedule maintenance job. #17
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
shyiko
merged 7 commits into
master
from
1-ice-rest-catalog-automate-catalog-maintenance
Apr 29, 2025
Merged
Changes from all commits
Commits
Show all changes
7 commits
Select commit
Hold shift + click to select a range
9162359
Added quartz scheduler to schedule maintenance job.
subkanthi e4b025b
Added logic to perform maintenance functions, expireSnapshots and rew…
subkanthi f438344
Merged changes from master.
subkanthi 56640d4
Use skeduler library instead of quartz to setup maintenance schedules.
subkanthi 189ab19
ice-catalog: review comments, added synchronization to prevent curren…
subkanthi 590b636
ice-catalog: review comments, added synchronization to prevent curren…
subkanthi 76eb25b
ice: Moved logic of reading config value of snapshot expiration days …
subkanthi File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
142 changes: 142 additions & 0 deletions
142
...rc/main/java/com/altinity/ice/rest/catalog/internal/maintenance/MaintenanceScheduler.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,142 @@ | ||
package com.altinity.ice.rest.catalog.internal.maintenance; | ||
|
||
import com.altinity.ice.rest.catalog.internal.config.Config; | ||
import com.github.shyiko.skedule.Schedule; | ||
import java.time.ZonedDateTime; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.concurrent.ScheduledFuture; | ||
import java.util.concurrent.ScheduledThreadPoolExecutor; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import org.apache.iceberg.Table; | ||
import org.apache.iceberg.catalog.Catalog; | ||
import org.apache.iceberg.catalog.Namespace; | ||
import org.apache.iceberg.catalog.SupportsNamespaces; | ||
import org.apache.iceberg.catalog.TableIdentifier; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class MaintenanceScheduler { | ||
private static final Logger logger = LoggerFactory.getLogger(MaintenanceScheduler.class); | ||
private static final int DEFAULT_EXPIRATION_DAYS = 30; | ||
|
||
private final Catalog catalog; | ||
private final AtomicBoolean isMaintenanceMode = new AtomicBoolean(false); | ||
private final ScheduledExecutorService executor; | ||
private final Schedule schedule; | ||
private final Object taskLock = new Object(); | ||
|
||
private ScheduledFuture<?> currentTask; | ||
private final Integer snapshotExpirationDays; | ||
|
||
public MaintenanceScheduler( | ||
Catalog catalog, Map<String, String> config, String maintenanceInterval) { | ||
this.catalog = catalog; | ||
this.executor = new ScheduledThreadPoolExecutor(1); | ||
((ScheduledThreadPoolExecutor) executor).setRemoveOnCancelPolicy(true); | ||
this.schedule = Schedule.parse(maintenanceInterval); | ||
if (config.containsKey(Config.OPTION_SNAPSHOT_EXPIRATION_DAYS)) { | ||
this.snapshotExpirationDays = | ||
Integer.parseInt(config.get(Config.OPTION_SNAPSHOT_EXPIRATION_DAYS)); | ||
} else { | ||
this.snapshotExpirationDays = DEFAULT_EXPIRATION_DAYS; | ||
} | ||
} | ||
|
||
public void startScheduledMaintenance() { | ||
scheduleNextMaintenance(); | ||
} | ||
|
||
public void stopScheduledMaintenance() { | ||
synchronized (taskLock) { | ||
if (currentTask != null) { | ||
currentTask.cancel(false); | ||
} | ||
executor.shutdown(); | ||
} | ||
} | ||
|
||
private void scheduleNextMaintenance() { | ||
synchronized (taskLock) { | ||
if (currentTask != null) { | ||
currentTask.cancel(false); | ||
} | ||
|
||
ZonedDateTime now = ZonedDateTime.now(); | ||
ZonedDateTime next = schedule.next(now); | ||
|
||
long delay = next.toEpochSecond() - now.toEpochSecond(); | ||
currentTask = | ||
executor.schedule( | ||
() -> { | ||
performMaintenance(); | ||
scheduleNextMaintenance(); // Schedule next run | ||
}, | ||
delay, | ||
TimeUnit.SECONDS); | ||
|
||
logger.info("Next maintenance scheduled for: {}", next); | ||
} | ||
} | ||
|
||
public void performMaintenance() { | ||
if (isMaintenanceMode.get()) { | ||
logger.info("Skipping maintenance task as system is already in maintenance mode"); | ||
return; | ||
} | ||
|
||
try { | ||
logger.info("Starting scheduled maintenance task"); | ||
setMaintenanceMode(true); | ||
|
||
if (catalog != null) { | ||
logger.info("Performing maintenance on catalog: {}", catalog.name()); | ||
List<Namespace> namespaces; | ||
if (catalog instanceof SupportsNamespaces) { | ||
SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; | ||
namespaces = nsCatalog.listNamespaces(); | ||
for (Namespace ns : namespaces) { | ||
logger.debug("Namespace: " + ns); | ||
} | ||
} else { | ||
logger.error("Catalog does not support namespace operations."); | ||
return; | ||
} | ||
|
||
for (Namespace namespace : namespaces) { | ||
List<TableIdentifier> tables = catalog.listTables(namespace); | ||
for (TableIdentifier tableIdent : tables) { | ||
long olderThanMillis = | ||
System.currentTimeMillis() - TimeUnit.DAYS.toMillis(snapshotExpirationDays); | ||
Table table = catalog.loadTable(tableIdent); | ||
|
||
// Check if table has any snapshots before performing maintenance | ||
if (table.currentSnapshot() == null) { | ||
logger.warn("Table {} has no snapshots, skipping maintenance", tableIdent); | ||
continue; | ||
} | ||
|
||
table.rewriteManifests().rewriteIf(manifest -> true).commit(); | ||
table.expireSnapshots().expireOlderThan(olderThanMillis).commit(); | ||
} | ||
} | ||
logger.info("Maintenance operations completed for catalog: {}", catalog.name()); | ||
} else { | ||
logger.warn("No catalog available for maintenance operations"); | ||
} | ||
|
||
logger.info("Scheduled maintenance task completed successfully"); | ||
} catch (Exception e) { | ||
logger.error("Error during scheduled maintenance task", e); | ||
} finally { | ||
setMaintenanceMode(false); | ||
} | ||
} | ||
|
||
private void setMaintenanceMode(boolean enabled) { | ||
isMaintenanceMode.set(enabled); | ||
logger.info("Maintenance mode {}", enabled ? "enabled" : "disabled"); | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
it would be nice to have maintenance optional (e.g. disable when maintenanceInterval is empty)
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.
logic added inside the function