|
| 1 | +package com.altinity.ice.rest.catalog.internal.maintenance; |
| 2 | + |
| 3 | +import com.altinity.ice.rest.catalog.internal.config.Config; |
| 4 | +import com.github.shyiko.skedule.Schedule; |
| 5 | +import java.time.ZonedDateTime; |
| 6 | +import java.util.List; |
| 7 | +import java.util.Map; |
| 8 | +import java.util.concurrent.ScheduledExecutorService; |
| 9 | +import java.util.concurrent.ScheduledFuture; |
| 10 | +import java.util.concurrent.ScheduledThreadPoolExecutor; |
| 11 | +import java.util.concurrent.TimeUnit; |
| 12 | +import java.util.concurrent.atomic.AtomicBoolean; |
| 13 | +import org.apache.iceberg.Table; |
| 14 | +import org.apache.iceberg.catalog.Catalog; |
| 15 | +import org.apache.iceberg.catalog.Namespace; |
| 16 | +import org.apache.iceberg.catalog.SupportsNamespaces; |
| 17 | +import org.apache.iceberg.catalog.TableIdentifier; |
| 18 | +import org.slf4j.Logger; |
| 19 | +import org.slf4j.LoggerFactory; |
| 20 | + |
| 21 | +public class MaintenanceScheduler { |
| 22 | + private static final Logger logger = LoggerFactory.getLogger(MaintenanceScheduler.class); |
| 23 | + private static final int DEFAULT_EXPIRATION_DAYS = 30; |
| 24 | + |
| 25 | + private final Catalog catalog; |
| 26 | + private final AtomicBoolean isMaintenanceMode = new AtomicBoolean(false); |
| 27 | + private final ScheduledExecutorService executor; |
| 28 | + private final Schedule schedule; |
| 29 | + private final Object taskLock = new Object(); |
| 30 | + |
| 31 | + private ScheduledFuture<?> currentTask; |
| 32 | + private final Integer snapshotExpirationDays; |
| 33 | + |
| 34 | + public MaintenanceScheduler( |
| 35 | + Catalog catalog, Map<String, String> config, String maintenanceInterval) { |
| 36 | + this.catalog = catalog; |
| 37 | + this.executor = new ScheduledThreadPoolExecutor(1); |
| 38 | + ((ScheduledThreadPoolExecutor) executor).setRemoveOnCancelPolicy(true); |
| 39 | + this.schedule = Schedule.parse(maintenanceInterval); |
| 40 | + if (config.containsKey(Config.OPTION_SNAPSHOT_EXPIRATION_DAYS)) { |
| 41 | + this.snapshotExpirationDays = |
| 42 | + Integer.parseInt(config.get(Config.OPTION_SNAPSHOT_EXPIRATION_DAYS)); |
| 43 | + } else { |
| 44 | + this.snapshotExpirationDays = DEFAULT_EXPIRATION_DAYS; |
| 45 | + } |
| 46 | + } |
| 47 | + |
| 48 | + public void startScheduledMaintenance() { |
| 49 | + scheduleNextMaintenance(); |
| 50 | + } |
| 51 | + |
| 52 | + public void stopScheduledMaintenance() { |
| 53 | + synchronized (taskLock) { |
| 54 | + if (currentTask != null) { |
| 55 | + currentTask.cancel(false); |
| 56 | + } |
| 57 | + executor.shutdown(); |
| 58 | + } |
| 59 | + } |
| 60 | + |
| 61 | + private void scheduleNextMaintenance() { |
| 62 | + synchronized (taskLock) { |
| 63 | + if (currentTask != null) { |
| 64 | + currentTask.cancel(false); |
| 65 | + } |
| 66 | + |
| 67 | + ZonedDateTime now = ZonedDateTime.now(); |
| 68 | + ZonedDateTime next = schedule.next(now); |
| 69 | + |
| 70 | + long delay = next.toEpochSecond() - now.toEpochSecond(); |
| 71 | + currentTask = |
| 72 | + executor.schedule( |
| 73 | + () -> { |
| 74 | + performMaintenance(); |
| 75 | + scheduleNextMaintenance(); // Schedule next run |
| 76 | + }, |
| 77 | + delay, |
| 78 | + TimeUnit.SECONDS); |
| 79 | + |
| 80 | + logger.info("Next maintenance scheduled for: {}", next); |
| 81 | + } |
| 82 | + } |
| 83 | + |
| 84 | + public void performMaintenance() { |
| 85 | + if (isMaintenanceMode.get()) { |
| 86 | + logger.info("Skipping maintenance task as system is already in maintenance mode"); |
| 87 | + return; |
| 88 | + } |
| 89 | + |
| 90 | + try { |
| 91 | + logger.info("Starting scheduled maintenance task"); |
| 92 | + setMaintenanceMode(true); |
| 93 | + |
| 94 | + if (catalog != null) { |
| 95 | + logger.info("Performing maintenance on catalog: {}", catalog.name()); |
| 96 | + List<Namespace> namespaces; |
| 97 | + if (catalog instanceof SupportsNamespaces) { |
| 98 | + SupportsNamespaces nsCatalog = (SupportsNamespaces) catalog; |
| 99 | + namespaces = nsCatalog.listNamespaces(); |
| 100 | + for (Namespace ns : namespaces) { |
| 101 | + logger.debug("Namespace: " + ns); |
| 102 | + } |
| 103 | + } else { |
| 104 | + logger.error("Catalog does not support namespace operations."); |
| 105 | + return; |
| 106 | + } |
| 107 | + |
| 108 | + for (Namespace namespace : namespaces) { |
| 109 | + List<TableIdentifier> tables = catalog.listTables(namespace); |
| 110 | + for (TableIdentifier tableIdent : tables) { |
| 111 | + long olderThanMillis = |
| 112 | + System.currentTimeMillis() - TimeUnit.DAYS.toMillis(snapshotExpirationDays); |
| 113 | + Table table = catalog.loadTable(tableIdent); |
| 114 | + |
| 115 | + // Check if table has any snapshots before performing maintenance |
| 116 | + if (table.currentSnapshot() == null) { |
| 117 | + logger.warn("Table {} has no snapshots, skipping maintenance", tableIdent); |
| 118 | + continue; |
| 119 | + } |
| 120 | + |
| 121 | + table.rewriteManifests().rewriteIf(manifest -> true).commit(); |
| 122 | + table.expireSnapshots().expireOlderThan(olderThanMillis).commit(); |
| 123 | + } |
| 124 | + } |
| 125 | + logger.info("Maintenance operations completed for catalog: {}", catalog.name()); |
| 126 | + } else { |
| 127 | + logger.warn("No catalog available for maintenance operations"); |
| 128 | + } |
| 129 | + |
| 130 | + logger.info("Scheduled maintenance task completed successfully"); |
| 131 | + } catch (Exception e) { |
| 132 | + logger.error("Error during scheduled maintenance task", e); |
| 133 | + } finally { |
| 134 | + setMaintenanceMode(false); |
| 135 | + } |
| 136 | + } |
| 137 | + |
| 138 | + private void setMaintenanceMode(boolean enabled) { |
| 139 | + isMaintenanceMode.set(enabled); |
| 140 | + logger.info("Maintenance mode {}", enabled ? "enabled" : "disabled"); |
| 141 | + } |
| 142 | +} |
0 commit comments