From 50273189e4b4137366b44a9065160b2bea4e2001 Mon Sep 17 00:00:00 2001 From: Devesh Kumar Singh Date: Mon, 4 May 2026 22:18:07 +0530 Subject: [PATCH 1/2] HDDS-15165. Recon: Add admin REST APIs to trigger, monitor, and cancel SCM DB snapshot sync. --- .../recon/api/TriggerDBSyncEndpoint.java | 34 +- .../ReconStorageContainerManagerFacade.java | 308 +++++++++++++++++- .../recon/api/TestTriggerDBSyncEndpoint.java | 100 +++++- 3 files changed, 429 insertions(+), 13 deletions(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TriggerDBSyncEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TriggerDBSyncEndpoint.java index 4f91b01db87a..07af7b7844d7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TriggerDBSyncEndpoint.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/TriggerDBSyncEndpoint.java @@ -19,10 +19,13 @@ import javax.inject.Inject; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.OzoneManagerServiceProvider; /** @@ -34,11 +37,14 @@ public class TriggerDBSyncEndpoint { private OzoneManagerServiceProvider ozoneManagerServiceProvider; + private ReconStorageContainerManagerFacade reconScm; @Inject public TriggerDBSyncEndpoint( - OzoneManagerServiceProvider ozoneManagerServiceProvider) { + OzoneManagerServiceProvider ozoneManagerServiceProvider, + OzoneStorageContainerManager reconScm) { this.ozoneManagerServiceProvider = ozoneManagerServiceProvider; + this.reconScm = (ReconStorageContainerManagerFacade) reconScm; } @GET @@ -48,4 +54,30 @@ public Response triggerOMDBSync() { ozoneManagerServiceProvider.triggerSyncDataFromOMImmediately(); return Response.ok(isSuccess).build(); } + + @POST + @Path("scm/snapshot") + public Response triggerSCMDBSnapshotSync() { + ReconStorageContainerManagerFacade.ScmDbSnapshotTriggerResponse response = + reconScm.triggerScmDbSnapshotSync(); + return response.isAccepted() + ? Response.accepted(response).build() + : Response.status(Response.Status.CONFLICT).entity(response).build(); + } + + @GET + @Path("scm/snapshot/status") + public Response getSCMDBSnapshotSyncStatus() { + return Response.ok(reconScm.getScmDbSnapshotSyncStatus()).build(); + } + + @POST + @Path("scm/snapshot/cancel") + public Response cancelSCMDBSnapshotSync() { + ReconStorageContainerManagerFacade.ScmDbSnapshotCancelResponse response = + reconScm.cancelScmDbSnapshotSync(); + return response.isCancelled() + ? Response.ok(response).build() + : Response.status(Response.Status.CONFLICT).entity(response).build(); + } } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java index 278bac0011dc..93d1fb29d7fc 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java @@ -51,7 +51,9 @@ import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -168,6 +170,144 @@ public class ReconStorageContainerManagerFacade private AtomicBoolean isSyncDataFromSCMRunning; private final String threadNamePrefix; private final ReconStorageContainerSyncHelper containerSyncHelper; + private final ExecutorService scmSnapshotExecutor; + private final Object scmSnapshotLock = new Object(); + private Future scmSnapshotFuture; + private ScmDbSnapshotSyncStatus scmSnapshotStatus = + ScmDbSnapshotSyncStatus.IDLE; + private ScmDbSnapshotSyncPhase scmSnapshotPhase = + ScmDbSnapshotSyncPhase.NONE; + private long scmSnapshotStartedAt; + private long scmSnapshotFinishedAt; + private boolean scmSnapshotCancelAllowed; + private boolean scmSnapshotTaskStarted; + private String scmSnapshotLastError; + + public enum ScmDbSnapshotSyncStatus { + IDLE, + IN_PROGRESS, + SUCCESS, + FAILED, + CANCELLED + } + + public enum ScmDbSnapshotSyncPhase { + NONE, + DOWNLOADING_CHECKPOINT, + INITIALIZING_DB, + SWAPPING_DB, + COMPLETED, + FAILED, + CANCELLED + } + + public static final class ScmDbSnapshotStatusResponse { + private final ScmDbSnapshotSyncStatus status; + private final ScmDbSnapshotSyncPhase phase; + private final long startedAt; + private final long finishedAt; + private final long durationMs; + private final boolean cancelAllowed; + private final String lastError; + + public ScmDbSnapshotStatusResponse(ScmDbSnapshotSyncStatus status, + ScmDbSnapshotSyncPhase phase, long startedAt, long finishedAt, + boolean cancelAllowed, String lastError) { + this.status = status; + this.phase = phase; + this.startedAt = startedAt; + this.finishedAt = finishedAt; + long endTime = finishedAt > 0 ? finishedAt : System.currentTimeMillis(); + this.durationMs = startedAt > 0 ? endTime - startedAt : 0; + this.cancelAllowed = cancelAllowed; + this.lastError = lastError; + } + + public ScmDbSnapshotSyncStatus getStatus() { + return status; + } + + public ScmDbSnapshotSyncPhase getPhase() { + return phase; + } + + public long getStartedAt() { + return startedAt; + } + + public long getFinishedAt() { + return finishedAt; + } + + public long getDurationMs() { + return durationMs; + } + + public boolean isCancelAllowed() { + return cancelAllowed; + } + + public String getLastError() { + return lastError; + } + } + + public static final class ScmDbSnapshotTriggerResponse { + private final boolean accepted; + private final ScmDbSnapshotSyncStatus status; + private final String message; + + public ScmDbSnapshotTriggerResponse(boolean accepted, + ScmDbSnapshotSyncStatus status, String message) { + this.accepted = accepted; + this.status = status; + this.message = message; + } + + public boolean isAccepted() { + return accepted; + } + + public ScmDbSnapshotSyncStatus getStatus() { + return status; + } + + public String getMessage() { + return message; + } + } + + public static final class ScmDbSnapshotCancelResponse { + private final boolean cancelled; + private final ScmDbSnapshotSyncStatus status; + private final ScmDbSnapshotSyncPhase phase; + private final String message; + + public ScmDbSnapshotCancelResponse(boolean cancelled, + ScmDbSnapshotSyncStatus status, ScmDbSnapshotSyncPhase phase, + String message) { + this.cancelled = cancelled; + this.status = status; + this.phase = phase; + this.message = message; + } + + public boolean isCancelled() { + return cancelled; + } + + public ScmDbSnapshotSyncStatus getStatus() { + return status; + } + + public ScmDbSnapshotSyncPhase getPhase() { + return phase; + } + + public String getMessage() { + return message; + } + } // To Do :- Refactor the constructor in a separate JIRA @Inject @@ -249,6 +389,11 @@ public ReconStorageContainerManagerFacade(OzoneConfiguration conf, scmhaManager, sequenceIdGen, pendingOps); this.scmServiceProvider = scmServiceProvider; this.isSyncDataFromSCMRunning = new AtomicBoolean(); + this.scmSnapshotExecutor = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder() + .setNameFormat(threadNamePrefix + "-SCM-Snapshot-Trigger-%d") + .setDaemon(true) + .build()); this.containerCountBySizeDao = containerCountBySizeDao; NodeReportHandler nodeReportHandler = new NodeReportHandler(nodeManager); @@ -499,6 +644,7 @@ public void stop() { IOUtils.cleanupWithLogger(LOG, pipelineManager); LOG.info("Flushing container replica history to DB."); containerManager.flushReplicaHistoryMapToDB(true); + scmSnapshotExecutor.shutdownNow(); IOUtils.close(LOG, dbStore); } @@ -550,23 +696,162 @@ private void initializeSCMDB() { public void updateReconSCMDBWithNewSnapshot() throws IOException { if (isSyncDataFromSCMRunning.compareAndSet(false, true)) { - DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot(); - if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) { - LOG.info("Got new checkpoint from SCM : " + - dbSnapshot.getCheckpointLocation()); - try { - initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile()); - } catch (IOException e) { - LOG.error("Unable to refresh Recon SCM DB Snapshot. ", e); - } - } else { - LOG.error("Null snapshot location got from SCM."); + try { + updateReconSCMDBWithNewSnapshotWithoutGuard(); + } finally { + isSyncDataFromSCMRunning.compareAndSet(true, false); } } else { LOG.warn("SCM DB sync is already running."); } } + private void updateReconSCMDBWithNewSnapshotWithoutGuard() + throws IOException { + DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot(); + if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) { + LOG.info("Got new checkpoint from SCM : {}", + dbSnapshot.getCheckpointLocation()); + initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile()); + } else { + throw new IOException("Null snapshot location got from SCM."); + } + } + + public ScmDbSnapshotTriggerResponse triggerScmDbSnapshotSync() { + synchronized (scmSnapshotLock) { + if (!isSyncDataFromSCMRunning.compareAndSet(false, true)) { + return new ScmDbSnapshotTriggerResponse(false, scmSnapshotStatus, + "SCM DB sync is already running."); + } + scmSnapshotStatus = ScmDbSnapshotSyncStatus.IN_PROGRESS; + scmSnapshotPhase = ScmDbSnapshotSyncPhase.DOWNLOADING_CHECKPOINT; + scmSnapshotStartedAt = System.currentTimeMillis(); + scmSnapshotFinishedAt = 0; + scmSnapshotCancelAllowed = true; + scmSnapshotTaskStarted = false; + scmSnapshotLastError = null; + scmSnapshotFuture = scmSnapshotExecutor.submit(this::runScmSnapshotSync); + return new ScmDbSnapshotTriggerResponse(true, scmSnapshotStatus, + "SCM DB snapshot sync started."); + } + } + + public ScmDbSnapshotStatusResponse getScmDbSnapshotSyncStatus() { + synchronized (scmSnapshotLock) { + return new ScmDbSnapshotStatusResponse(scmSnapshotStatus, + scmSnapshotPhase, scmSnapshotStartedAt, scmSnapshotFinishedAt, + scmSnapshotCancelAllowed, scmSnapshotLastError); + } + } + + public ScmDbSnapshotCancelResponse cancelScmDbSnapshotSync() { + synchronized (scmSnapshotLock) { + if (scmSnapshotStatus != ScmDbSnapshotSyncStatus.IN_PROGRESS) { + return new ScmDbSnapshotCancelResponse(false, scmSnapshotStatus, + scmSnapshotPhase, "No SCM DB snapshot sync is running."); + } + if (!scmSnapshotCancelAllowed) { + return new ScmDbSnapshotCancelResponse(false, scmSnapshotStatus, + scmSnapshotPhase, + "Cancellation is not allowed after DB initialization has started."); + } + boolean cancelled = scmSnapshotFuture != null && + scmSnapshotFuture.cancel(true); + if (cancelled) { + scmSnapshotStatus = ScmDbSnapshotSyncStatus.CANCELLED; + scmSnapshotPhase = ScmDbSnapshotSyncPhase.CANCELLED; + scmSnapshotFinishedAt = System.currentTimeMillis(); + scmSnapshotCancelAllowed = false; + if (!scmSnapshotTaskStarted) { + isSyncDataFromSCMRunning.compareAndSet(true, false); + } + } + return new ScmDbSnapshotCancelResponse(cancelled, scmSnapshotStatus, + scmSnapshotPhase, cancelled ? "SCM DB snapshot sync cancelled." : + "Unable to cancel SCM DB snapshot sync."); + } + } + + private void runScmSnapshotSync() { + File checkpointLocation = null; + boolean initialized = false; + try { + synchronized (scmSnapshotLock) { + scmSnapshotTaskStarted = true; + } + DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot(); + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("SCM DB snapshot sync interrupted."); + } + if (dbSnapshot == null || dbSnapshot.getCheckpointLocation() == null) { + throw new IOException("Null snapshot location got from SCM."); + } + checkpointLocation = dbSnapshot.getCheckpointLocation().toFile(); + synchronized (scmSnapshotLock) { + if (scmSnapshotStatus == ScmDbSnapshotSyncStatus.CANCELLED) { + return; + } + scmSnapshotPhase = ScmDbSnapshotSyncPhase.INITIALIZING_DB; + scmSnapshotCancelAllowed = false; + } + initializeNewRdbStore(checkpointLocation); + initialized = true; + synchronized (scmSnapshotLock) { + scmSnapshotStatus = ScmDbSnapshotSyncStatus.SUCCESS; + scmSnapshotPhase = ScmDbSnapshotSyncPhase.COMPLETED; + scmSnapshotFinishedAt = System.currentTimeMillis(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + markScmSnapshotCancelled(); + } catch (Throwable t) { + LOG.error("Unable to refresh Recon SCM DB Snapshot.", t); + synchronized (scmSnapshotLock) { + if (scmSnapshotStatus != ScmDbSnapshotSyncStatus.CANCELLED) { + scmSnapshotStatus = ScmDbSnapshotSyncStatus.FAILED; + scmSnapshotPhase = ScmDbSnapshotSyncPhase.FAILED; + scmSnapshotLastError = t.getMessage(); + scmSnapshotFinishedAt = System.currentTimeMillis(); + } + } + } finally { + cleanupFailedOrCancelledCheckpoint(checkpointLocation, initialized); + synchronized (scmSnapshotLock) { + scmSnapshotCancelAllowed = false; + } + isSyncDataFromSCMRunning.compareAndSet(true, false); + } + } + + private void markScmSnapshotCancelled() { + synchronized (scmSnapshotLock) { + scmSnapshotStatus = ScmDbSnapshotSyncStatus.CANCELLED; + scmSnapshotPhase = ScmDbSnapshotSyncPhase.CANCELLED; + scmSnapshotFinishedAt = System.currentTimeMillis(); + scmSnapshotCancelAllowed = false; + } + } + + private void cleanupFailedOrCancelledCheckpoint(File checkpointLocation, + boolean initialized) { + if (checkpointLocation == null || initialized) { + return; + } + synchronized (scmSnapshotLock) { + if (scmSnapshotStatus != ScmDbSnapshotSyncStatus.FAILED && + scmSnapshotStatus != ScmDbSnapshotSyncStatus.CANCELLED) { + return; + } + } + try { + FileUtils.deleteDirectory(checkpointLocation); + } catch (IOException e) { + LOG.warn("Unable to clean up SCM DB snapshot checkpoint directory {}.", + checkpointLocation, e); + } + } + public boolean syncWithSCMContainerInfo() { if (isSyncDataFromSCMRunning.compareAndSet(false, true)) { return containerSyncHelper.syncWithSCMContainerInfo(); @@ -620,6 +905,7 @@ private void initializeNewRdbStore(File dbFile) throws IOException { dbFile.getAbsolutePath()); } catch (IOException ioEx) { LOG.error("Unable to initialize Recon SCM DB snapshot store.", ioEx); + throw ioEx; } } diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java index da7edc620f32..937cd21d7fec 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java @@ -24,6 +24,8 @@ import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.OZONE_RECON_OM_SNAPSHOT_DB_DIR; import static org.apache.hadoop.ozone.recon.ReconUtils.createTarFile; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyString; @@ -51,6 +53,11 @@ import org.apache.hadoop.ozone.recon.ReconUtils; import org.apache.hadoop.ozone.recon.common.ReconTestUtils; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotCancelResponse; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotStatusResponse; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotSyncPhase; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotSyncStatus; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotTriggerResponse; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; @@ -133,7 +140,7 @@ reconUtilsMock, ozoneManagerProtocol, new ReconContext(configuration, reconUtils .addBinding(StorageContainerServiceProvider.class, mock(StorageContainerServiceProviderImpl.class)) .addBinding(OzoneStorageContainerManager.class, - ReconStorageContainerManagerFacade.class) + mock(ReconStorageContainerManagerFacade.class)) .withContainerDB() .addBinding(NodeEndpoint.class) .addBinding(MetricsServiceProviderFactory.class) @@ -151,4 +158,95 @@ public void testTriggerDBSyncEndpointWithOM() { assertEquals(200, response.getStatus()); assertEquals(true, response.getEntity()); } + + @Test + public void testTriggerSCMDBSnapshotSyncAccepted() { + OzoneManagerServiceProviderImpl omProvider = + mock(OzoneManagerServiceProviderImpl.class); + ReconStorageContainerManagerFacade reconScm = + mock(ReconStorageContainerManagerFacade.class); + when(reconScm.triggerScmDbSnapshotSync()).thenReturn( + new ScmDbSnapshotTriggerResponse(true, + ScmDbSnapshotSyncStatus.IN_PROGRESS, + "SCM DB snapshot sync started.")); + + TriggerDBSyncEndpoint endpoint = + new TriggerDBSyncEndpoint(omProvider, reconScm); + Response response = endpoint.triggerSCMDBSnapshotSync(); + + assertEquals(202, response.getStatus()); + ScmDbSnapshotTriggerResponse entity = + (ScmDbSnapshotTriggerResponse) response.getEntity(); + assertTrue(entity.isAccepted()); + assertEquals(ScmDbSnapshotSyncStatus.IN_PROGRESS, entity.getStatus()); + } + + @Test + public void testTriggerSCMDBSnapshotSyncConflict() { + OzoneManagerServiceProviderImpl omProvider = + mock(OzoneManagerServiceProviderImpl.class); + ReconStorageContainerManagerFacade reconScm = + mock(ReconStorageContainerManagerFacade.class); + when(reconScm.triggerScmDbSnapshotSync()).thenReturn( + new ScmDbSnapshotTriggerResponse(false, + ScmDbSnapshotSyncStatus.IN_PROGRESS, + "SCM DB sync is already running.")); + + TriggerDBSyncEndpoint endpoint = + new TriggerDBSyncEndpoint(omProvider, reconScm); + Response response = endpoint.triggerSCMDBSnapshotSync(); + + assertEquals(409, response.getStatus()); + ScmDbSnapshotTriggerResponse entity = + (ScmDbSnapshotTriggerResponse) response.getEntity(); + assertFalse(entity.isAccepted()); + } + + @Test + public void testGetSCMDBSnapshotSyncStatus() { + OzoneManagerServiceProviderImpl omProvider = + mock(OzoneManagerServiceProviderImpl.class); + ReconStorageContainerManagerFacade reconScm = + mock(ReconStorageContainerManagerFacade.class); + when(reconScm.getScmDbSnapshotSyncStatus()).thenReturn( + new ScmDbSnapshotStatusResponse( + ScmDbSnapshotSyncStatus.IN_PROGRESS, + ScmDbSnapshotSyncPhase.DOWNLOADING_CHECKPOINT, + 1L, 0L, true, null)); + + TriggerDBSyncEndpoint endpoint = + new TriggerDBSyncEndpoint(omProvider, reconScm); + Response response = endpoint.getSCMDBSnapshotSyncStatus(); + + assertEquals(200, response.getStatus()); + ScmDbSnapshotStatusResponse entity = + (ScmDbSnapshotStatusResponse) response.getEntity(); + assertEquals(ScmDbSnapshotSyncStatus.IN_PROGRESS, entity.getStatus()); + assertEquals(ScmDbSnapshotSyncPhase.DOWNLOADING_CHECKPOINT, + entity.getPhase()); + assertTrue(entity.isCancelAllowed()); + } + + @Test + public void testCancelSCMDBSnapshotSync() { + OzoneManagerServiceProviderImpl omProvider = + mock(OzoneManagerServiceProviderImpl.class); + ReconStorageContainerManagerFacade reconScm = + mock(ReconStorageContainerManagerFacade.class); + when(reconScm.cancelScmDbSnapshotSync()).thenReturn( + new ScmDbSnapshotCancelResponse(true, + ScmDbSnapshotSyncStatus.CANCELLED, + ScmDbSnapshotSyncPhase.CANCELLED, + "SCM DB snapshot sync cancelled.")); + + TriggerDBSyncEndpoint endpoint = + new TriggerDBSyncEndpoint(omProvider, reconScm); + Response response = endpoint.cancelSCMDBSnapshotSync(); + + assertEquals(200, response.getStatus()); + ScmDbSnapshotCancelResponse entity = + (ScmDbSnapshotCancelResponse) response.getEntity(); + assertTrue(entity.isCancelled()); + assertEquals(ScmDbSnapshotSyncStatus.CANCELLED, entity.getStatus()); + } } From a364126e195adbce4dc2985a4682f981780bfe9d Mon Sep 17 00:00:00 2001 From: Devesh Kumar Singh Date: Mon, 4 May 2026 22:49:52 +0530 Subject: [PATCH 2/2] HDDS-15165. Fixed chckstyle issues. --- .../scm/ReconStorageContainerManagerFacade.java | 15 +++++++++++++++ .../recon/api/TestTriggerDBSyncEndpoint.java | 2 +- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java index 93d1fb29d7fc..c25ab5587e99 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java @@ -183,6 +183,9 @@ public class ReconStorageContainerManagerFacade private boolean scmSnapshotTaskStarted; private String scmSnapshotLastError; + /** + * Status values for an explicitly triggered SCM DB snapshot sync. + */ public enum ScmDbSnapshotSyncStatus { IDLE, IN_PROGRESS, @@ -191,6 +194,9 @@ public enum ScmDbSnapshotSyncStatus { CANCELLED } + /** + * Phase values for an explicitly triggered SCM DB snapshot sync. + */ public enum ScmDbSnapshotSyncPhase { NONE, DOWNLOADING_CHECKPOINT, @@ -201,6 +207,9 @@ public enum ScmDbSnapshotSyncPhase { CANCELLED } + /** + * Response payload for the SCM DB snapshot sync status endpoint. + */ public static final class ScmDbSnapshotStatusResponse { private final ScmDbSnapshotSyncStatus status; private final ScmDbSnapshotSyncPhase phase; @@ -252,6 +261,9 @@ public String getLastError() { } } + /** + * Response payload for the SCM DB snapshot sync trigger endpoint. + */ public static final class ScmDbSnapshotTriggerResponse { private final boolean accepted; private final ScmDbSnapshotSyncStatus status; @@ -277,6 +289,9 @@ public String getMessage() { } } + /** + * Response payload for the SCM DB snapshot sync cancellation endpoint. + */ public static final class ScmDbSnapshotCancelResponse { private final boolean cancelled; private final ScmDbSnapshotSyncStatus status; diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java index 937cd21d7fec..6e2ad040ce16 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java @@ -53,12 +53,12 @@ import org.apache.hadoop.ozone.recon.ReconUtils; import org.apache.hadoop.ozone.recon.common.ReconTestUtils; import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager; +import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotCancelResponse; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotStatusResponse; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotSyncPhase; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotSyncStatus; import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade.ScmDbSnapshotTriggerResponse; -import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade; import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider; import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl; import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl;