You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by al...@apache.org on 2021/06/30 14:46:01 UTC

[ignite] branch master updated: IGNITE-15011 Move native SQL system views to GridSystemViewManager - Fixes #9209.

This is an automated email from the ASF dual-hosted git repository.

alexpl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new a1e6ff5  IGNITE-15011 Move native SQL system views to GridSystemViewManager - Fixes #9209.
a1e6ff5 is described below

commit a1e6ff5fb0400763a561b9bb4d10d3ed2b8c2616
Author: Aleksey Plekhanov <pl...@gmail.com>
AuthorDate: Wed Jun 30 17:43:03 2021 +0300

    IGNITE-15011 Move native SQL system views to GridSystemViewManager - Fixes #9209.
    
    Signed-off-by: Aleksey Plekhanov <pl...@gmail.com>
---
 .../SystemViewRowAttributeWalkerGenerator.java     |   8 +
 .../managers/discovery/GridDiscoveryManager.java   |  98 +++-
 .../systemview/walker/BaselineNodeViewWalker.java  |  46 ++
 .../systemview/walker/CacheGroupIoViewWalker.java  |  50 ++
 .../systemview/walker/NodeAttributeViewWalker.java |  68 +++
 .../systemview/walker/NodeMetricsViewWalker.java   | 158 ++++++
 .../processors/cache/GridCacheProcessor.java       |  24 +
 .../cluster/GridClusterStateProcessor.java         |  45 +-
 .../spi/systemview/view/BaselineNodeView.java      |  58 +++
 .../spi/systemview/view/CacheGroupIoView.java      |  82 +++
 .../spi/systemview/view/NodeAttributeView.java     |  75 +++
 .../spi/systemview/view/NodeMetricsView.java       | 556 +++++++++++++++++++++
 .../ignite/internal/metric/SystemViewSelfTest.java | 131 +++++
 .../processors/query/h2/SchemaManager.java         |  31 --
 .../h2/sys/view/SqlSystemViewBaselineNodes.java    |  88 ----
 .../view/SqlSystemViewCacheGroupsIOStatistics.java | 124 -----
 .../h2/sys/view/SqlSystemViewNodeAttributes.java   | 108 ----
 .../h2/sys/view/SqlSystemViewNodeMetrics.java      | 211 --------
 18 files changed, 1395 insertions(+), 566 deletions(-)

diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java
index d48f1c8..894f452 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java
@@ -40,7 +40,9 @@ import org.apache.ignite.internal.processors.query.stat.view.StatisticsColumnLoc
 import org.apache.ignite.internal.processors.query.stat.view.StatisticsColumnPartitionDataView;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.systemview.view.BaselineNodeAttributeView;
+import org.apache.ignite.spi.systemview.view.BaselineNodeView;
 import org.apache.ignite.spi.systemview.view.BinaryMetadataView;
+import org.apache.ignite.spi.systemview.view.CacheGroupIoView;
 import org.apache.ignite.spi.systemview.view.CacheGroupView;
 import org.apache.ignite.spi.systemview.view.CachePagesListView;
 import org.apache.ignite.spi.systemview.view.CacheView;
@@ -50,6 +52,8 @@ import org.apache.ignite.spi.systemview.view.ComputeJobView;
 import org.apache.ignite.spi.systemview.view.ComputeTaskView;
 import org.apache.ignite.spi.systemview.view.ContinuousQueryView;
 import org.apache.ignite.spi.systemview.view.MetastorageView;
+import org.apache.ignite.spi.systemview.view.NodeAttributeView;
+import org.apache.ignite.spi.systemview.view.NodeMetricsView;
 import org.apache.ignite.spi.systemview.view.PagesListView;
 import org.apache.ignite.spi.systemview.view.PartitionStateView;
 import org.apache.ignite.spi.systemview.view.ScanQueryView;
@@ -131,6 +135,10 @@ public class SystemViewRowAttributeWalkerGenerator {
         gen.generateAndWrite(ReentrantLockView.class, DFLT_SRC_DIR);
         gen.generateAndWrite(SemaphoreView.class, DFLT_SRC_DIR);
         gen.generateAndWrite(BaselineNodeAttributeView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(BaselineNodeView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(NodeAttributeView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(NodeMetricsView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(CacheGroupIoView.class, DFLT_SRC_DIR);
 
         gen.generateAndWrite(SqlSchemaView.class, INDEXING_SRC_DIR);
         gen.generateAndWrite(SqlTableView.class, INDEXING_SRC_DIR);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 38a0c3b..da91b54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -37,6 +37,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
@@ -74,6 +75,8 @@ import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.managers.systemview.walker.ClusterNodeViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.NodeAttributeViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.NodeMetricsViewWalker;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessage;
@@ -133,6 +136,8 @@ import org.apache.ignite.spi.discovery.IgniteDiscoveryThread;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.systemview.view.ClusterNodeView;
+import org.apache.ignite.spi.systemview.view.NodeAttributeView;
+import org.apache.ignite.spi.systemview.view.NodeMetricsView;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.thread.OomExceptionHandler;
 import org.jetbrains.annotations.NotNull;
@@ -187,6 +192,18 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** */
     public static final String NODES_SYS_VIEW_DESC = "Cluster nodes";
 
+    /** */
+    public static final String NODE_ATTRIBUTES_SYS_VIEW = metricName("node", "attributes");
+
+    /** */
+    public static final String NODE_ATTRIBUTES_SYS_VIEW_DESC = "Node attributes";
+
+    /** */
+    public static final String NODE_METRICS_SYS_VIEW = metricName("node", "metrics");
+
+    /** */
+    public static final String NODE_METRICS_SYS_VIEW_DESC = "Node metrics";
+
     /** @see IgniteSystemProperties#IGNITE_DISCOVERY_HISTORY_SIZE */
     public static final int DFLT_DISCOVERY_HISTORY_SIZE = 500;
 
@@ -299,10 +316,29 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         super(ctx, ctx.config().getDiscoverySpi());
 
         if (ctx.systemView().view(NODES_SYS_VIEW) == null) {
-            ctx.systemView().registerView(NODES_SYS_VIEW, NODES_SYS_VIEW_DESC,
+            ctx.systemView().registerView(
+                NODES_SYS_VIEW,
+                NODES_SYS_VIEW_DESC,
                 new ClusterNodeViewWalker(),
                 () -> F.concat(false, allNodes(), daemonNodes()),
-                ClusterNodeView::new);
+                ClusterNodeView::new
+            );
+
+            ctx.systemView().registerView(
+                NODE_METRICS_SYS_VIEW,
+                NODE_METRICS_SYS_VIEW_DESC,
+                new NodeMetricsViewWalker(),
+                () -> F.concat(false, allNodes(), daemonNodes()),
+                NodeMetricsView::new
+            );
+
+            ctx.systemView().registerFiltrableView(
+                NODE_ATTRIBUTES_SYS_VIEW,
+                NODE_ATTRIBUTES_SYS_VIEW_DESC,
+                new NodeAttributeViewWalker(),
+                this::nodeAttributeViewSupplier,
+                Function.identity()
+            );
         }
     }
 
@@ -3540,4 +3576,62 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             discoCache.minimumNodeVersion(),
             discoCache.minimumServerNodeVersion());
     }
+
+    /**
+     * Node attributes view supplier.
+     *
+     * @param filter Filter.
+     */
+    private Iterable<NodeAttributeView> nodeAttributeViewSupplier(Map<String, Object> filter) {
+        Object nodeFilter = filter.get(NodeAttributeViewWalker.NODE_ID_FILTER);
+
+        UUID nodeId = null;
+
+        if (nodeFilter instanceof UUID)
+            nodeId = (UUID)nodeFilter;
+        else if (nodeFilter instanceof String) {
+            try {
+                nodeId = UUID.fromString((String)nodeFilter);
+            }
+            catch (RuntimeException ignored) {
+                return Collections.emptyList();
+            }
+        }
+        else if (nodeFilter != null)
+            return Collections.emptyList();
+
+        Collection<ClusterNode> nodes;
+
+        if (nodeId != null) {
+            ClusterNode node = ctx.discovery().node(nodeId);
+
+            if (node != null)
+                nodes = Collections.singleton(node);
+            else
+                nodes = Collections.emptySet();
+        }
+        else
+            nodes = F.concat(false, allNodes(), daemonNodes());
+
+        String attrName = (String)filter.get(NodeAttributeViewWalker.NAME_FILTER);
+
+        return F.flat(F.iterator(nodes, node -> {
+            Map<String, Object> attrs = node.attributes();
+
+            if (attrName != null) {
+                Object attrVal = attrs.get(attrName);
+
+                if (attrVal == null)
+                    return Collections.emptyList();
+
+                attrs = F.asMap(attrName, attrs.get(attrName));
+            }
+
+            return F.iterator(
+                attrs.entrySet(),
+                na -> new NodeAttributeView(node.id(), na.getKey(), na.getValue()),
+                true
+            );
+        }, true));
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/BaselineNodeViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/BaselineNodeViewWalker.java
new file mode 100644
index 0000000..63bf053
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/BaselineNodeViewWalker.java
@@ -0,0 +1,46 @@
+/*
+ * 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.managers.systemview.walker;
+
+import org.apache.ignite.spi.systemview.view.BaselineNodeView;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link BaselineNodeView} attributes walker.
+ * 
+ * @see BaselineNodeView
+ */
+public class BaselineNodeViewWalker implements SystemViewRowAttributeWalker<BaselineNodeView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "consistentId", String.class);
+        v.accept(1, "online", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(BaselineNodeView row, AttributeWithValueVisitor v) {
+        v.accept(0, "consistentId", String.class, row.consistentId());
+        v.acceptBoolean(1, "online", row.online());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 2;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/CacheGroupIoViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/CacheGroupIoViewWalker.java
new file mode 100644
index 0000000..40173c7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/CacheGroupIoViewWalker.java
@@ -0,0 +1,50 @@
+/*
+ * 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.managers.systemview.walker;
+
+import org.apache.ignite.spi.systemview.view.CacheGroupIoView;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link CacheGroupIoView} attributes walker.
+ * 
+ * @see CacheGroupIoView
+ */
+public class CacheGroupIoViewWalker implements SystemViewRowAttributeWalker<CacheGroupIoView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "cacheGroupId", int.class);
+        v.accept(1, "cacheGroupName", String.class);
+        v.accept(2, "physicalReads", long.class);
+        v.accept(3, "logicalReads", long.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(CacheGroupIoView row, AttributeWithValueVisitor v) {
+        v.acceptInt(0, "cacheGroupId", row.cacheGroupId());
+        v.accept(1, "cacheGroupName", String.class, row.cacheGroupName());
+        v.acceptLong(2, "physicalReads", row.physicalReads());
+        v.acceptLong(3, "logicalReads", row.logicalReads());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 4;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/NodeAttributeViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/NodeAttributeViewWalker.java
new file mode 100644
index 0000000..9df2e06
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/NodeAttributeViewWalker.java
@@ -0,0 +1,68 @@
+/*
+ * 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.managers.systemview.walker;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.systemview.view.NodeAttributeView;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link NodeAttributeView} attributes walker.
+ * 
+ * @see NodeAttributeView
+ */
+public class NodeAttributeViewWalker implements SystemViewRowAttributeWalker<NodeAttributeView> {
+    /** Filter key for attribute "nodeId" */
+    public static final String NODE_ID_FILTER = "nodeId";
+
+    /** Filter key for attribute "name" */
+    public static final String NAME_FILTER = "name";
+
+    /** List of filtrable attributes. */
+    private static final List<String> FILTRABLE_ATTRS = Collections.unmodifiableList(F.asList(
+        "nodeId", "name"
+    ));
+
+    /** {@inheritDoc} */
+    @Override public List<String> filtrableAttributes() {
+        return FILTRABLE_ATTRS;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "nodeId", UUID.class);
+        v.accept(1, "name", String.class);
+        v.accept(2, "value", String.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(NodeAttributeView row, AttributeWithValueVisitor v) {
+        v.accept(0, "nodeId", UUID.class, row.nodeId());
+        v.accept(1, "name", String.class, row.name());
+        v.accept(2, "value", String.class, row.value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 3;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/NodeMetricsViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/NodeMetricsViewWalker.java
new file mode 100644
index 0000000..f1bcc8b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/NodeMetricsViewWalker.java
@@ -0,0 +1,158 @@
+/*
+ * 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.managers.systemview.walker;
+
+import java.util.Date;
+import java.util.UUID;
+import org.apache.ignite.spi.systemview.view.NodeMetricsView;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link NodeMetricsView} attributes walker.
+ * 
+ * @see NodeMetricsView
+ */
+public class NodeMetricsViewWalker implements SystemViewRowAttributeWalker<NodeMetricsView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "nodeId", UUID.class);
+        v.accept(1, "lastUpdateTime", Date.class);
+        v.accept(2, "maxActiveJobs", int.class);
+        v.accept(3, "curActiveJobs", int.class);
+        v.accept(4, "avgActiveJobs", float.class);
+        v.accept(5, "maxWaitingJobs", int.class);
+        v.accept(6, "curWaitingJobs", int.class);
+        v.accept(7, "avgWaitingJobs", float.class);
+        v.accept(8, "maxRejectedJobs", int.class);
+        v.accept(9, "curRejectedJobs", int.class);
+        v.accept(10, "avgRejectedJobs", float.class);
+        v.accept(11, "totalRejectedJobs", int.class);
+        v.accept(12, "maxCanceledJobs", int.class);
+        v.accept(13, "curCanceledJobs", int.class);
+        v.accept(14, "avgCanceledJobs", float.class);
+        v.accept(15, "totalCanceledJobs", int.class);
+        v.accept(16, "maxJobsWaitTime", long.class);
+        v.accept(17, "curJobsWaitTime", long.class);
+        v.accept(18, "avgJobsWaitTime", long.class);
+        v.accept(19, "maxJobsExecuteTime", long.class);
+        v.accept(20, "curJobsExecuteTime", long.class);
+        v.accept(21, "avgJobsExecuteTime", long.class);
+        v.accept(22, "totalJobsExecuteTime", long.class);
+        v.accept(23, "totalExecutedJobs", int.class);
+        v.accept(24, "totalExecutedTasks", int.class);
+        v.accept(25, "totalBusyTime", long.class);
+        v.accept(26, "totalIdleTime", long.class);
+        v.accept(27, "curIdleTime", long.class);
+        v.accept(28, "busyTimePercentage", float.class);
+        v.accept(29, "idleTimePercentage", float.class);
+        v.accept(30, "totalCpu", int.class);
+        v.accept(31, "curCpuLoad", double.class);
+        v.accept(32, "avgCpuLoad", double.class);
+        v.accept(33, "curGcCpuLoad", double.class);
+        v.accept(34, "heapMemoryInit", long.class);
+        v.accept(35, "heapMemoryUsed", long.class);
+        v.accept(36, "heapMemoryCommited", long.class);
+        v.accept(37, "heapMemoryMax", long.class);
+        v.accept(38, "heapMemoryTotal", long.class);
+        v.accept(39, "nonheapMemoryInit", long.class);
+        v.accept(40, "nonheapMemoryUsed", long.class);
+        v.accept(41, "nonheapMemoryCommited", long.class);
+        v.accept(42, "nonheapMemoryMax", long.class);
+        v.accept(43, "nonheapMemoryTotal", long.class);
+        v.accept(44, "uptime", long.class);
+        v.accept(45, "jvmStartTime", Date.class);
+        v.accept(46, "nodeStartTime", Date.class);
+        v.accept(47, "lastDataVersion", long.class);
+        v.accept(48, "curThreadCount", int.class);
+        v.accept(49, "maxThreadCount", int.class);
+        v.accept(50, "totalThreadCount", long.class);
+        v.accept(51, "curDaemonThreadCount", int.class);
+        v.accept(52, "sentMessagesCount", int.class);
+        v.accept(53, "sentBytesCount", long.class);
+        v.accept(54, "receivedMessagesCount", int.class);
+        v.accept(55, "receivedBytesCount", long.class);
+        v.accept(56, "outboundMessagesQueue", int.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(NodeMetricsView row, AttributeWithValueVisitor v) {
+        v.accept(0, "nodeId", UUID.class, row.nodeId());
+        v.accept(1, "lastUpdateTime", Date.class, row.lastUpdateTime());
+        v.acceptInt(2, "maxActiveJobs", row.maxActiveJobs());
+        v.acceptInt(3, "curActiveJobs", row.curActiveJobs());
+        v.acceptFloat(4, "avgActiveJobs", row.avgActiveJobs());
+        v.acceptInt(5, "maxWaitingJobs", row.maxWaitingJobs());
+        v.acceptInt(6, "curWaitingJobs", row.curWaitingJobs());
+        v.acceptFloat(7, "avgWaitingJobs", row.avgWaitingJobs());
+        v.acceptInt(8, "maxRejectedJobs", row.maxRejectedJobs());
+        v.acceptInt(9, "curRejectedJobs", row.curRejectedJobs());
+        v.acceptFloat(10, "avgRejectedJobs", row.avgRejectedJobs());
+        v.acceptInt(11, "totalRejectedJobs", row.totalRejectedJobs());
+        v.acceptInt(12, "maxCanceledJobs", row.maxCanceledJobs());
+        v.acceptInt(13, "curCanceledJobs", row.curCanceledJobs());
+        v.acceptFloat(14, "avgCanceledJobs", row.avgCanceledJobs());
+        v.acceptInt(15, "totalCanceledJobs", row.totalCanceledJobs());
+        v.acceptLong(16, "maxJobsWaitTime", row.maxJobsWaitTime());
+        v.acceptLong(17, "curJobsWaitTime", row.curJobsWaitTime());
+        v.acceptLong(18, "avgJobsWaitTime", row.avgJobsWaitTime());
+        v.acceptLong(19, "maxJobsExecuteTime", row.maxJobsExecuteTime());
+        v.acceptLong(20, "curJobsExecuteTime", row.curJobsExecuteTime());
+        v.acceptLong(21, "avgJobsExecuteTime", row.avgJobsExecuteTime());
+        v.acceptLong(22, "totalJobsExecuteTime", row.totalJobsExecuteTime());
+        v.acceptInt(23, "totalExecutedJobs", row.totalExecutedJobs());
+        v.acceptInt(24, "totalExecutedTasks", row.totalExecutedTasks());
+        v.acceptLong(25, "totalBusyTime", row.totalBusyTime());
+        v.acceptLong(26, "totalIdleTime", row.totalIdleTime());
+        v.acceptLong(27, "curIdleTime", row.curIdleTime());
+        v.acceptFloat(28, "busyTimePercentage", row.busyTimePercentage());
+        v.acceptFloat(29, "idleTimePercentage", row.idleTimePercentage());
+        v.acceptInt(30, "totalCpu", row.totalCpu());
+        v.acceptDouble(31, "curCpuLoad", row.curCpuLoad());
+        v.acceptDouble(32, "avgCpuLoad", row.avgCpuLoad());
+        v.acceptDouble(33, "curGcCpuLoad", row.curGcCpuLoad());
+        v.acceptLong(34, "heapMemoryInit", row.heapMemoryInit());
+        v.acceptLong(35, "heapMemoryUsed", row.heapMemoryUsed());
+        v.acceptLong(36, "heapMemoryCommited", row.heapMemoryCommited());
+        v.acceptLong(37, "heapMemoryMax", row.heapMemoryMax());
+        v.acceptLong(38, "heapMemoryTotal", row.heapMemoryTotal());
+        v.acceptLong(39, "nonheapMemoryInit", row.nonheapMemoryInit());
+        v.acceptLong(40, "nonheapMemoryUsed", row.nonheapMemoryUsed());
+        v.acceptLong(41, "nonheapMemoryCommited", row.nonheapMemoryCommited());
+        v.acceptLong(42, "nonheapMemoryMax", row.nonheapMemoryMax());
+        v.acceptLong(43, "nonheapMemoryTotal", row.nonheapMemoryTotal());
+        v.acceptLong(44, "uptime", row.uptime());
+        v.accept(45, "jvmStartTime", Date.class, row.jvmStartTime());
+        v.accept(46, "nodeStartTime", Date.class, row.nodeStartTime());
+        v.acceptLong(47, "lastDataVersion", row.lastDataVersion());
+        v.acceptInt(48, "curThreadCount", row.curThreadCount());
+        v.acceptInt(49, "maxThreadCount", row.maxThreadCount());
+        v.acceptLong(50, "totalThreadCount", row.totalThreadCount());
+        v.acceptInt(51, "curDaemonThreadCount", row.curDaemonThreadCount());
+        v.acceptInt(52, "sentMessagesCount", row.sentMessagesCount());
+        v.acceptLong(53, "sentBytesCount", row.sentBytesCount());
+        v.acceptInt(54, "receivedMessagesCount", row.receivedMessagesCount());
+        v.acceptLong(55, "receivedBytesCount", row.receivedBytesCount());
+        v.acceptInt(56, "outboundMessagesQueue", row.outboundMessagesQueue());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 57;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 5d129f0..611e3f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -83,8 +83,10 @@ import org.apache.ignite.internal.cluster.DetachedClusterNode;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
 import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
+import org.apache.ignite.internal.managers.systemview.walker.CacheGroupIoViewWalker;
 import org.apache.ignite.internal.managers.systemview.walker.CachePagesListViewWalker;
 import org.apache.ignite.internal.managers.systemview.walker.PartitionStateViewWalker;
+import org.apache.ignite.internal.metric.IoStatisticsType;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -141,6 +143,7 @@ import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMess
 import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage;
 import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheManager;
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.QuerySchema;
@@ -189,6 +192,7 @@ import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
+import org.apache.ignite.spi.systemview.view.CacheGroupIoView;
 import org.apache.ignite.spi.systemview.view.CachePagesListView;
 import org.apache.ignite.spi.systemview.view.PartitionStateView;
 import org.jetbrains.annotations.NotNull;
@@ -220,6 +224,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearE
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache;
 import static org.apache.ignite.internal.processors.cache.ValidationOnNodeJoinUtils.validateHashIdResolvers;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition.DFLT_CACHE_REMOVE_ENTRIES_TTL;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 import static org.apache.ignite.internal.util.IgniteUtils.doInParallel;
 
 /**
@@ -248,6 +253,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** System view description for partition states. */
     public static final String PART_STATES_VIEW_DESC = "Distribution of cache group partitions across cluster nodes";
 
+    /** System view name for cache group IO. */
+    public static final String CACHE_GRP_IO_VIEW = metricName("local", "cache", "groups", "io");
+
+    /** System view description for cache group IO. */
+    public static final String CACHE_GRP_IO_VIEW_DESC = "Local node IO statistics for cache groups";
+
     /** @see IgniteSystemProperties#IGNITE_ALLOW_START_CACHES_IN_PARALLEL */
     public static final boolean DFLT_ALLOW_START_CACHES_IN_PARALLEL = true;
 
@@ -643,6 +654,19 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             this::partStatesViewSupplier,
             Function.identity()
         );
+
+        ctx.systemView().registerView(
+            CACHE_GRP_IO_VIEW,
+            CACHE_GRP_IO_VIEW_DESC,
+            new CacheGroupIoViewWalker(),
+            () -> F.view(cacheGrps.values(), grp -> !grp.systemCache()),
+            grpCtx -> {
+                MetricRegistry mreg = ctx.metric().registry(metricName(IoStatisticsType.CACHE_GROUP.metricGroupName(),
+                    grpCtx.cacheOrGroupName()));
+
+                return new CacheGroupIoView(grpCtx, mreg);
+            }
+        );
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index f99555f..d21ca61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -59,6 +59,7 @@ import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.managers.systemview.walker.BaselineNodeAttributeViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.BaselineNodeViewWalker;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.ExchangeActions;
@@ -96,6 +97,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.systemview.view.BaselineNodeAttributeView;
+import org.apache.ignite.spi.systemview.view.BaselineNodeView;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cluster.ClusterState.ACTIVE;
@@ -129,6 +131,12 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
         "in-memory caches (without persistence) including the system caches.";
 
     /** */
+    public static final String BASELINE_NODES_SYS_VIEW = metricName("baseline", "nodes");
+
+    /** */
+    public static final String BASELINE_NODES_SYS_VIEW_DESC = "Baseline topology nodes";
+
+    /** */
     public static final String BASELINE_NODE_ATTRIBUTES_SYS_VIEW = metricName("baseline", "node", "attributes");
 
     /** */
@@ -225,6 +233,14 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
             EVT_BASELINE_AUTO_ADJUST_AWAITING_TIME_CHANGED
         ));
 
+        ctx.systemView().registerView(
+            BASELINE_NODES_SYS_VIEW,
+            BASELINE_NODES_SYS_VIEW_DESC,
+            new BaselineNodeViewWalker(),
+            this::nodeViewSupplier,
+            Function.identity()
+        );
+
         ctx.systemView().registerFiltrableView(
             BASELINE_NODE_ATTRIBUTES_SYS_VIEW,
             BASELINE_NODE_ATTRIBUTES_SYS_VIEW_DESC,
@@ -1858,6 +1874,29 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
     }
 
     /**
+     * Baseline topology nodes view supplier.
+     */
+    private Collection<BaselineNodeView> nodeViewSupplier() {
+        BaselineTopology blt = globalState.baselineTopology();
+
+        if (blt == null)
+            return Collections.emptyList();
+
+        Set<Object> consistentIds = blt.consistentIds();
+
+        List<BaselineNodeView> rows = new ArrayList<>(consistentIds.size());
+
+        Collection<ClusterNode> srvNodes = ctx.discovery().aliveServerNodes();
+
+        Set<Object> aliveNodeIds = new HashSet<>(F.nodeConsistentIds(srvNodes));
+
+        for (Object consistentId : consistentIds)
+            rows.add(new BaselineNodeView(consistentId, aliveNodeIds.contains(consistentId)));
+
+        return rows;
+    }
+
+    /**
      * Baseline node attributes view supplier.
      *
      * @param filter Filter.
@@ -1886,9 +1925,11 @@ public class GridClusterStateProcessor extends GridProcessorAdapter implements I
                 attrs = F.asMap(attrName, attrs.get(attrName));
             }
 
-            return F.iterator(attrs.entrySet(),
+            return F.iterator(
+                attrs.entrySet(),
                 na -> new BaselineNodeAttributeView(node.consistentId(), na.getKey(), na.getValue()),
-                true);
+                true
+            );
         }, true));
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/BaselineNodeView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/BaselineNodeView.java
new file mode 100644
index 0000000..d48c1d5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/BaselineNodeView.java
@@ -0,0 +1,58 @@
+/*
+ * 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.spi.systemview.view;
+
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+
+import static org.apache.ignite.internal.util.IgniteUtils.toStringSafe;
+
+/**
+ * Baseline topology node representation for a {@link SystemView}.
+ */
+public class BaselineNodeView {
+    /** Consistent id. */
+    private final Object consistentId;
+
+    /** Online flag. */
+    private final boolean online;
+
+    /**
+     * @param consistentId Node consistent id.
+     * @param online Is node online.
+     */
+    public BaselineNodeView(Object consistentId, boolean online) {
+        this.consistentId = consistentId;
+        this.online = online;
+    }
+
+    /**
+     * @return Node consistend id.
+     */
+    @Order
+    public String consistentId() {
+        return toStringSafe(consistentId);
+    }
+
+    /**
+     * @return Attribute name.
+     */
+    @Order(1)
+    public boolean online() {
+        return online;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/CacheGroupIoView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/CacheGroupIoView.java
new file mode 100644
index 0000000..88161f6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/CacheGroupIoView.java
@@ -0,0 +1,82 @@
+/*
+ * 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.spi.systemview.view;
+
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.spi.metric.LongMetric;
+
+import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.LOGICAL_READS;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.PHYSICAL_READS;
+
+/**
+ * Cache group IO statistics representation for a {@link SystemView}.
+ */
+public class CacheGroupIoView {
+    /** Cache group. */
+    private final CacheGroupContext grpCtx;
+
+    /** Metric registry for current cache group IO statistics. */
+    private final MetricRegistry mreg;
+
+    /**
+     * @param grpCtx Cache group context.
+     * @param mreg Metric registry for current cache group IO statistics.
+     */
+    public CacheGroupIoView(CacheGroupContext grpCtx, MetricRegistry mreg) {
+        this.grpCtx = grpCtx;
+        this.mreg = mreg;
+    }
+
+    /**
+     * @return Cache group id.
+     */
+    @Order
+    public int cacheGroupId() {
+        return grpCtx.groupId();
+    }
+
+    /**
+     * @return Cache group name.
+     */
+    @Order(1)
+    public String cacheGroupName() {
+        return grpCtx.cacheOrGroupName();
+    }
+
+    /**
+     * @return Physical reads.
+     */
+    @Order(2)
+    public long physicalReads() {
+        LongMetric metric = mreg.findMetric(PHYSICAL_READS);
+
+        return metric != null ? metric.value() : 0;
+    }
+
+    /**
+     * @return Logical reads.
+     */
+    @Order(3)
+    public long logicalReads() {
+        LongMetric metric = mreg.findMetric(LOGICAL_READS);
+
+        return metric != null ? metric.value() : 0;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/NodeAttributeView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/NodeAttributeView.java
new file mode 100644
index 0000000..8203000
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/NodeAttributeView.java
@@ -0,0 +1,75 @@
+/*
+ * 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.spi.systemview.view;
+
+import java.util.UUID;
+import org.apache.ignite.internal.managers.systemview.walker.Filtrable;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+
+import static org.apache.ignite.internal.util.IgniteUtils.toStringSafe;
+
+/**
+ * Node attribute representation for a {@link SystemView}.
+ */
+public class NodeAttributeView {
+    /** Node id. */
+    private final UUID nodeId;
+
+    /** Attribute name. */
+    private final String name;
+
+    /** Attribute value. */
+    private final Object val;
+
+    /**
+     * @param nodeId Node id.
+     * @param name Attribute name.
+     * @param val Attribute value.
+     */
+    public NodeAttributeView(UUID nodeId, String name, Object val) {
+        this.nodeId = nodeId;
+        this.name = name;
+        this.val = val;
+    }
+
+    /**
+     * @return Node id.
+     */
+    @Order
+    @Filtrable
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * @return Attribute name.
+     */
+    @Order(1)
+    @Filtrable
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return Attribute value.
+     */
+    @Order(2)
+    public String value() {
+        return toStringSafe(val);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/NodeMetricsView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/NodeMetricsView.java
new file mode 100644
index 0000000..ccfbbb0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/NodeMetricsView.java
@@ -0,0 +1,556 @@
+/*
+ * 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.spi.systemview.view;
+
+import java.util.Date;
+import java.util.UUID;
+import org.apache.ignite.cluster.ClusterMetrics;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+
+/**
+ * Node metrics representation for a {@link SystemView}.
+ */
+public class NodeMetricsView {
+    /** Cluster node. */
+    private final ClusterNode node;
+
+    /** Node metrics. */
+    private final ClusterMetrics metrics;
+
+    /**
+     * @param node Cluster node.
+     */
+    public NodeMetricsView(ClusterNode node) {
+        this.node = node;
+        metrics = node.metrics();
+    }
+
+    /**
+     * @return Node id.
+     * @see ClusterNode#id()
+     */
+    @Order
+    public UUID nodeId() {
+        return node.id();
+    }
+
+    /**
+     * @return Metrics last update time.
+     * @see ClusterMetrics#getLastUpdateTime()
+     */
+    @Order(1)
+    public Date lastUpdateTime() {
+        return new Date(metrics.getLastUpdateTime());
+    }
+
+    /**
+     * @return Maximum active jobs count.
+     * @see ClusterMetrics#getMaximumActiveJobs()
+     */
+    @Order(2)
+    public int maxActiveJobs() {
+        return metrics.getMaximumActiveJobs();
+    }
+
+    /**
+     * @return Current active jobs count.
+     * @see ClusterMetrics#getCurrentActiveJobs()
+     */
+    @Order(3)
+    public int curActiveJobs() {
+        return metrics.getCurrentActiveJobs();
+    }
+
+    /**
+     * @return Average active jobs count.
+     * @see ClusterMetrics#getAverageActiveJobs()
+     */
+    @Order(4)
+    public float avgActiveJobs() {
+        return metrics.getAverageActiveJobs();
+    }
+
+    /**
+     * @return Maximum waiting jobs count.
+     * @see ClusterMetrics#getMaximumWaitingJobs()
+     */
+    @Order(5)
+    public int maxWaitingJobs() {
+        return metrics.getMaximumWaitingJobs();
+    }
+
+    /**
+     * @return Current waiting jobs count.
+     * @see ClusterMetrics#getCurrentWaitingJobs()
+     */
+    @Order(6)
+    public int curWaitingJobs() {
+        return metrics.getCurrentWaitingJobs();
+    }
+
+    /**
+     * @return Average waiting jobs count.
+     * @see ClusterMetrics#getAverageWaitingJobs()
+     */
+    @Order(7)
+    public float avgWaitingJobs() {
+        return metrics.getAverageWaitingJobs();
+    }
+
+    /**
+     * @return Maximum number of jobs rejected at once.
+     * @see ClusterMetrics#getMaximumRejectedJobs()
+     */
+    @Order(8)
+    public int maxRejectedJobs() {
+        return metrics.getMaximumRejectedJobs();
+    }
+
+    /**
+     * @return Number of jobs rejected after more recent collision resolution operation.
+     * @see ClusterMetrics#getCurrentRejectedJobs()
+     */
+    @Order(9)
+    public int curRejectedJobs() {
+        return metrics.getCurrentRejectedJobs();
+    }
+
+    /**
+     * @return Average number of jobs this node rejects during collision resolution operations.
+     * @see ClusterMetrics#getAverageRejectedJobs()
+     */
+    @Order(10)
+    public float avgRejectedJobs() {
+        return metrics.getAverageRejectedJobs();
+    }
+
+    /**
+     * @return Total number of jobs this node rejects during collision resolution operations since node startup.
+     * @see ClusterMetrics#getTotalRejectedJobs()
+     */
+    @Order(11)
+    public int totalRejectedJobs() {
+        return metrics.getTotalRejectedJobs();
+    }
+
+    /**
+     * @return Maximum number of cancelled jobs.
+     * @see ClusterMetrics#getMaximumCancelledJobs()
+     */
+    @Order(12)
+    public int maxCanceledJobs() {
+        return metrics.getMaximumCancelledJobs();
+    }
+
+    /**
+     * @return Number of cancelled jobs that are still running.
+     * @see ClusterMetrics#getCurrentCancelledJobs()
+     */
+    @Order(13)
+    public int curCanceledJobs() {
+        return metrics.getCurrentCancelledJobs();
+    }
+
+    /**
+     * @return Average number of cancelled jobs.
+     * @see ClusterMetrics#getAverageCancelledJobs()
+     */
+    @Order(14)
+    public float avgCanceledJobs() {
+        return metrics.getAverageCancelledJobs();
+    }
+
+    /**
+     * @return Total number of cancelled jobs since node startup.
+     * @see ClusterMetrics#getTotalCancelledJobs()
+     */
+    @Order(15)
+    public int totalCanceledJobs() {
+        return metrics.getTotalCancelledJobs();
+    }
+
+    /**
+     * @return Maximum jobs wait time.
+     * @see ClusterMetrics#getMaximumJobWaitTime()
+     */
+    @Order(16)
+    public long maxJobsWaitTime() {
+        return metrics.getMaximumJobWaitTime();
+    }
+
+    /**
+     * @return Current wait time of oldest job.
+     * @see ClusterMetrics#getCurrentJobWaitTime()
+     */
+    @Order(17)
+    public long curJobsWaitTime() {
+        return metrics.getCurrentJobWaitTime();
+    }
+
+    /**
+     * @return Average jobs wait time.
+     * @see ClusterMetrics#getAverageJobWaitTime()
+     */
+    @Order(18)
+    public long avgJobsWaitTime() {
+        return (long)metrics.getAverageJobWaitTime();
+    }
+
+    /**
+     * @return Maximum jobs execute time.
+     * @see ClusterMetrics#getMaximumJobExecuteTime()
+     */
+    @Order(19)
+    public long maxJobsExecuteTime() {
+        return metrics.getMaximumJobExecuteTime();
+    }
+
+    /**
+     * @return Current jobs execute time.
+     * @see ClusterMetrics#getCurrentJobExecuteTime()
+     */
+    @Order(20)
+    public long curJobsExecuteTime() {
+        return metrics.getCurrentJobExecuteTime();
+    }
+
+    /**
+     * @return Average jobs execute time.
+     * @see ClusterMetrics#getAverageJobExecuteTime()
+     */
+    @Order(21)
+    public long avgJobsExecuteTime() {
+        return (long)metrics.getAverageJobExecuteTime();
+    }
+
+    /**
+     * @return Total jobs execute time.
+     * @see ClusterMetrics#getTotalJobsExecutionTime()
+     */
+    @Order(22)
+    public long totalJobsExecuteTime() {
+        return metrics.getTotalJobsExecutionTime();
+    }
+
+    /**
+     * @return Total executed jobs.
+     * @see ClusterMetrics#getTotalExecutedJobs()
+     */
+    @Order(23)
+    public int totalExecutedJobs() {
+        return metrics.getTotalExecutedJobs();
+    }
+
+    /**
+     * @return Total executed tasks.
+     * @see ClusterMetrics#getTotalExecutedTasks()
+     */
+    @Order(24)
+    public int totalExecutedTasks() {
+        return metrics.getTotalExecutedTasks();
+    }
+
+    /**
+     * @return Total busy time.
+     * @see ClusterMetrics#getTotalBusyTime()
+     */
+    @Order(25)
+    public long totalBusyTime() {
+        return metrics.getTotalBusyTime();
+    }
+
+    /**
+     * @return Total idle time.
+     * @see ClusterMetrics#getTotalIdleTime()
+     */
+    @Order(26)
+    public long totalIdleTime() {
+        return metrics.getTotalIdleTime();
+    }
+
+    /**
+     * @return Current idle time.
+     * @see ClusterMetrics#getCurrentIdleTime()
+     */
+    @Order(27)
+    public long curIdleTime() {
+        return metrics.getCurrentIdleTime();
+    }
+
+    /**
+     * @return Busy time percentage.
+     * @see ClusterMetrics#getBusyTimePercentage()
+     */
+    @Order(28)
+    public float busyTimePercentage() {
+        return metrics.getBusyTimePercentage();
+    }
+
+    /**
+     * @return Idle time percentage.
+     * @see ClusterMetrics#getIdleTimePercentage()
+     */
+    @Order(29)
+    public float idleTimePercentage() {
+        return metrics.getIdleTimePercentage();
+    }
+
+    /**
+     * @return The number of processors available to the virtual machine.
+     * @see ClusterMetrics#getTotalCpus()
+     */
+    @Order(30)
+    public int totalCpu() {
+        return metrics.getTotalCpus();
+    }
+
+    /**
+     * @return The estimated CPU usage in {@code [0, 1]} range.
+     * @see ClusterMetrics#getCurrentCpuLoad()
+     */
+    @Order(31)
+    public double curCpuLoad() {
+        return metrics.getCurrentCpuLoad();
+    }
+
+    /**
+     * @return Average of CPU load value in {@code [0, 1]} range.
+     * @see ClusterMetrics#getAverageCpuLoad()
+     */
+    @Order(32)
+    public double avgCpuLoad() {
+        return metrics.getAverageCpuLoad();
+    }
+
+    /**
+     * @return Average time spent in CG since the last update.
+     * @see ClusterMetrics#getCurrentGcCpuLoad()
+     */
+    @Order(33)
+    public double curGcCpuLoad() {
+        return metrics.getCurrentGcCpuLoad();
+    }
+
+    /**
+     * @return The initial size of memory in bytes; {@code -1} if undefined.
+     * @see ClusterMetrics#getHeapMemoryInitialized()
+     */
+    @Order(34)
+    public long heapMemoryInit() {
+        return metrics.getHeapMemoryInitialized();
+    }
+
+    /**
+     * @return Heap memory used.
+     * @see ClusterMetrics#getHeapMemoryUsed()
+     */
+    @Order(35)
+    public long heapMemoryUsed() {
+        return metrics.getHeapMemoryUsed();
+    }
+
+    /**
+     * @return Heap memory commited.
+     * @see ClusterMetrics#getHeapMemoryCommitted()
+     */
+    @Order(36)
+    public long heapMemoryCommited() {
+        return metrics.getHeapMemoryCommitted();
+    }
+
+    /**
+     * @return The maximum amount of memory in bytes; {@code -1} if undefined.
+     * @see ClusterMetrics#getHeapMemoryMaximum()
+     */
+    @Order(37)
+    public long heapMemoryMax() {
+        return metrics.getHeapMemoryMaximum();
+    }
+
+    /**
+     * @return Heap memory total.
+     * @see ClusterMetrics#getHeapMemoryTotal()
+     */
+    @Order(38)
+    public long heapMemoryTotal() {
+        return metrics.getHeapMemoryTotal();
+    }
+
+    /**
+     * @return The initial size of memory in bytes; {@code -1} if undefined.
+     * @see ClusterMetrics#getNonHeapMemoryInitialized()
+     */
+    @Order(39)
+    public long nonheapMemoryInit() {
+        return metrics.getNonHeapMemoryInitialized();
+    }
+
+    /**
+     * @return Nonheap memory used.
+     * @see ClusterMetrics#getNonHeapMemoryUsed()
+     */
+    @Order(40)
+    public long nonheapMemoryUsed() {
+        return metrics.getNonHeapMemoryUsed();
+    }
+
+    /**
+     * @return Nonheap memory commited.
+     * @see ClusterMetrics#getNonHeapMemoryCommitted()
+     */
+    @Order(41)
+    public long nonheapMemoryCommited() {
+        return metrics.getNonHeapMemoryCommitted();
+    }
+
+    /**
+     * @return The maximum amount of memory in bytes; {@code -1} if undefined.
+     * @see ClusterMetrics#getNonHeapMemoryMaximum()
+     */
+    @Order(42)
+    public long nonheapMemoryMax() {
+        return metrics.getNonHeapMemoryMaximum();
+    }
+
+    /**
+     * @return The total amount of memory in bytes; {@code -1} if undefined.
+     * @see ClusterMetrics#getNonHeapMemoryTotal()
+     */
+    @Order(43)
+    public long nonheapMemoryTotal() {
+        return metrics.getNonHeapMemoryTotal();
+    }
+
+    /**
+     * @return Uptime of the JVM in milliseconds.
+     * @see ClusterMetrics#getUpTime()
+     */
+    @Order(44)
+    public long uptime() {
+        return metrics.getUpTime();
+    }
+
+    /**
+     * @return Start time of the JVM in milliseconds.
+     * @see ClusterMetrics#getStartTime()
+     */
+    @Order(45)
+    public Date jvmStartTime() {
+        return new Date(metrics.getStartTime());
+    }
+
+    /**
+     * @return Node start time.
+     * @see ClusterMetrics#getNodeStartTime()
+     */
+    @Order(46)
+    public Date nodeStartTime() {
+        return new Date(metrics.getNodeStartTime());
+    }
+
+    /**
+     * @return Last data version.
+     * @see ClusterMetrics#getLastDataVersion()
+     */
+    @Order(47)
+    public long lastDataVersion() {
+        return metrics.getLastDataVersion();
+    }
+
+    /**
+     * @return Current thread count.
+     * @see ClusterMetrics#getCurrentThreadCount()
+     */
+    @Order(48)
+    public int curThreadCount() {
+        return metrics.getCurrentThreadCount();
+    }
+
+    /**
+     * @return Maximum thread count.
+     * @see ClusterMetrics#getMaximumThreadCount()
+     */
+    @Order(49)
+    public int maxThreadCount() {
+        return metrics.getMaximumThreadCount();
+    }
+
+    /**
+     * @return Total started thread count.
+     * @see ClusterMetrics#getTotalStartedThreadCount()
+     */
+    @Order(50)
+    public long totalThreadCount() {
+        return metrics.getTotalStartedThreadCount();
+    }
+
+    /**
+     * @return Current daemon thread count.
+     * @see ClusterMetrics#getCurrentDaemonThreadCount()
+     */
+    @Order(51)
+    public int curDaemonThreadCount() {
+        return metrics.getCurrentDaemonThreadCount();
+    }
+
+    /**
+     * @return Sent messages count.
+     * @see ClusterMetrics#getSentMessagesCount()
+     */
+    @Order(52)
+    public int sentMessagesCount() {
+        return metrics.getSentMessagesCount();
+    }
+
+    /**
+     * @return Sent bytes count.
+     * @see ClusterMetrics#getSentBytesCount()
+     */
+    @Order(53)
+    public long sentBytesCount() {
+        return metrics.getSentBytesCount();
+    }
+
+    /**
+     * @return Received messages count.
+     * @see ClusterMetrics#getReceivedMessagesCount()
+     */
+    @Order(54)
+    public int receivedMessagesCount() {
+        return metrics.getReceivedMessagesCount();
+    }
+
+    /**
+     * @return Received bytes count.
+     * @see ClusterMetrics#getReceivedBytesCount()
+     */
+    @Order(55)
+    public long receivedBytesCount() {
+        return metrics.getReceivedBytesCount();
+    }
+
+    /**
+     * @return Outbound messages queue size.
+     * @see ClusterMetrics#getOutboundMessagesQueueSize()
+     */
+    @Order(56)
+    public int outboundMessagesQueue() {
+        return metrics.getOutboundMessagesQueueSize();
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/SystemViewSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/SystemViewSelfTest.java
index 95240e2..3aa297a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/metric/SystemViewSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/SystemViewSelfTest.java
@@ -75,6 +75,7 @@ import org.apache.ignite.internal.binary.mutabletest.GridBinaryTestClasses.TestO
 import org.apache.ignite.internal.client.thin.ProtocolVersion;
 import org.apache.ignite.internal.managers.systemview.walker.BaselineNodeAttributeViewWalker;
 import org.apache.ignite.internal.managers.systemview.walker.CachePagesListViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.NodeAttributeViewWalker;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
@@ -91,7 +92,9 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.services.ServiceConfiguration;
 import org.apache.ignite.spi.systemview.view.BaselineNodeAttributeView;
+import org.apache.ignite.spi.systemview.view.BaselineNodeView;
 import org.apache.ignite.spi.systemview.view.BinaryMetadataView;
+import org.apache.ignite.spi.systemview.view.CacheGroupIoView;
 import org.apache.ignite.spi.systemview.view.CacheGroupView;
 import org.apache.ignite.spi.systemview.view.CachePagesListView;
 import org.apache.ignite.spi.systemview.view.CacheView;
@@ -101,6 +104,8 @@ import org.apache.ignite.spi.systemview.view.ComputeTaskView;
 import org.apache.ignite.spi.systemview.view.ContinuousQueryView;
 import org.apache.ignite.spi.systemview.view.FiltrableSystemView;
 import org.apache.ignite.spi.systemview.view.MetastorageView;
+import org.apache.ignite.spi.systemview.view.NodeAttributeView;
+import org.apache.ignite.spi.systemview.view.NodeMetricsView;
 import org.apache.ignite.spi.systemview.view.PagesListView;
 import org.apache.ignite.spi.systemview.view.ScanQueryView;
 import org.apache.ignite.spi.systemview.view.ServiceView;
@@ -125,11 +130,14 @@ import org.junit.Test;
 
 import static org.apache.ignite.configuration.AtomicConfiguration.DFLT_ATOMIC_SEQUENCE_RESERVE_SIZE;
 import static org.apache.ignite.internal.managers.discovery.GridDiscoveryManager.NODES_SYS_VIEW;
+import static org.apache.ignite.internal.managers.discovery.GridDiscoveryManager.NODE_ATTRIBUTES_SYS_VIEW;
+import static org.apache.ignite.internal.managers.discovery.GridDiscoveryManager.NODE_METRICS_SYS_VIEW;
 import static org.apache.ignite.internal.managers.systemview.GridSystemViewManager.STREAM_POOL_QUEUE_VIEW;
 import static org.apache.ignite.internal.managers.systemview.GridSystemViewManager.SYS_POOL_QUEUE_VIEW;
 import static org.apache.ignite.internal.managers.systemview.ScanQuerySystemView.SCAN_QRY_SYS_VIEW;
 import static org.apache.ignite.internal.processors.cache.ClusterCachesInfo.CACHES_VIEW;
 import static org.apache.ignite.internal.processors.cache.ClusterCachesInfo.CACHE_GRPS_VIEW;
+import static org.apache.ignite.internal.processors.cache.GridCacheProcessor.CACHE_GRP_IO_VIEW;
 import static org.apache.ignite.internal.processors.cache.GridCacheProcessor.CACHE_GRP_PAGE_LIST_VIEW;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheGroupId;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.cacheId;
@@ -137,6 +145,7 @@ import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBina
 import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.METASTORE_VIEW;
 import static org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager.DATA_REGION_PAGE_LIST_VIEW;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager.TXS_MON_LIST;
+import static org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor.BASELINE_NODES_SYS_VIEW;
 import static org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor.BASELINE_NODE_ATTRIBUTES_SYS_VIEW;
 import static org.apache.ignite.internal.processors.continuous.GridContinuousProcessor.CQ_SYS_VIEW;
 import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.DEFAULT_DS_GROUP_NAME;
@@ -622,6 +631,99 @@ public class SystemViewSelfTest extends GridCommonAbstractTest {
     }
 
     /** */
+    @Test
+    public void testNodeAttributes() throws Exception {
+        try (
+            IgniteEx ignite0 = startGrid(getConfiguration(getTestIgniteInstanceName(0))
+                .setUserAttributes(F.asMap("name", "val0")));
+            IgniteEx ignite1 = startGrid(getConfiguration(getTestIgniteInstanceName(1))
+                .setUserAttributes(F.asMap("name", "val1")))
+        ) {
+            awaitPartitionMapExchange();
+
+            SystemView<NodeAttributeView> view = ignite0.context().systemView().view(NODE_ATTRIBUTES_SYS_VIEW);
+
+            assertEquals(ignite0.cluster().localNode().attributes().size() +
+                ignite1.cluster().localNode().attributes().size(), view.size());
+
+            assertEquals(1, F.size(view.iterator(), row -> "name".equals(row.name()) && "val0".equals(row.value())));
+            assertEquals(1, F.size(view.iterator(), row -> "name".equals(row.name()) && "val1".equals(row.value())));
+
+            // Test filtering.
+            assertTrue(view instanceof FiltrableSystemView);
+
+            Iterator<NodeAttributeView> iter = ((FiltrableSystemView<NodeAttributeView>)view)
+                .iterator(F.asMap(NodeAttributeViewWalker.NODE_ID_FILTER, ignite0.cluster().localNode().id()));
+
+            assertEquals(1, F.size(iter, row -> "name".equals(row.name()) && "val0".equals(row.value())));
+
+            iter = ((FiltrableSystemView<NodeAttributeView>)view).iterator(
+                F.asMap(NodeAttributeViewWalker.NODE_ID_FILTER, ignite1.cluster().localNode().id().toString()));
+
+            assertEquals(1, F.size(iter, row -> "name".equals(row.name()) && "val1".equals(row.value())));
+
+            iter = ((FiltrableSystemView<NodeAttributeView>)view).iterator(
+                F.asMap(NodeAttributeViewWalker.NODE_ID_FILTER, "malformed-id"));
+
+            assertEquals(0, F.size(iter));
+
+            iter = ((FiltrableSystemView<NodeAttributeView>)view).iterator(
+                F.asMap(NodeAttributeViewWalker.NAME_FILTER, "name"));
+
+            assertEquals(2, F.size(iter));
+
+            iter = ((FiltrableSystemView<NodeAttributeView>)view)
+                .iterator(F.asMap(NodeAttributeViewWalker.NODE_ID_FILTER, ignite0.cluster().localNode().id(),
+                    NodeAttributeViewWalker.NAME_FILTER, "name"));
+
+            assertEquals(1, F.size(iter));
+        }
+    }
+
+    /** */
+    @Test
+    public void testNodeMetrics() throws Exception {
+        long ts = U.currentTimeMillis();
+
+        try (IgniteEx ignite0 = startGrid(0); IgniteEx ignite1 = startGrid(1)) {
+            awaitPartitionMapExchange();
+
+            SystemView<NodeMetricsView> view = ignite0.context().systemView().view(NODE_METRICS_SYS_VIEW);
+
+            assertEquals(2, view.size());
+            assertEquals(1, F.size(view.iterator(), row -> row.nodeId().equals(ignite0.cluster().localNode().id())));
+            assertEquals(1, F.size(view.iterator(), row -> row.nodeId().equals(ignite1.cluster().localNode().id())));
+            assertEquals(2, F.size(view.iterator(), row -> row.lastUpdateTime().getTime() >= ts));
+        }
+    }
+
+    /** */
+    @Test
+    public void testCacheGroupIo() throws Exception {
+        cleanPersistenceDir();
+
+        try (IgniteEx ignite = startGrid(getConfiguration().setDataStorageConfiguration(
+            new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration().setPersistenceEnabled(true))))
+        ) {
+            ignite.cluster().state(ClusterState.ACTIVE);
+
+            IgniteCache<Object, Object> cache = ignite.createCache("cache");
+
+            cache.put(0, 0);
+            cache.get(0);
+
+            SystemView<CacheGroupIoView> view = ignite.context().systemView().view(CACHE_GRP_IO_VIEW);
+
+            CacheGroupIoView row = F.find(view, null,
+                (IgnitePredicate<CacheGroupIoView>)r -> "cache".equals(r.cacheGroupName()));
+
+            assertNotNull(row);
+            assertTrue(row.logicalReads() > 0);
+        }
+    }
+
+    /** */
     private void checkViewsState(SystemView<ClusterNodeView> views, ClusterNode loc, ClusterNode rmt) {
         assertEquals(2, views.size());
 
@@ -1871,6 +1973,35 @@ public class SystemViewSelfTest extends GridCommonAbstractTest {
 
     /** */
     @Test
+    public void testBaselineNodes() throws Exception {
+        cleanPersistenceDir();
+
+        try (
+            IgniteEx ignite0 = startGrid(getConfiguration(getTestIgniteInstanceName(0))
+                .setDataStorageConfiguration(new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(true)))
+                .setConsistentId("consId0"));
+            IgniteEx ignite1 = startGrid(getConfiguration(getTestIgniteInstanceName(1))
+                .setDataStorageConfiguration(new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(true)))
+                .setConsistentId("consId1"));
+        ) {
+            ignite0.cluster().state(ClusterState.ACTIVE);
+
+            ignite1.close();
+
+            awaitPartitionMapExchange();
+
+            SystemView<BaselineNodeView> view = ignite0.context().systemView().view(BASELINE_NODES_SYS_VIEW);
+
+            assertEquals(2, view.size());
+            assertEquals(1, F.size(view.iterator(), row -> "consId0".equals(row.consistentId()) && row.online()));
+            assertEquals(1, F.size(view.iterator(), row -> "consId1".equals(row.consistentId()) && !row.online()));
+        }
+    }
+
+    /** */
+    @Test
     public void testBaselineNodeAttributes() throws Exception {
         cleanPersistenceDir();
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
index f7ec0a1..8b5d691 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
@@ -32,7 +32,6 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.BiConsumer;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -58,10 +57,6 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.sys.SqlSystemTableEngine;
 import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemView;
-import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewBaselineNodes;
-import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewCacheGroupsIOStatistics;
-import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeAttributes;
-import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemViewNodeMetrics;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.F;
@@ -204,9 +199,6 @@ public class SchemaManager {
         // Register PUBLIC schema which is always present.
         schemas.put(QueryUtils.DFLT_SCHEMA, new H2Schema(QueryUtils.DFLT_SCHEMA, true));
 
-        // Create system views.
-        createSystemViews();
-
         // Create schemas listed in node's configuration.
         createPredefinedSchemas(schemaNames);
     }
@@ -255,29 +247,6 @@ public class SchemaManager {
     }
 
     /**
-     * Create system views.
-     */
-    private void createSystemViews() throws IgniteCheckedException {
-        for (SqlSystemView view : systemViews(ctx))
-            createSystemView(QueryUtils.SCHEMA_SYS, view);
-    }
-
-    /**
-     * @param ctx Context.
-     * @return Predefined system views.
-     */
-    private Collection<SqlSystemView> systemViews(GridKernalContext ctx) {
-        Collection<SqlSystemView> views = new ArrayList<>();
-
-        views.add(new SqlSystemViewNodeAttributes(ctx));
-        views.add(new SqlSystemViewBaselineNodes(ctx));
-        views.add(new SqlSystemViewNodeMetrics(ctx));
-        views.add(new SqlSystemViewCacheGroupsIOStatistics(ctx));
-
-        return views;
-    }
-
-    /**
      * Create predefined schemas.
      *
      * @param schemaNames Schema names.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewBaselineNodes.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewBaselineNodes.java
deleted file mode 100644
index 2090830..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewBaselineNodes.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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.processors.query.h2.sys.view;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cluster.BaselineTopology;
-import org.apache.ignite.internal.util.typedef.F;
-import org.h2.engine.Session;
-import org.h2.result.Row;
-import org.h2.result.SearchRow;
-import org.h2.value.Value;
-
-/**
- * System view: baseline nodes.
- */
-public class SqlSystemViewBaselineNodes extends SqlAbstractLocalSystemView {
-    /**
-     * @param ctx Grid context.
-     */
-    public SqlSystemViewBaselineNodes(GridKernalContext ctx) {
-        super("BASELINE_NODES", "Baseline topology nodes", ctx,
-            newColumn("CONSISTENT_ID"),
-            newColumn("ONLINE", Value.BOOLEAN)
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Row> getRows(Session ses, SearchRow first, SearchRow last) {
-        List<Row> rows = new ArrayList<>();
-
-        BaselineTopology blt = ctx.state().clusterState().baselineTopology();
-
-        if (blt == null)
-            return rows.iterator();
-
-        Set<Object> consistentIds = blt.consistentIds();
-
-        Collection<ClusterNode> srvNodes = ctx.discovery().aliveServerNodes();
-
-        Set<Object> aliveNodeIds = new HashSet<>(F.nodeConsistentIds(srvNodes));
-
-        for (Object consistentId : consistentIds) {
-            rows.add(
-                createRow(
-                    ses,
-                    toStringSafe(consistentId),
-                    aliveNodeIds.contains(consistentId)
-                )
-            );
-        }
-
-        return rows.iterator();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean canGetRowCount() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getRowCount() {
-        BaselineTopology blt = ctx.state().clusterState().baselineTopology();
-
-        return blt == null ? 0 : blt.consistentIds().size();
-    }
-}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroupsIOStatistics.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroupsIOStatistics.java
deleted file mode 100644
index 33e73795..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewCacheGroupsIOStatistics.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.processors.query.h2.sys.view;
-
-import java.util.Collection;
-import java.util.Iterator;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.metric.GridMetricManager;
-import org.apache.ignite.internal.processors.metric.MetricRegistry;
-import org.apache.ignite.spi.metric.IntMetric;
-import org.apache.ignite.spi.metric.LongMetric;
-import org.apache.ignite.spi.metric.ObjectMetric;
-import org.h2.engine.Session;
-import org.h2.result.Row;
-import org.h2.result.SearchRow;
-import org.h2.value.Value;
-
-import static java.util.Collections.emptyIterator;
-import static java.util.Collections.singleton;
-import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.LOGICAL_READS;
-import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.PHYSICAL_READS;
-import static org.apache.ignite.internal.metric.IoStatisticsType.CACHE_GROUP;
-import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
-import static org.apache.ignite.internal.util.lang.GridFunc.iterator;
-
-/**
- * System view of cache group IO statistics.
- */
-public class SqlSystemViewCacheGroupsIOStatistics extends SqlAbstractLocalSystemView {
-    /**
-     * @param ctx Grid context.
-     */
-    public SqlSystemViewCacheGroupsIOStatistics(GridKernalContext ctx) {
-        super("LOCAL_CACHE_GROUPS_IO", "Local node IO statistics for cache groups", ctx, "CACHE_GROUP_NAME",
-            newColumn("CACHE_GROUP_ID", Value.INT),
-            newColumn("CACHE_GROUP_NAME"),
-            newColumn("PHYSICAL_READS", Value.LONG),
-            newColumn("LOGICAL_READS", Value.LONG)
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Row> getRows(Session ses, SearchRow first, SearchRow last) {
-        SqlSystemViewColumnCondition nameCond = conditionForColumn("CACHE_GROUP_NAME", first, last);
-
-        if (nameCond.isEquality()) {
-            String cacheGrpName = nameCond.valueForEquality().getString();
-
-            MetricRegistry mreg = ctx.metric().registry(metricName(CACHE_GROUP.metricGroupName(), cacheGrpName));
-
-            IntMetric grpId = mreg.findMetric("grpId");
-            ObjectMetric<String> grpName = mreg.findMetric("name");
-
-            if (grpId == null)
-                emptyIterator();
-
-            if (mreg != null) {
-                return singleton(toRow(ses,
-                    grpId.value(),
-                    grpName.value(),
-                    mreg)
-                ).iterator();
-            }
-        }
-        else {
-            Collection<CacheGroupContext> grpCtxs = ctx.cache().cacheGroups();
-
-            GridMetricManager mmgr = ctx.metric();
-
-            return iterator(grpCtxs,
-                grpCtx -> toRow(ses,
-                    grpCtx.groupId(),
-                    grpCtx.cacheOrGroupName(),
-                    mmgr.registry(metricName(CACHE_GROUP.metricGroupName(), grpCtx.cacheOrGroupName()))),
-                true,
-                grpCtx -> !grpCtx.systemCache());
-        }
-
-        return emptyIterator();
-    }
-
-    /** */
-    private Row toRow(Session ses, int grpId, String grpName, MetricRegistry mreg) {
-        IntMetric grpIdMetric = mreg.findMetric("grpId");
-
-        if (grpIdMetric == null)
-            return createRow(ses, grpId, grpName, 0, 0);
-
-        return createRow(
-            ses,
-            grpId,
-            grpName,
-            mreg.<LongMetric>findMetric(PHYSICAL_READS).value(),
-            mreg.<LongMetric>findMetric(LOGICAL_READS).value()
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean canGetRowCount() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getRowCount() {
-        return ctx.cache().cacheGroups().size();
-    }
-}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeAttributes.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeAttributes.java
deleted file mode 100644
index 15398c6..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeAttributes.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.processors.query.h2.sys.view;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.util.typedef.F;
-import org.h2.engine.Session;
-import org.h2.result.Row;
-import org.h2.result.SearchRow;
-import org.h2.value.Value;
-
-/**
- * System view: node attributes.
- */
-public class SqlSystemViewNodeAttributes extends SqlAbstractLocalSystemView {
-    /**
-     * @param ctx Grid context.
-     */
-    public SqlSystemViewNodeAttributes(GridKernalContext ctx) {
-        super("NODE_ATTRIBUTES", "Node attributes", ctx, new String[] {"NODE_ID,NAME", "NAME"},
-            newColumn("NODE_ID", Value.UUID),
-            newColumn("NAME"),
-            newColumn("VALUE")
-        );
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public Iterator<Row> getRows(Session ses, SearchRow first, SearchRow last) {
-        Collection<ClusterNode> nodes;
-
-        SqlSystemViewColumnCondition idCond = conditionForColumn("NODE_ID", first, last);
-        SqlSystemViewColumnCondition nameCond = conditionForColumn("NAME", first, last);
-
-        if (idCond.isEquality()) {
-            try {
-                UUID nodeId = uuidFromValue(idCond.valueForEquality());
-
-                ClusterNode node = nodeId == null ? null : ctx.discovery().node(nodeId);
-
-                if (node != null)
-                    nodes = Collections.singleton(node);
-                else
-                    nodes = Collections.emptySet();
-            }
-            catch (Exception e) {
-                nodes = Collections.emptySet();
-            }
-        }
-        else
-            nodes = F.concat(false, ctx.discovery().allNodes(), ctx.discovery().daemonNodes());
-
-        if (nameCond.isEquality()) {
-            String attrName = nameCond.valueForEquality().getString();
-
-            List<Row> rows = new ArrayList<>();
-
-            for (ClusterNode node : nodes) {
-                if (node.attributes().containsKey(attrName)) {
-                    rows.add(
-                        createRow(
-                            ses,
-                            node.id(),
-                            attrName,
-                            node.attribute(attrName)
-                        )
-                    );
-                }
-            }
-
-            return rows.iterator();
-        }
-        else {
-            return F.concat(F.iterator(nodes,
-                node -> F.iterator(node.attributes().entrySet(),
-                    attr -> createRow(
-                        ses,
-                        node.id(),
-                        attr.getKey(),
-                        attr.getValue()),
-                    true).iterator(),
-                true)
-            );
-        }
-    }
-}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
deleted file mode 100644
index 6183907..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sys/view/SqlSystemViewNodeMetrics.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/*
- * 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.processors.query.h2.sys.view;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.cluster.ClusterMetrics;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.util.typedef.F;
-import org.h2.engine.Session;
-import org.h2.result.Row;
-import org.h2.result.SearchRow;
-import org.h2.value.Value;
-
-/**
- * System view: node metrics.
- */
-public class SqlSystemViewNodeMetrics extends SqlAbstractLocalSystemView {
-    /**
-     * @param ctx Grid context.
-     */
-    public SqlSystemViewNodeMetrics(GridKernalContext ctx) {
-        super("NODE_METRICS", "Node metrics", ctx, "NODE_ID",
-            newColumn("NODE_ID", Value.UUID),
-            newColumn("LAST_UPDATE_TIME", Value.TIMESTAMP),
-            newColumn("MAX_ACTIVE_JOBS", Value.INT),
-            newColumn("CUR_ACTIVE_JOBS", Value.INT),
-            newColumn("AVG_ACTIVE_JOBS", Value.FLOAT),
-            newColumn("MAX_WAITING_JOBS", Value.INT),
-            newColumn("CUR_WAITING_JOBS", Value.INT),
-            newColumn("AVG_WAITING_JOBS", Value.FLOAT),
-            newColumn("MAX_REJECTED_JOBS", Value.INT),
-            newColumn("CUR_REJECTED_JOBS", Value.INT),
-            newColumn("AVG_REJECTED_JOBS", Value.FLOAT),
-            newColumn("TOTAL_REJECTED_JOBS", Value.INT),
-            newColumn("MAX_CANCELED_JOBS", Value.INT),
-            newColumn("CUR_CANCELED_JOBS", Value.INT),
-            newColumn("AVG_CANCELED_JOBS", Value.FLOAT),
-            newColumn("TOTAL_CANCELED_JOBS", Value.INT),
-            newColumn("MAX_JOBS_WAIT_TIME", Value.LONG),
-            newColumn("CUR_JOBS_WAIT_TIME", Value.LONG),
-            newColumn("AVG_JOBS_WAIT_TIME", Value.LONG),
-            newColumn("MAX_JOBS_EXECUTE_TIME", Value.LONG),
-            newColumn("CUR_JOBS_EXECUTE_TIME", Value.LONG),
-            newColumn("AVG_JOBS_EXECUTE_TIME", Value.LONG),
-            newColumn("TOTAL_JOBS_EXECUTE_TIME", Value.LONG),
-            newColumn("TOTAL_EXECUTED_JOBS", Value.INT),
-            newColumn("TOTAL_EXECUTED_TASKS", Value.INT),
-            newColumn("TOTAL_BUSY_TIME", Value.LONG),
-            newColumn("TOTAL_IDLE_TIME", Value.LONG),
-            newColumn("CUR_IDLE_TIME", Value.LONG),
-            newColumn("BUSY_TIME_PERCENTAGE", Value.FLOAT),
-            newColumn("IDLE_TIME_PERCENTAGE", Value.FLOAT),
-            newColumn("TOTAL_CPU", Value.INT),
-            newColumn("CUR_CPU_LOAD", Value.DOUBLE),
-            newColumn("AVG_CPU_LOAD", Value.DOUBLE),
-            newColumn("CUR_GC_CPU_LOAD", Value.DOUBLE),
-            newColumn("HEAP_MEMORY_INIT", Value.LONG),
-            newColumn("HEAP_MEMORY_USED", Value.LONG),
-            newColumn("HEAP_MEMORY_COMMITED", Value.LONG),
-            newColumn("HEAP_MEMORY_MAX", Value.LONG),
-            newColumn("HEAP_MEMORY_TOTAL", Value.LONG),
-            newColumn("NONHEAP_MEMORY_INIT", Value.LONG),
-            newColumn("NONHEAP_MEMORY_USED", Value.LONG),
-            newColumn("NONHEAP_MEMORY_COMMITED", Value.LONG),
-            newColumn("NONHEAP_MEMORY_MAX", Value.LONG),
-            newColumn("NONHEAP_MEMORY_TOTAL", Value.LONG),
-            newColumn("UPTIME", Value.LONG),
-            newColumn("JVM_START_TIME", Value.TIMESTAMP),
-            newColumn("NODE_START_TIME", Value.TIMESTAMP),
-            newColumn("LAST_DATA_VERSION", Value.LONG),
-            newColumn("CUR_THREAD_COUNT", Value.INT),
-            newColumn("MAX_THREAD_COUNT", Value.INT),
-            newColumn("TOTAL_THREAD_COUNT", Value.LONG),
-            newColumn("CUR_DAEMON_THREAD_COUNT", Value.INT),
-            newColumn("SENT_MESSAGES_COUNT", Value.INT),
-            newColumn("SENT_BYTES_COUNT", Value.LONG),
-            newColumn("RECEIVED_MESSAGES_COUNT", Value.INT),
-            newColumn("RECEIVED_BYTES_COUNT", Value.LONG),
-            newColumn("OUTBOUND_MESSAGES_QUEUE", Value.INT)
-        );
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterator<Row> getRows(Session ses, SearchRow first, SearchRow last) {
-        List<Row> rows = new ArrayList<>();
-
-        Collection<ClusterNode> nodes;
-
-        SqlSystemViewColumnCondition idCond = conditionForColumn("NODE_ID", first, last);
-
-        if (idCond.isEquality()) {
-            try {
-                UUID nodeId = uuidFromValue(idCond.valueForEquality());
-
-                ClusterNode node = nodeId == null ? null : ctx.discovery().node(nodeId);
-
-                if (node != null)
-                    nodes = Collections.singleton(node);
-                else
-                    nodes = Collections.emptySet();
-            }
-            catch (Exception e) {
-                nodes = Collections.emptySet();
-            }
-        }
-        else
-            nodes = F.concat(false, ctx.discovery().allNodes(), ctx.discovery().daemonNodes());
-
-        for (ClusterNode node : nodes) {
-            if (node != null) {
-                ClusterMetrics metrics = node.metrics();
-
-                rows.add(
-                    createRow(
-                        ses,
-                        node.id(),
-                        valueTimestampFromMillis(metrics.getLastUpdateTime()),
-                        metrics.getMaximumActiveJobs(),
-                        metrics.getCurrentActiveJobs(),
-                        metrics.getAverageActiveJobs(),
-                        metrics.getMaximumWaitingJobs(),
-                        metrics.getCurrentWaitingJobs(),
-                        metrics.getAverageWaitingJobs(),
-                        metrics.getMaximumRejectedJobs(),
-                        metrics.getCurrentRejectedJobs(),
-                        metrics.getAverageRejectedJobs(),
-                        metrics.getTotalRejectedJobs(),
-                        metrics.getMaximumCancelledJobs(),
-                        metrics.getCurrentCancelledJobs(),
-                        metrics.getAverageCancelledJobs(),
-                        metrics.getTotalCancelledJobs(),
-                        metrics.getMaximumJobWaitTime(),
-                        metrics.getCurrentJobWaitTime(),
-                        (long)metrics.getAverageJobWaitTime(),
-                        metrics.getMaximumJobExecuteTime(),
-                        metrics.getCurrentJobExecuteTime(),
-                        (long)metrics.getAverageJobExecuteTime(),
-                        metrics.getTotalJobsExecutionTime(),
-                        metrics.getTotalExecutedJobs(),
-                        metrics.getTotalExecutedTasks(),
-                        metrics.getTotalBusyTime(),
-                        metrics.getTotalIdleTime(),
-                        metrics.getCurrentIdleTime(),
-                        metrics.getBusyTimePercentage(),
-                        metrics.getIdleTimePercentage(),
-                        metrics.getTotalCpus(),
-                        metrics.getCurrentCpuLoad(),
-                        metrics.getAverageCpuLoad(),
-                        metrics.getCurrentGcCpuLoad(),
-                        metrics.getHeapMemoryInitialized(),
-                        metrics.getHeapMemoryUsed(),
-                        metrics.getHeapMemoryCommitted(),
-                        metrics.getHeapMemoryMaximum(),
-                        metrics.getHeapMemoryTotal(),
-                        metrics.getNonHeapMemoryInitialized(),
-                        metrics.getNonHeapMemoryUsed(),
-                        metrics.getNonHeapMemoryCommitted(),
-                        metrics.getNonHeapMemoryMaximum(),
-                        metrics.getNonHeapMemoryTotal(),
-                        metrics.getUpTime(),
-                        valueTimestampFromMillis(metrics.getStartTime()),
-                        valueTimestampFromMillis(metrics.getNodeStartTime()),
-                        metrics.getLastDataVersion(),
-                        metrics.getCurrentThreadCount(),
-                        metrics.getMaximumThreadCount(),
-                        metrics.getTotalStartedThreadCount(),
-                        metrics.getCurrentDaemonThreadCount(),
-                        metrics.getSentMessagesCount(),
-                        metrics.getSentBytesCount(),
-                        metrics.getReceivedMessagesCount(),
-                        metrics.getReceivedBytesCount(),
-                        metrics.getOutboundMessagesQueueSize()
-                    )
-                );
-            }
-        }
-
-        return rows.iterator();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean canGetRowCount() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getRowCount() {
-        return F.concat(false, ctx.discovery().allNodes(), ctx.discovery().daemonNodes()).size();
-    }
-}