You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/09/29 11:18:47 UTC

[GitHub] [ignite] Sega76 opened a new pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Sega76 opened a new pull request #9444:
URL: https://github.com/apache/ignite/pull/9444


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r731024417



##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
##########
@@ -3024,6 +3026,90 @@ public void testClusterSnapshotCreate() throws Exception {
         assertTrue("Snapshot must contains cache data [left=" + range + ']', range.isEmpty());
     }
 
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterSnapshotStatus() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+
+        IgniteEx ig = startGrid(0);
+        startGrid(1);
+
+        ig.cluster().state(ACTIVE);
+
+        int size = ig.cluster().nodes().size();
+
+        createCacheAndPreload(ig, keysCnt);
+
+        CommandHandler h = new CommandHandler();
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
+
+        assertEquals(size, countMatches(h.getLastOperationResult().toString(), "No snapshot operation."));
+
+        TestRecordingCommunicationSpi spi0 = TestRecordingCommunicationSpi.spi(grid(0));
+        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(grid(1));
+
+        spi0.blockMessages((node, msg) -> msg instanceof SingleNodeMessage &&
+            ((SingleNodeMessage<?>)msg).type() == END_SNAPSHOT.ordinal());
+
+        spi1.blockMessages((node, msg) -> msg instanceof SingleNodeMessage &&
+            ((SingleNodeMessage<?>)msg).type() == END_SNAPSHOT.ordinal());
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "create", snpName));
+
+        assertTrue(waitForCondition(() -> {
+            assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
+
+            return size == countMatches(h.getLastOperationResult().toString(), "Creating the snapshot with name: " + snpName);
+        }, 10_000));

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r745422413



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,23 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    protected @Nullable String getCreatingSnapshotName() {

Review comment:
       Thanks, fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r731025037



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1018,6 +1043,19 @@ public void cancelLocalSnapshotTask(String name) {
         return restoreCacheGrpProc.cancel(new IgniteCheckedException("Operation has been canceled by the user."), name);
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Map<Object, String>> statusSnapshot() {

Review comment:
       Renamed to "snapshotStatus"

##########
File path: modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
##########
@@ -157,6 +157,9 @@ This utility can do the following commands:
     Parameters:
       snapshot_name  - Snapshot name.
 
+  Status snapshot:
+    control.(sh|bat) --snapshot status
+

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] xtern commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
xtern commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r729092617



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1018,6 +1043,19 @@ public void cancelLocalSnapshotTask(String name) {
         return restoreCacheGrpProc.cancel(new IgniteCheckedException("Operation has been canceled by the user."), name);
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Map<Object, String>> statusSnapshot() {
+        cctx.kernalContext().security().authorize(ADMIN_SNAPSHOT);
+
+        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
+            (node) -> CU.baselineNode(node, cctx.kernalContext().state().clusterState()));
+
+        cctx.kernalContext().task().setThreadContext(TC_SKIP_AUTH, true);
+        cctx.kernalContext().task().setThreadContext(TC_SUBGRID, bltNodes);
+
+        return new IgniteFutureImpl<>(cctx.kernalContext().task().execute(new SnapshotStatusTask(), null));

Review comment:
       Let's try to do the following:
   change signature of executeRestoreManagementTask to
   ```
       /**
        * @param taskCls Snapshot operation management task class.
        * @param arg Task argument.
        */
       private <K, V> IgniteFuture<V> executeSnapshotManagementTask(
           Class<? extends ComputeTask<K, V>> taskCls,
           K arg
       ) {
   ```
   
   and replace this code with
   ```
   return executeSnapshotManagementTask(SnapshotStatusTask.class, null);
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r731025539



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,24 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    public @Nullable String getCreatingSnapshotName() {

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] xtern commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
xtern commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r736581438



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,23 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    protected @Nullable String getCreatingSnapshotName() {

Review comment:
       How about combining `getCreatingSnapshotName` and `getRestoringSnapshotName` into a single `status()` method, and move the relevant code from the job to the `status()`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1732,12 +1766,12 @@ static void copy(FileIOFactory factory, File from, File to, long length) {
     }
 
     /**
-     * @param taskCls Snapshot restore operation management task class.
-     * @param snpName Snapshot name.
+     * @param taskCls Snapshot operation management task class.
+     * @param arg Task argument.
      */
-    private IgniteFuture<Boolean> executeRestoreManagementTask(
-        Class<? extends ComputeTask<String, Boolean>> taskCls,
-        String snpName
+    private <K, V> IgniteFuture<V> executeSnapshotManagementTask(

Review comment:
       K, V is for key-value, use T, S instead
   https://docs.oracle.com/javase/tutorial/java/generics/types.html




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r731025715



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -884,6 +902,13 @@ public boolean isRestoring() {
         return restoreCacheGrpProc.restoringSnapshotName() != null;
     }
 
+    /**
+     * @return Snapshot name if the snapshot restore operation is in progress or {@code null}.
+     */
+    public @Nullable String getRestoringSnapshotName() {

Review comment:
       done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] xtern commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
xtern commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r736600642



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,23 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    protected @Nullable String getCreatingSnapshotName() {

Review comment:
       We already have `snapshotStatus`, so we should use relative naming for local operations.
   Mb `localSnapshotStatus()` for local operation.
   Or
   `status()` - cluster-wide operation
   `localStatus()` - local operation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] xtern commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
xtern commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r736600642



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,23 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    protected @Nullable String getCreatingSnapshotName() {

Review comment:
       We already have `snapshotStatus`, so we should use relative naming for local operations.
   Here is my suggestion:
   `clusterStatus()`  for cluster-wide status (instead of `snapshotStatus`).
   `localStatus()` for local operation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] xtern commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
xtern commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r736600642



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,23 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    protected @Nullable String getCreatingSnapshotName() {

Review comment:
       We already have `snapshotStatus`, so we should use relative naming for local operations.
   Mb `localSnapshotStatus()` for local operation.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r745422172



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1732,12 +1766,12 @@ static void copy(FileIOFactory factory, File from, File to, long length) {
     }
 
     /**
-     * @param taskCls Snapshot restore operation management task class.
-     * @param snpName Snapshot name.
+     * @param taskCls Snapshot operation management task class.
+     * @param arg Task argument.
      */
-    private IgniteFuture<Boolean> executeRestoreManagementTask(
-        Class<? extends ComputeTask<String, Boolean>> taskCls,
-        String snpName
+    private <K, V> IgniteFuture<V> executeSnapshotManagementTask(

Review comment:
       Thanks, fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r732706622



##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
##########
@@ -3024,6 +3026,90 @@ public void testClusterSnapshotCreate() throws Exception {
         assertTrue("Snapshot must contains cache data [left=" + range + ']', range.isEmpty());
     }
 
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterSnapshotStatus() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+
+        IgniteEx ig = startGrid(0);
+        startGrid(1);

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r731023978



##########
File path: modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
##########
@@ -71,4 +72,12 @@
      * future will be {@code false} if the restore process with the specified snapshot name is not running at all.
      */
     public IgniteFuture<Boolean> cancelSnapshotRestore(String name);
+
+    /**
+     * Status snapshot operation.
+     * Checks if running snapshot operations exist on nodes.
+     *
+     * @return Future which Map contains Consistent ID's and description snapshot operation.
+     */
+    public IgniteFuture<Map<Object, String>> statusSnapshot();

Review comment:
       Removed from public API.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Sega76 commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
Sega76 commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r731024190



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,24 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    public @Nullable String getCreatingSnapshotName() {
+        if (clusterSnpReq != null)
+            return clusterSnpReq.snapshotName();
+
+        synchronized (snpOpMux) {
+            if (clusterSnpReq != null)
+                return clusterSnpReq.snapshotName();
+
+            if (clusterSnpFut != null)
+                return clusterSnpFut.name;
+
+            return null;
+        }

Review comment:
       fixed

##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
##########
@@ -3024,6 +3026,90 @@ public void testClusterSnapshotCreate() throws Exception {
         assertTrue("Snapshot must contains cache data [left=" + range + ']', range.isEmpty());
     }
 
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterSnapshotStatus() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+
+        IgniteEx ig = startGrid(0);
+        startGrid(1);
+
+        ig.cluster().state(ACTIVE);
+
+        int size = ig.cluster().nodes().size();

Review comment:
       fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] xtern commented on a change in pull request #9444: IGNITE-13510: Getting status of snapshot execution via command line and jmx.

Posted by GitBox <gi...@apache.org>.
xtern commented on a change in pull request #9444:
URL: https://github.com/apache/ignite/pull/9444#discussion_r729043428



##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
##########
@@ -3024,6 +3026,90 @@ public void testClusterSnapshotCreate() throws Exception {
         assertTrue("Snapshot must contains cache data [left=" + range + ']', range.isEmpty());
     }
 
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterSnapshotStatus() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+
+        IgniteEx ig = startGrid(0);
+        startGrid(1);

Review comment:
       ```suggestion
           IgniteEx ig = startGrids(2);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -884,6 +902,13 @@ public boolean isRestoring() {
         return restoreCacheGrpProc.restoringSnapshotName() != null;
     }
 
+    /**
+     * @return Snapshot name if the snapshot restore operation is in progress or {@code null}.
+     */
+    public @Nullable String getRestoringSnapshotName() {

Review comment:
       The visibility of the method can be reduced (at least to a `protected`).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,24 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    public @Nullable String getCreatingSnapshotName() {

Review comment:
       The visibility of the method can be reduced (at least to a `protected`).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1018,6 +1043,19 @@ public void cancelLocalSnapshotTask(String name) {
         return restoreCacheGrpProc.cancel(new IgniteCheckedException("Operation has been canceled by the user."), name);
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Map<Object, String>> statusSnapshot() {
+        cctx.kernalContext().security().authorize(ADMIN_SNAPSHOT);
+
+        Collection<ClusterNode> bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE),
+            (node) -> CU.baselineNode(node, cctx.kernalContext().state().clusterState()));
+
+        cctx.kernalContext().task().setThreadContext(TC_SKIP_AUTH, true);
+        cctx.kernalContext().task().setThreadContext(TC_SUBGRID, bltNodes);
+
+        return new IgniteFutureImpl<>(cctx.kernalContext().task().execute(new SnapshotStatusTask(), null));

Review comment:
       Let's try to do the following:
   change signature of executeRestoreManagementTask to
   ```
       /**
        * @param taskCls Snapshot operation management task class.
        * @param arg Task argument.
        */
       private <K, V> IgniteFuture<V> executeSnapshotManagementTask(
           Class<? extends ComputeTask<K, V>> taskCls,
           K arg
       ) {
   ```
   
   and replace this code with
   ```
   return executeSnapshotManagementTask(SnapshotStatusTask.class, null);
   ```
   
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
##########
@@ -71,4 +72,12 @@
      * future will be {@code false} if the restore process with the specified snapshot name is not running at all.
      */
     public IgniteFuture<Boolean> cancelSnapshotRestore(String name);
+
+    /**
+     * Status snapshot operation.
+     * Checks if running snapshot operations exist on nodes.
+     *
+     * @return Future which Map contains Consistent ID's and description snapshot operation.
+     */
+    public IgniteFuture<Map<Object, String>> statusSnapshot();

Review comment:
       I don't think that the end-user may find something useful from the current implementation.
   We must either not make a public Java API, or carefully redesign the output format. 

##########
File path: modules/core/src/test/resources/org.apache.ignite.util/GridCommandHandlerClusterByClassTest_help.output
##########
@@ -157,6 +157,9 @@ This utility can do the following commands:
     Parameters:
       snapshot_name  - Snapshot name.
 
+  Status snapshot:
+    control.(sh|bat) --snapshot status
+

Review comment:
       We should not forget to deprecate the old restore status - unless we plan to support both of them.
   
   So I propose to add a warning about the deprecation of the old restore command, see example in ActivateCommand.java (printUsage and execute).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -1018,6 +1043,19 @@ public void cancelLocalSnapshotTask(String name) {
         return restoreCacheGrpProc.cancel(new IgniteCheckedException("Operation has been canceled by the user."), name);
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteFuture<Map<Object, String>> statusSnapshot() {

Review comment:
       What do you think about renaming in all the places "statusSnapshot" to "snapshotStatus"?
   I understand that we have cancelSnapshot, restoreSnapshot, etc, but it seems like it sounds much better.

##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
##########
@@ -3024,6 +3026,90 @@ public void testClusterSnapshotCreate() throws Exception {
         assertTrue("Snapshot must contains cache data [left=" + range + ']', range.isEmpty());
     }
 
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterSnapshotStatus() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+
+        IgniteEx ig = startGrid(0);
+        startGrid(1);
+
+        ig.cluster().state(ACTIVE);
+
+        int size = ig.cluster().nodes().size();
+
+        createCacheAndPreload(ig, keysCnt);
+
+        CommandHandler h = new CommandHandler();
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
+
+        assertEquals(size, countMatches(h.getLastOperationResult().toString(), "No snapshot operation."));
+
+        TestRecordingCommunicationSpi spi0 = TestRecordingCommunicationSpi.spi(grid(0));
+        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(grid(1));
+
+        spi0.blockMessages((node, msg) -> msg instanceof SingleNodeMessage &&
+            ((SingleNodeMessage<?>)msg).type() == END_SNAPSHOT.ordinal());
+
+        spi1.blockMessages((node, msg) -> msg instanceof SingleNodeMessage &&
+            ((SingleNodeMessage<?>)msg).type() == END_SNAPSHOT.ordinal());
+
+        assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "create", snpName));
+
+        assertTrue(waitForCondition(() -> {
+            assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
+
+            return size == countMatches(h.getLastOperationResult().toString(), "Creating the snapshot with name: " + snpName);
+        }, 10_000));

Review comment:
       As I see it, we can only block one node and wait for the blocking on spi instead of constantly executing the status command. Something like that: 
   ```
   TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1));
   spi.blockMessages...
   
   assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "create", snpName));
   
   spi.waitForBlocked();
   
   assertEquals(EXIT_CODE_OK, execute(h, "--snapshot", "status"));
   assertEquals(size, countMatches(h.getLastOperationResult().toString(), "Creating the snapshot with name: " + snpName));
   assertTrue(h.getLastOperationResult().toString().contains(ig.context().discovery().localNode().consistentId().toString())););
   ...
   ```

##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerTest.java
##########
@@ -3024,6 +3026,90 @@ public void testClusterSnapshotCreate() throws Exception {
         assertTrue("Snapshot must contains cache data [left=" + range + ']', range.isEmpty());
     }
 
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterSnapshotStatus() throws Exception {
+        int keysCnt = 100;
+        String snpName = "snapshot_02052020";
+
+        IgniteEx ig = startGrid(0);
+        startGrid(1);
+
+        ig.cluster().state(ACTIVE);
+
+        int size = ig.cluster().nodes().size();

Review comment:
       Let's introduce
   ```
   int nodesCnt = 2;
   
   IgniteEx ig = startGrids(nodesCnt)
   ...
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java
##########
@@ -875,6 +875,24 @@ public boolean isSnapshotCreating() {
         }
     }
 
+    /**
+     * @return Snapshot name if the snapshot create operation is in progress or {@code null}.
+     */
+    public @Nullable String getCreatingSnapshotName() {
+        if (clusterSnpReq != null)
+            return clusterSnpReq.snapshotName();
+
+        synchronized (snpOpMux) {
+            if (clusterSnpReq != null)
+                return clusterSnpReq.snapshotName();
+
+            if (clusterSnpFut != null)
+                return clusterSnpFut.name;
+
+            return null;
+        }

Review comment:
       Can we simplify it to something like the following code?
   ```
   SnapshotOperationRequest req = clusterSnpReq;
   
   if (req != null)
       return req.snapshotName();
       
   ClusterSnapshotFuture fut = clusterSnpFut;
   
   if (fut != null)
      return fut.name;
   ```
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org