You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2017/07/04 12:30:41 UTC

ignite git commit: ignite-2.1 Added persistent store metrics.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.1 a5abdd579 -> 6c9d2228d


ignite-2.1 Added persistent store metrics.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/6c9d2228
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/6c9d2228
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/6c9d2228

Branch: refs/heads/ignite-2.1
Commit: 6c9d2228d2aba6ba5be64bd20ef244e634aeaa5a
Parents: a5abdd5
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Jul 4 19:30:46 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Jul 4 19:30:46 2017 +0700

----------------------------------------------------------------------
 .../visor/node/VisorNodeDataCollectorJob.java   |  32 ++-
 .../node/VisorNodeDataCollectorJobResult.java   |  73 +++++--
 .../visor/node/VisorNodeDataCollectorTask.java  |  14 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |  28 ++-
 .../visor/node/VisorPersistenceMetrics.java     | 214 +++++++++++++++++++
 .../visor/service/VisorServiceTask.java         |  10 +-
 .../frontend/app/filters/duration.filter.js     |   2 +-
 7 files changed, 341 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index f5ea567..3fd7b0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.visor.cache.VisorMemoryMetrics;
 import org.apache.ignite.internal.visor.compute.VisorComputeMonitoringHolder;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
 import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
+import org.apache.ignite.internal.visor.util.VisorExceptionWrapper;
 import org.apache.ignite.lang.IgniteProductVersion;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isIgfsCache;
@@ -119,7 +120,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             events0(res, arg.getEventsOrderKey(), arg.getEventsThrottleCounterKey(), arg.isTaskMonitoringEnabled());
         }
         catch (Exception e) {
-            res.setEventsEx(e);
+            res.setEventsEx(new VisorExceptionWrapper(e));
         }
     }
 
@@ -160,9 +161,9 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                 memoryMetrics.add(new VisorMemoryMetrics(m));
         }
         catch (Exception e) {
-            res.setMemoryMetricsEx(e);
+            res.setMemoryMetricsEx(new VisorExceptionWrapper(e));
         }
-}
+    }
 
     /**
      * Collect caches.
@@ -205,7 +206,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             }
         }
         catch (Exception e) {
-            res.setCachesEx(e);
+            res.setCachesEx(new VisorExceptionWrapper(e));
         }
     }
 
@@ -245,7 +246,21 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             }
         }
         catch (Exception e) {
-            res.setIgfssEx(e);
+            res.setIgfssEx(new VisorExceptionWrapper(e));
+        }
+    }
+
+    /**
+     * Collect persistence metrics.
+     *
+     * @param res Job result.
+     */
+    protected void persistenceMetrics(VisorNodeDataCollectorJobResult res) {
+        try {
+            res.setPersistenceMetrics(new VisorPersistenceMetrics(ignite.persistentStoreMetrics()));
+        }
+        catch (Exception e) {
+            res.setPersistenceMetricsEx(new VisorExceptionWrapper(e));
         }
     }
 
@@ -292,7 +307,12 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         igfs(res);
 
         if (debug)
-            log(ignite.log(), "Collected igfs", getClass(), start0);
+            start0 = log(ignite.log(), "Collected igfs", getClass(), start0);
+
+        persistenceMetrics(res);
+
+        if (debug)
+            log(ignite.log(), "Collected persistence metrics", getClass(), start0);
 
         res.setErrorCount(ignite.context().exceptionRegistry().errorCount());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
index ce4f9fc..90ecf6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.visor.cache.VisorMemoryMetrics;
 import org.apache.ignite.internal.visor.event.VisorGridEvent;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
 import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
+import org.apache.ignite.internal.visor.util.VisorExceptionWrapper;
 
 /**
  * Data collector job result.
@@ -51,19 +52,19 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     private List<VisorGridEvent> evts = new ArrayList<>();
 
     /** Exception while collecting node events. */
-    private Throwable evtsEx;
+    private VisorExceptionWrapper evtsEx;
 
     /** Node memory metrics. */
     private List<VisorMemoryMetrics> memoryMetrics = new ArrayList<>();
 
     /** Exception while collecting memory metrics. */
-    private Throwable memoryMetricsEx;
+    private VisorExceptionWrapper memoryMetricsEx;
 
     /** Node caches. */
     private List<VisorCache> caches = new ArrayList<>();
 
     /** Exception while collecting node caches. */
-    private Throwable cachesEx;
+    private VisorExceptionWrapper cachesEx;
 
     /** Node IGFSs. */
     private List<VisorIgfs> igfss = new ArrayList<>();
@@ -72,7 +73,7 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     private List<VisorIgfsEndpoint> igfsEndpoints = new ArrayList<>();
 
     /** Exception while collecting node IGFSs. */
-    private Throwable igfssEx;
+    private VisorExceptionWrapper igfssEx;
 
     /** Errors count. */
     private long errCnt;
@@ -83,6 +84,12 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /** Whether pending exchange future exists. */
     private boolean hasPendingExchange;
 
+    /** Persistence metrics. */
+    private VisorPersistenceMetrics persistenceMetrics;
+
+    /** Exception while collecting persistence metrics. */
+    private VisorExceptionWrapper persistenceMetricsEx;
+
     /**
      * Default constructor.
      */
@@ -142,14 +149,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting events.
      */
-    public Throwable getEventsEx() {
+    public VisorExceptionWrapper getEventsEx() {
         return evtsEx;
     }
 
     /**
      * @param evtsEx Exception caught during collecting events.
      */
-    public void setEventsEx(Throwable evtsEx) {
+    public void setEventsEx(VisorExceptionWrapper evtsEx) {
         this.evtsEx = evtsEx;
     }
 
@@ -163,14 +170,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting memory metrics.
      */
-    public Throwable getMemoryMetricsEx() {
+    public VisorExceptionWrapper getMemoryMetricsEx() {
         return memoryMetricsEx;
     }
 
     /**
      * @param memoryMetricsEx Exception caught during collecting memory metrics.
      */
-    public void setMemoryMetricsEx(Throwable memoryMetricsEx) {
+    public void setMemoryMetricsEx(VisorExceptionWrapper memoryMetricsEx) {
         this.memoryMetricsEx = memoryMetricsEx;
     }
 
@@ -184,14 +191,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting caches metrics.
      */
-    public Throwable getCachesEx() {
+    public VisorExceptionWrapper getCachesEx() {
         return cachesEx;
     }
 
     /**
      * @param cachesEx Exception caught during collecting caches metrics.
      */
-    public void setCachesEx(Throwable cachesEx) {
+    public void setCachesEx(VisorExceptionWrapper cachesEx) {
         this.cachesEx = cachesEx;
     }
 
@@ -212,14 +219,14 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /**
      * @return Exception caught during collecting IGFSs metrics.
      */
-    public Throwable getIgfssEx() {
+    public VisorExceptionWrapper getIgfssEx() {
         return igfssEx;
     }
 
     /**
      * @param igfssEx Exception caught during collecting IGFSs metrics.
      */
-    public void setIgfssEx(Throwable igfssEx) {
+    public void setIgfssEx(VisorExceptionWrapper igfssEx) {
         this.igfssEx = igfssEx;
     }
 
@@ -265,6 +272,36 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         this.hasPendingExchange = hasPendingExchange;
     }
 
+    /**
+     * Get persistence metrics.
+     */
+    public VisorPersistenceMetrics getPersistenceMetrics() {
+        return persistenceMetrics;
+    }
+
+    /**
+     * Set persistence metrics.
+     *
+     * @param persistenceMetrics Persistence metrics.
+     */
+    public void setPersistenceMetrics(VisorPersistenceMetrics persistenceMetrics) {
+        this.persistenceMetrics = persistenceMetrics;
+    }
+
+    /**
+     * @return Exception caught during collecting persistence metrics.
+     */
+    public VisorExceptionWrapper getPersistenceMetricsEx() {
+        return persistenceMetricsEx;
+    }
+
+    /**
+     * @param persistenceMetricsEx Exception caught during collecting persistence metrics.
+     */
+    public void setPersistenceMetricsEx(VisorExceptionWrapper persistenceMetricsEx) {
+        this.persistenceMetricsEx = persistenceMetricsEx;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, gridName);
@@ -282,6 +319,8 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         out.writeLong(errCnt);
         out.writeObject(readyTopVer);
         out.writeBoolean(hasPendingExchange);
+        out.writeObject(persistenceMetrics);
+        out.writeObject(persistenceMetricsEx);
     }
 
     /** {@inheritDoc} */
@@ -290,17 +329,19 @@ public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
         topVer = in.readLong();
         taskMonitoringEnabled = in.readBoolean();
         evts = U.readList(in);
-        evtsEx = (Throwable)in.readObject();
+        evtsEx = (VisorExceptionWrapper)in.readObject();
         memoryMetrics = U.readList(in);
-        memoryMetricsEx = (Throwable)in.readObject();
+        memoryMetricsEx = (VisorExceptionWrapper)in.readObject();
         caches = U.readList(in);
-        cachesEx = (Throwable)in.readObject();
+        cachesEx = (VisorExceptionWrapper)in.readObject();
         igfss = U.readList(in);
         igfsEndpoints = U.readList(in);
-        igfssEx = (Throwable)in.readObject();
+        igfssEx = (VisorExceptionWrapper)in.readObject();
         errCnt = in.readLong();
         readyTopVer = (VisorAffinityTopologyVersion)in.readObject();
         hasPendingExchange = in.readBoolean();
+        persistenceMetrics = (VisorPersistenceMetrics)in.readObject();
+        persistenceMetricsEx = (VisorExceptionWrapper)in.readObject();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index 56b3718..80664a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -97,19 +97,19 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.getEvents().addAll(jobRes.getEvents());
 
         if (jobRes.getEventsEx() != null)
-            taskRes.getEventsEx().put(nid, new VisorExceptionWrapper(jobRes.getEventsEx()));
+            taskRes.getEventsEx().put(nid, jobRes.getEventsEx());
 
         if (!jobRes.getMemoryMetrics().isEmpty())
             taskRes.getMemoryMetrics().put(nid, jobRes.getMemoryMetrics());
 
         if (jobRes.getMemoryMetricsEx() != null)
-            taskRes.getMemoryMetricsEx().put(nid, new VisorExceptionWrapper(jobRes.getMemoryMetricsEx()));
+            taskRes.getMemoryMetricsEx().put(nid, jobRes.getMemoryMetricsEx());
 
         if (!jobRes.getCaches().isEmpty())
             taskRes.getCaches().put(nid, jobRes.getCaches());
 
         if (jobRes.getCachesEx() != null)
-            taskRes.getCachesEx().put(nid, new VisorExceptionWrapper(jobRes.getCachesEx()));
+            taskRes.getCachesEx().put(nid, jobRes.getCachesEx());
 
         if (!jobRes.getIgfss().isEmpty())
             taskRes.getIgfss().put(nid, jobRes.getIgfss());
@@ -118,7 +118,13 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.getIgfsEndpoints().put(nid, jobRes.getIgfsEndpoints());
 
         if (jobRes.getIgfssEx() != null)
-            taskRes.getIgfssEx().put(nid, new VisorExceptionWrapper(jobRes.getIgfssEx()));
+            taskRes.getIgfssEx().put(nid, jobRes.getIgfssEx());
+
+        if (jobRes.getPersistenceMetrics() != null)
+            taskRes.getPersistenceMetrics().put(nid, jobRes.getPersistenceMetrics());
+
+        if (jobRes.getPersistenceMetricsEx() != null)
+            taskRes.getPersistenceMetricsEx().put(nid, jobRes.getPersistenceMetricsEx());
 
         taskRes.getReadyAffinityVersions().put(nid, jobRes.getReadyAffinityVersion());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
index cef3a29..093e867 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
@@ -94,6 +94,12 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
     /** Whether pending exchange future exists from nodes. */
     private Map<UUID, Boolean> pendingExchanges = new HashMap<>();
 
+    /** All persistence metrics collected from nodes. */
+    private Map<UUID, VisorPersistenceMetrics> persistenceMetrics = new HashMap<>();
+
+    /** Exceptions caught during collecting persistence metrics from nodes. */
+    private Map<UUID, VisorExceptionWrapper> persistenceMetricsEx = new HashMap<>();
+
     /**
      * Default constructor.
      */
@@ -120,7 +126,9 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
             igfsEndpoints.isEmpty() &&
             igfssEx.isEmpty() &&
             readyTopVers.isEmpty() &&
-            pendingExchanges.isEmpty();
+            pendingExchanges.isEmpty() &&
+            persistenceMetrics.isEmpty() &&
+            persistenceMetricsEx.isEmpty();
     }
 
     /**
@@ -249,6 +257,20 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         return pendingExchanges;
     }
 
+    /**
+     * All persistence metrics collected from nodes.
+     */
+    public Map<UUID, VisorPersistenceMetrics> getPersistenceMetrics() {
+        return persistenceMetrics;
+    }
+
+    /**
+     * @return Exceptions caught during collecting persistence metrics from nodes.
+     */
+    public Map<UUID, VisorExceptionWrapper> getPersistenceMetricsEx() {
+        return persistenceMetricsEx;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         out.writeBoolean(active);
@@ -268,6 +290,8 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         U.writeMap(out, igfssEx);
         U.writeMap(out, readyTopVers);
         U.writeMap(out, pendingExchanges);
+        U.writeMap(out, persistenceMetrics);
+        U.writeMap(out, persistenceMetricsEx);
     }
 
     /** {@inheritDoc} */
@@ -289,6 +313,8 @@ public class VisorNodeDataCollectorTaskResult extends VisorDataTransferObject {
         igfssEx = U.readMap(in);
         readyTopVers = U.readMap(in);
         pendingExchanges = U.readMap(in);
+        persistenceMetrics = U.readMap(in);
+        persistenceMetricsEx = U.readMap(in);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
new file mode 100644
index 0000000..c838161
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistenceMetrics.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.visor.node;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.PersistenceMetrics;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * DTO object for {@link PersistenceMetrics}.
+ */
+public class VisorPersistenceMetrics extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private float walLoggingRate;
+
+    /** */
+    private float walWritingRate;
+
+    /** */
+    private int walArchiveSegments;
+
+    /** */
+    private float walFsyncTimeAvg;
+
+    /** */
+    private long lastCpDuration;
+
+    /** */
+    private long lastCpLockWaitDuration;
+
+    /** */
+    private long lastCpMmarkDuration;
+
+    /** */
+    private long lastCpPagesWriteDuration;
+
+    /** */
+    private long lastCpFsyncDuration;
+
+    /** */
+    private long lastCpTotalPages;
+
+    /** */
+    private long lastCpDataPages;
+
+    /** */
+    private long lastCpCowPages;
+
+    /**
+     * Default constructor.
+     */
+    public VisorPersistenceMetrics() {
+        // No-op.
+    }
+
+    /**
+     * @param metrics Persistence metrics.
+     */
+    public VisorPersistenceMetrics(PersistenceMetrics metrics) {
+        walLoggingRate = metrics.getWalLoggingRate();
+        walWritingRate = metrics.getWalWritingRate();
+        walArchiveSegments = metrics.getWalArchiveSegments();
+        walFsyncTimeAvg = metrics.getWalFsyncTimeAverage();
+        lastCpDuration = metrics.getLastCheckpointingDuration();
+        lastCpLockWaitDuration = metrics.getLastCheckpointLockWaitDuration();
+        lastCpMmarkDuration = metrics.getLastCheckpointMarkDuration();
+        lastCpPagesWriteDuration = metrics.getLastCheckpointPagesWriteDuration();
+        lastCpFsyncDuration = metrics.getLastCheckpointFsyncDuration();
+        lastCpTotalPages = metrics.getLastCheckpointTotalPagesNumber();
+        lastCpDataPages = metrics.getLastCheckpointDataPagesNumber();
+        lastCpCowPages = metrics.getLastCheckpointCopiedOnWritePagesNumber();
+    }
+
+    /**
+     * @return Average number of WAL records per second written during the last time interval.
+     */
+    public float getWalLoggingRate() {
+        return walLoggingRate;
+    }
+
+    /**
+     * @return Average number of bytes per second written during the last time interval.
+     */
+    public float getWalWritingRate(){
+        return walWritingRate;
+    }
+
+    /**
+     * @return Current number of WAL segments in the WAL archive.
+     */
+    public int getWalArchiveSegments(){
+        return walArchiveSegments;
+    }
+
+    /**
+     * @return Average WAL fsync duration in microseconds over the last time interval.
+     */
+    public float getWalFsyncTimeAverage(){
+        return walFsyncTimeAvg;
+    }
+
+    /**
+     * @return Total checkpoint duration in milliseconds.
+     */
+    public long getLastCheckpointingDuration(){
+        return lastCpDuration;
+    }
+
+    /**
+     * @return Checkpoint lock wait time in milliseconds.
+     */
+    public long getLastCheckpointLockWaitDuration(){
+        return lastCpLockWaitDuration;
+    }
+
+    /**
+     * @return Checkpoint mark duration in milliseconds.
+     */
+    public long getLastCheckpointMarkDuration(){
+        return lastCpMmarkDuration;
+    }
+
+    /**
+     * @return Checkpoint pages write phase in milliseconds.
+     */
+    public long getLastCheckpointPagesWriteDuration(){
+        return lastCpPagesWriteDuration;
+    }
+
+    /**
+     * @return Checkpoint fsync time in milliseconds.
+     */
+    public long getLastCheckpointFsyncDuration(){
+        return lastCpFsyncDuration;
+    }
+
+    /**
+     * @return Total number of pages written during the last checkpoint.
+     */
+    public long getLastCheckpointTotalPagesNumber(){
+        return lastCpTotalPages;
+    }
+
+    /**
+     * @return Total number of data pages written during the last checkpoint.
+     */
+    public long getLastCheckpointDataPagesNumber(){
+        return lastCpDataPages;
+    }
+
+    /**
+     * @return Total number of pages copied to a temporary checkpoint buffer during the last checkpoint.
+     */
+    public long getLastCheckpointCopiedOnWritePagesNumber(){
+        return lastCpCowPages;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeFloat(walLoggingRate);
+        out.writeFloat(walWritingRate);
+        out.writeInt(walArchiveSegments);
+        out.writeFloat(walFsyncTimeAvg);
+        out.writeLong(lastCpDuration);
+        out.writeLong(lastCpLockWaitDuration);
+        out.writeLong(lastCpMmarkDuration);
+        out.writeLong(lastCpPagesWriteDuration);
+        out.writeLong(lastCpFsyncDuration);
+        out.writeLong(lastCpTotalPages);
+        out.writeLong(lastCpDataPages);
+        out.writeLong(lastCpCowPages);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        walLoggingRate = in.readFloat();
+        walWritingRate = in.readFloat();
+        walArchiveSegments = in.readInt();
+        walFsyncTimeAvg = in.readFloat();
+        lastCpDuration = in.readLong();
+        lastCpLockWaitDuration = in.readLong();
+        lastCpMmarkDuration = in.readLong();
+        lastCpPagesWriteDuration = in.readLong();
+        lastCpFsyncDuration = in.readLong();
+        lastCpTotalPages = in.readLong();
+        lastCpDataPages = in.readLong();
+        lastCpCowPages = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorPersistenceMetrics.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
index 1b3495c..f2489bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
@@ -57,12 +57,14 @@ public class VisorServiceTask extends VisorOneNodeTask<Void, Collection<VisorSer
 
         /** {@inheritDoc} */
         @Override protected Collection<VisorServiceDescriptor> run(final Void arg) {
-            Collection<ServiceDescriptor> services = ignite.services().serviceDescriptors();
+            Collection<VisorServiceDescriptor> res = new ArrayList<>();
 
-            Collection<VisorServiceDescriptor> res = new ArrayList<>(services.size());
+            if (ignite.active()) {
+                Collection<ServiceDescriptor> services = ignite.services().serviceDescriptors();
 
-            for (ServiceDescriptor srvc: services)
-                res.add(new VisorServiceDescriptor(srvc));
+                for (ServiceDescriptor srvc : services)
+                    res.add(new VisorServiceDescriptor(srvc));
+            }
 
             return res;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6c9d2228/modules/web-console/frontend/app/filters/duration.filter.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/filters/duration.filter.js b/modules/web-console/frontend/app/filters/duration.filter.js
index 770bca3..46d50d8 100644
--- a/modules/web-console/frontend/app/filters/duration.filter.js
+++ b/modules/web-console/frontend/app/filters/duration.filter.js
@@ -34,7 +34,7 @@ export default [() => {
         const h = Math.floor((t - d * cd) / ch);
         const m = Math.floor((t - d * cd - h * ch) / cm);
         const s = Math.floor((t - d * cd - h * ch - m * cm) / cs);
-        const ms = t % 1000;
+        const ms = Math.round(t % 1000);
 
         return a(d, 'd') + a(h, 'h') + a(m, 'm') + a(s, 's') + (t < cm ? ms + 'ms' : '');
     };