You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ni...@apache.org on 2021/03/26 07:33:18 UTC

[ignite] branch master updated: IGNITE-14368 Data structures system views (#8911)

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

nizhikov 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 7b033cf  IGNITE-14368 Data structures system views (#8911)
7b033cf is described below

commit 7b033cfe1f311d5a479605f565290bdc388a1823
Author: Nikolay <ni...@apache.org>
AuthorDate: Fri Mar 26 10:32:58 2021 +0300

    IGNITE-14368 Data structures system views (#8911)
---
 docs/_docs/monitoring-metrics/system-views.adoc    | 153 +++++
 .../internal/jdbc2/JdbcMetadataSelfTest.java       |  11 +-
 .../ignite/jdbc/thin/JdbcThinMetadataSelfTest.java |  76 ++-
 .../SystemViewRowAttributeWalkerGenerator.java     |  18 +
 .../systemview/walker/AtomicLongViewWalker.java    |  52 ++
 .../walker/AtomicReferenceViewWalker.java          |  52 ++
 .../walker/AtomicSequenceViewWalker.java           |  54 ++
 .../systemview/walker/AtomicStampedViewWalker.java |  54 ++
 .../walker/CountDownLatchViewWalker.java           |  56 ++
 .../systemview/walker/QueueViewWalker.java         |  61 ++
 .../systemview/walker/ReentrantLockViewWalker.java |  60 ++
 .../systemview/walker/SemaphoreViewWalker.java     |  60 ++
 .../managers/systemview/walker/SetViewWalker.java  |  57 ++
 .../datastructures/CacheDataStructuresManager.java |  10 +
 .../datastructures/DataStructuresProcessor.java    | 158 ++++-
 .../datastructures/GridCacheQueueProxy.java        |  10 +
 .../datastructures/GridCacheSetProxy.java          |  10 +
 .../datastructures/AbstractDataStructureView.java  |  58 ++
 .../view/datastructures/AtomicLongView.java        |  48 ++
 .../view/datastructures/AtomicReferenceView.java   |  50 ++
 .../view/datastructures/AtomicSequenceView.java    |  57 ++
 .../view/datastructures/AtomicStampedView.java     |  59 ++
 .../view/datastructures/CountDownLatchView.java    |  64 ++
 .../systemview/view/datastructures/QueueView.java  |  89 +++
 .../view/datastructures/ReentrantLockView.java     |  82 +++
 .../view/datastructures/SemaphoreView.java         |  82 +++
 .../systemview/view/datastructures/SetView.java    |  78 +++
 .../ignite/internal/metric/SystemViewSelfTest.java | 677 +++++++++++++++++++++
 28 files changed, 2292 insertions(+), 4 deletions(-)

diff --git a/docs/_docs/monitoring-metrics/system-views.adoc b/docs/_docs/monitoring-metrics/system-views.adoc
index ac45667..784c123 100644
--- a/docs/_docs/monitoring-metrics/system-views.adoc
+++ b/docs/_docs/monitoring-metrics/system-views.adoc
@@ -703,3 +703,156 @@ This view exposes the contents of the metastorage cache.
 |NAME | string | Name
 |VALUE | string | String or raw binary (if data could not be deserialized for some reason) representation of an element
 |===
+
+== DS_QUEUES
+
+This view exposes the list of `IgniteQueue`.
+Note, queue will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|ID | UUID | ID
+|NAME | string | Data structure name
+|CAPACITY | int | Capacity
+|SIZE | int | Current size
+|BOUNDED | boolean | `True` when queue capacity is bounded
+|COLLOCATED | boolean | `True` when collocated
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_SETS
+
+This view exposes the list of `IgniteSet`.
+Note, set will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|ID | UUID | ID
+|NAME | string | Data structure name
+|SIZE | int | Current size
+|COLLOCATED | boolean | `True` when collocated
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_ATOMICSEQUENCES
+
+This view exposes the list of `IgniteAtomicSequence`.
+Note, atomicsequence will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|VALUE | long | Current sequence value
+|BATCH_SIZE | long | Local batch size
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_ATOMICLONGS
+
+This view exposes the list of `IgniteAtomicLong`.
+Note, atomiclong will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|VALUE | long | Current value
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_ATOMICREFERENCES
+
+This view exposes the list of `IgniteAtomicReference`.
+Note, atomicreference will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|VALUE | string | Current value
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_ATOMICSTAMPED
+
+This view exposes the list of `IgniteAtomicStamped`.
+Note, atomicstamped will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|VALUE | string | Current value
+|STAMP | string | Current stamp value
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_COUNTDOWNLATCHES
+
+This view exposes the list of `IgniteCountDownLatch`.
+Note, latch will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|COUNT | int | Current count
+|INITIAL_COUT | int | Initial count
+|AUTO_DELETE | boolean | True to automatically delete the latch from a cache when its count reaches zero
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_SEMAPHORES
+
+This view exposes the list of `IgniteSemaphore`.
+Note, semaphores will be shown on the non parent nodes only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|AVAILABLE_PERMITS | long | Number of permits available
+|HAS_QUEUED_THREADS | boolean | `True` if there may be other threads waiting to acquire the lock
+|QUEUE_LENGTH | int | The estimated number of nodes waiting for this lock
+|FAILOVER_SAFE | boolean | `True` if failover safe
+|BROKEN | boolean | `True` if a node failed on this semaphore and `FAILOVER_SAFE` flag was set to `false`, `false` otherwise.
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
+
+== DS_REENTRANTLOCKS
+
+This view exposes the contents of `IgniteLock`.
+Note, lock will be shown on the non parent node only after initial usage on that node.
+
+[{table_opts}]
+|===
+|Column | Data type |  Description
+|NAME | string | Data structure name
+|LOCKED | boolean | `True` if locked
+|HAS_QUEUED_THREADS | boolean | `True` if there may be other threads waiting to acquire the lock
+|FAILOVER_SAFE | boolean | `True` if failover safe
+|FAIR | boolean | `True` if lock is fair
+|BROKEN | boolean | `True` if a node failed on this semaphore and `FAILOVER_SAFE` flag was set to `false`, `false` otherwise.
+|GROUP_NAME | string | Cache group name to store data structure
+|GROUP_ID | int | Cache group id to store data structure
+|REMOVED | boolean | `True` if removed
+|===
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
index 07131cc..b5e3714 100755
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
@@ -348,7 +348,16 @@ public class JdbcMetadataSelfTest extends GridCommonAbstractTest {
             "PARTITION_STATES",
             "BINARY_METADATA",
             "DISTRIBUTED_METASTORAGE",
-            "METRICS"
+            "METRICS",
+            "DS_QUEUES",
+            "DS_SETS",
+            "DS_ATOMICSEQUENCES",
+            "DS_ATOMICLONGS",
+            "DS_ATOMICREFERENCES",
+            "DS_ATOMICSTAMPED",
+            "DS_COUNTDOWNLATCHES",
+            "DS_SEMAPHORES",
+            "DS_REENTRANTLOCKS"
         ));
 
         Set<String> actViews = new HashSet<>();
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 9f32479..b6840f5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -442,7 +442,16 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 "SYS.DATA_REGION_PAGE_LISTS",
                 "SYS.PARTITION_STATES",
                 "SYS.BINARY_METADATA",
-                "SYS.DISTRIBUTED_METASTORAGE"
+                "SYS.DISTRIBUTED_METASTORAGE",
+                "SYS.DS_QUEUES",
+                "SYS.DS_SETS",
+                "SYS.DS_ATOMICSEQUENCES",
+                "SYS.DS_ATOMICLONGS",
+                "SYS.DS_ATOMICREFERENCES",
+                "SYS.DS_ATOMICSTAMPED",
+                "SYS.DS_COUNTDOWNLATCHES",
+                "SYS.DS_SEMAPHORES",
+                "SYS.DS_REENTRANTLOCKS"
             ))
         );
     }
@@ -1000,7 +1009,70 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 "SYS.BINARY_METADATA.FIELDS_COUNT.null.10",
                 "SYS.BINARY_METADATA.TYPE_NAME.null.2147483647",
                 "SYS.DISTRIBUTED_METASTORAGE.NAME.null.2147483647",
-                "SYS.DISTRIBUTED_METASTORAGE.VALUE.null.2147483647"
+                "SYS.DISTRIBUTED_METASTORAGE.VALUE.null.2147483647",
+                "SYS.DS_ATOMICLONGS.GROUP_ID.null.10",
+                "SYS.DS_ATOMICLONGS.GROUP_NAME.null.2147483647",
+                "SYS.DS_ATOMICLONGS.NAME.null.2147483647",
+                "SYS.DS_ATOMICLONGS.REMOVED.null.1",
+                "SYS.DS_ATOMICLONGS.VALUE.null.19",
+                "SYS.DS_ATOMICREFERENCES.GROUP_ID.null.10",
+                "SYS.DS_ATOMICREFERENCES.GROUP_NAME.null.2147483647",
+                "SYS.DS_ATOMICREFERENCES.NAME.null.2147483647",
+                "SYS.DS_ATOMICREFERENCES.REMOVED.null.1",
+                "SYS.DS_ATOMICREFERENCES.VALUE.null.2147483647",
+                "SYS.DS_ATOMICSEQUENCES.BATCH_SIZE.null.19",
+                "SYS.DS_ATOMICSEQUENCES.GROUP_ID.null.10",
+                "SYS.DS_ATOMICSEQUENCES.GROUP_NAME.null.2147483647",
+                "SYS.DS_ATOMICSEQUENCES.NAME.null.2147483647",
+                "SYS.DS_ATOMICSEQUENCES.REMOVED.null.1",
+                "SYS.DS_ATOMICSEQUENCES.VALUE.null.19",
+                "SYS.DS_ATOMICSTAMPED.GROUP_ID.null.10",
+                "SYS.DS_ATOMICSTAMPED.GROUP_NAME.null.2147483647",
+                "SYS.DS_ATOMICSTAMPED.NAME.null.2147483647",
+                "SYS.DS_ATOMICSTAMPED.REMOVED.null.1",
+                "SYS.DS_ATOMICSTAMPED.STAMP.null.2147483647",
+                "SYS.DS_ATOMICSTAMPED.VALUE.null.2147483647",
+                "SYS.DS_COUNTDOWNLATCHES.AUTO_DELETE.null.1",
+                "SYS.DS_COUNTDOWNLATCHES.COUNT.null.10",
+                "SYS.DS_COUNTDOWNLATCHES.GROUP_ID.null.10",
+                "SYS.DS_COUNTDOWNLATCHES.GROUP_NAME.null.2147483647",
+                "SYS.DS_COUNTDOWNLATCHES.INITIAL_COUNT.null.10",
+                "SYS.DS_COUNTDOWNLATCHES.NAME.null.2147483647",
+                "SYS.DS_COUNTDOWNLATCHES.REMOVED.null.1",
+                "SYS.DS_QUEUES.BOUNDED.null.1",
+                "SYS.DS_QUEUES.CAPACITY.null.10",
+                "SYS.DS_QUEUES.SIZE.null.10",
+                "SYS.DS_QUEUES.COLLOCATED.null.1",
+                "SYS.DS_QUEUES.GROUP_ID.null.10",
+                "SYS.DS_QUEUES.GROUP_NAME.null.2147483647",
+                "SYS.DS_QUEUES.ID.null.2147483647",
+                "SYS.DS_QUEUES.NAME.null.2147483647",
+                "SYS.DS_QUEUES.REMOVED.null.1",
+                "SYS.DS_REENTRANTLOCKS.BROKEN.null.1",
+                "SYS.DS_REENTRANTLOCKS.FAILOVER_SAFE.null.1",
+                "SYS.DS_REENTRANTLOCKS.FAIR.null.1",
+                "SYS.DS_REENTRANTLOCKS.GROUP_ID.null.10",
+                "SYS.DS_REENTRANTLOCKS.GROUP_NAME.null.2147483647",
+                "SYS.DS_REENTRANTLOCKS.HAS_QUEUED_THREADS.null.1",
+                "SYS.DS_REENTRANTLOCKS.LOCKED.null.1",
+                "SYS.DS_REENTRANTLOCKS.NAME.null.2147483647",
+                "SYS.DS_REENTRANTLOCKS.REMOVED.null.1",
+                "SYS.DS_SEMAPHORES.AVAILABLE_PERMITS.null.19",
+                "SYS.DS_SEMAPHORES.BROKEN.null.1",
+                "SYS.DS_SEMAPHORES.FAILOVER_SAFE.null.1",
+                "SYS.DS_SEMAPHORES.GROUP_ID.null.10",
+                "SYS.DS_SEMAPHORES.GROUP_NAME.null.2147483647",
+                "SYS.DS_SEMAPHORES.HAS_QUEUED_THREADS.null.1",
+                "SYS.DS_SEMAPHORES.NAME.null.2147483647",
+                "SYS.DS_SEMAPHORES.QUEUE_LENGTH.null.10",
+                "SYS.DS_SEMAPHORES.REMOVED.null.1",
+                "SYS.DS_SETS.COLLOCATED.null.1",
+                "SYS.DS_SETS.GROUP_ID.null.10",
+                "SYS.DS_SETS.GROUP_NAME.null.2147483647",
+                "SYS.DS_SETS.ID.null.2147483647",
+                "SYS.DS_SETS.NAME.null.2147483647",
+                "SYS.DS_SETS.REMOVED.null.1",
+                "SYS.DS_SETS.SIZE.null.10"
                 ));
 
             Assert.assertEquals(expectedCols, actualSystemCols);
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 f668412..32fc198 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
@@ -62,6 +62,15 @@ import org.apache.ignite.spi.systemview.view.StripedExecutorTaskView;
 import org.apache.ignite.spi.systemview.view.SystemView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
 import org.apache.ignite.spi.systemview.view.TransactionView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicLongView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicReferenceView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicSequenceView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicStampedView;
+import org.apache.ignite.spi.systemview.view.datastructures.CountDownLatchView;
+import org.apache.ignite.spi.systemview.view.datastructures.QueueView;
+import org.apache.ignite.spi.systemview.view.datastructures.ReentrantLockView;
+import org.apache.ignite.spi.systemview.view.datastructures.SemaphoreView;
+import org.apache.ignite.spi.systemview.view.datastructures.SetView;
 
 import static org.apache.ignite.codegen.MessageCodeGenerator.DFLT_SRC_DIR;
 import static org.apache.ignite.codegen.MessageCodeGenerator.INDEXING_SRC_DIR;
@@ -108,6 +117,15 @@ public class SystemViewRowAttributeWalkerGenerator {
         gen.generateAndWrite(PartitionStateView.class, DFLT_SRC_DIR);
         gen.generateAndWrite(BinaryMetadataView.class, DFLT_SRC_DIR);
         gen.generateAndWrite(MetastorageView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(QueueView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SetView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(AtomicLongView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(AtomicReferenceView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(AtomicSequenceView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(AtomicStampedView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(CountDownLatchView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(ReentrantLockView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SemaphoreView.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/systemview/walker/AtomicLongViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicLongViewWalker.java
new file mode 100644
index 0000000..793c2eb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicLongViewWalker.java
@@ -0,0 +1,52 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicLongView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link AtomicLongView} attributes walker.
+ * 
+ * @see AtomicLongView
+ */
+public class AtomicLongViewWalker implements SystemViewRowAttributeWalker<AtomicLongView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "value", long.class);
+        v.accept(2, "groupName", String.class);
+        v.accept(3, "groupId", int.class);
+        v.accept(4, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(AtomicLongView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.acceptLong(1, "value", row.value());
+        v.accept(2, "groupName", String.class, row.groupName());
+        v.acceptInt(3, "groupId", row.groupId());
+        v.acceptBoolean(4, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 5;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicReferenceViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicReferenceViewWalker.java
new file mode 100644
index 0000000..aa5ce55
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicReferenceViewWalker.java
@@ -0,0 +1,52 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicReferenceView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link AtomicReferenceView} attributes walker.
+ * 
+ * @see AtomicReferenceView
+ */
+public class AtomicReferenceViewWalker implements SystemViewRowAttributeWalker<AtomicReferenceView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "value", String.class);
+        v.accept(2, "groupName", String.class);
+        v.accept(3, "groupId", int.class);
+        v.accept(4, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(AtomicReferenceView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.accept(1, "value", String.class, row.value());
+        v.accept(2, "groupName", String.class, row.groupName());
+        v.acceptInt(3, "groupId", row.groupId());
+        v.acceptBoolean(4, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 5;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicSequenceViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicSequenceViewWalker.java
new file mode 100644
index 0000000..33f159e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicSequenceViewWalker.java
@@ -0,0 +1,54 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicSequenceView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link AtomicSequenceView} attributes walker.
+ * 
+ * @see AtomicSequenceView
+ */
+public class AtomicSequenceViewWalker implements SystemViewRowAttributeWalker<AtomicSequenceView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "value", long.class);
+        v.accept(2, "batchSize", long.class);
+        v.accept(3, "groupName", String.class);
+        v.accept(4, "groupId", int.class);
+        v.accept(5, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(AtomicSequenceView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.acceptLong(1, "value", row.value());
+        v.acceptLong(2, "batchSize", row.batchSize());
+        v.accept(3, "groupName", String.class, row.groupName());
+        v.acceptInt(4, "groupId", row.groupId());
+        v.acceptBoolean(5, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 6;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicStampedViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicStampedViewWalker.java
new file mode 100644
index 0000000..00132b2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/AtomicStampedViewWalker.java
@@ -0,0 +1,54 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicStampedView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link AtomicStampedView} attributes walker.
+ * 
+ * @see AtomicStampedView
+ */
+public class AtomicStampedViewWalker implements SystemViewRowAttributeWalker<AtomicStampedView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "value", String.class);
+        v.accept(2, "stamp", String.class);
+        v.accept(3, "groupName", String.class);
+        v.accept(4, "groupId", int.class);
+        v.accept(5, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(AtomicStampedView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.accept(1, "value", String.class, row.value());
+        v.accept(2, "stamp", String.class, row.stamp());
+        v.accept(3, "groupName", String.class, row.groupName());
+        v.acceptInt(4, "groupId", row.groupId());
+        v.acceptBoolean(5, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 6;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/CountDownLatchViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/CountDownLatchViewWalker.java
new file mode 100644
index 0000000..41af721
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/CountDownLatchViewWalker.java
@@ -0,0 +1,56 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.CountDownLatchView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link CountDownLatchView} attributes walker.
+ * 
+ * @see CountDownLatchView
+ */
+public class CountDownLatchViewWalker implements SystemViewRowAttributeWalker<CountDownLatchView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "count", int.class);
+        v.accept(2, "initialCount", int.class);
+        v.accept(3, "autoDelete", boolean.class);
+        v.accept(4, "groupName", String.class);
+        v.accept(5, "groupId", int.class);
+        v.accept(6, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(CountDownLatchView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.acceptInt(1, "count", row.count());
+        v.acceptInt(2, "initialCount", row.initialCount());
+        v.acceptBoolean(3, "autoDelete", row.autoDelete());
+        v.accept(4, "groupName", String.class, row.groupName());
+        v.acceptInt(5, "groupId", row.groupId());
+        v.acceptBoolean(6, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 7;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/QueueViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/QueueViewWalker.java
new file mode 100644
index 0000000..521e75c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/QueueViewWalker.java
@@ -0,0 +1,61 @@
+/*
+ * 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.lang.IgniteUuid;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.QueueView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link QueueView} attributes walker.
+ * 
+ * @see QueueView
+ */
+public class QueueViewWalker implements SystemViewRowAttributeWalker<QueueView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "id", IgniteUuid.class);
+        v.accept(1, "name", String.class);
+        v.accept(2, "capacity", int.class);
+        v.accept(3, "size", int.class);
+        v.accept(4, "groupName", String.class);
+        v.accept(5, "groupId", int.class);
+        v.accept(6, "bounded", boolean.class);
+        v.accept(7, "collocated", boolean.class);
+        v.accept(8, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(QueueView row, AttributeWithValueVisitor v) {
+        v.accept(0, "id", IgniteUuid.class, row.id());
+        v.accept(1, "name", String.class, row.name());
+        v.acceptInt(2, "capacity", row.capacity());
+        v.acceptInt(3, "size", row.size());
+        v.accept(4, "groupName", String.class, row.groupName());
+        v.acceptInt(5, "groupId", row.groupId());
+        v.acceptBoolean(6, "bounded", row.bounded());
+        v.acceptBoolean(7, "collocated", row.collocated());
+        v.acceptBoolean(8, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 9;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/ReentrantLockViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/ReentrantLockViewWalker.java
new file mode 100644
index 0000000..34257f3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/ReentrantLockViewWalker.java
@@ -0,0 +1,60 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.ReentrantLockView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link ReentrantLockView} attributes walker.
+ * 
+ * @see ReentrantLockView
+ */
+public class ReentrantLockViewWalker implements SystemViewRowAttributeWalker<ReentrantLockView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "locked", boolean.class);
+        v.accept(2, "hasQueuedThreads", boolean.class);
+        v.accept(3, "failoverSafe", boolean.class);
+        v.accept(4, "fair", boolean.class);
+        v.accept(5, "broken", boolean.class);
+        v.accept(6, "groupName", String.class);
+        v.accept(7, "groupId", int.class);
+        v.accept(8, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(ReentrantLockView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.acceptBoolean(1, "locked", row.locked());
+        v.acceptBoolean(2, "hasQueuedThreads", row.hasQueuedThreads());
+        v.acceptBoolean(3, "failoverSafe", row.failoverSafe());
+        v.acceptBoolean(4, "fair", row.fair());
+        v.acceptBoolean(5, "broken", row.broken());
+        v.accept(6, "groupName", String.class, row.groupName());
+        v.acceptInt(7, "groupId", row.groupId());
+        v.acceptBoolean(8, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 9;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SemaphoreViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SemaphoreViewWalker.java
new file mode 100644
index 0000000..7260804
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SemaphoreViewWalker.java
@@ -0,0 +1,60 @@
+/*
+ * 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.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.SemaphoreView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link SemaphoreView} attributes walker.
+ * 
+ * @see SemaphoreView
+ */
+public class SemaphoreViewWalker implements SystemViewRowAttributeWalker<SemaphoreView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "name", String.class);
+        v.accept(1, "availablePermits", long.class);
+        v.accept(2, "hasQueuedThreads", boolean.class);
+        v.accept(3, "queueLength", int.class);
+        v.accept(4, "failoverSafe", boolean.class);
+        v.accept(5, "broken", boolean.class);
+        v.accept(6, "groupName", String.class);
+        v.accept(7, "groupId", int.class);
+        v.accept(8, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(SemaphoreView row, AttributeWithValueVisitor v) {
+        v.accept(0, "name", String.class, row.name());
+        v.acceptLong(1, "availablePermits", row.availablePermits());
+        v.acceptBoolean(2, "hasQueuedThreads", row.hasQueuedThreads());
+        v.acceptInt(3, "queueLength", row.queueLength());
+        v.acceptBoolean(4, "failoverSafe", row.failoverSafe());
+        v.acceptBoolean(5, "broken", row.broken());
+        v.accept(6, "groupName", String.class, row.groupName());
+        v.acceptInt(7, "groupId", row.groupId());
+        v.acceptBoolean(8, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 9;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SetViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SetViewWalker.java
new file mode 100644
index 0000000..402bcd4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SetViewWalker.java
@@ -0,0 +1,57 @@
+/*
+ * 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.lang.IgniteUuid;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.datastructures.SetView;
+
+/**
+ * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
+ * {@link SetView} attributes walker.
+ * 
+ * @see SetView
+ */
+public class SetViewWalker implements SystemViewRowAttributeWalker<SetView> {
+    /** {@inheritDoc} */
+    @Override public void visitAll(AttributeVisitor v) {
+        v.accept(0, "id", IgniteUuid.class);
+        v.accept(1, "name", String.class);
+        v.accept(2, "size", int.class);
+        v.accept(3, "groupName", String.class);
+        v.accept(4, "groupId", int.class);
+        v.accept(5, "collocated", boolean.class);
+        v.accept(6, "removed", boolean.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void visitAll(SetView row, AttributeWithValueVisitor v) {
+        v.accept(0, "id", IgniteUuid.class, row.id());
+        v.accept(1, "name", String.class, row.name());
+        v.acceptInt(2, "size", row.size());
+        v.accept(3, "groupName", String.class, row.groupName());
+        v.acceptInt(4, "groupId", row.groupId());
+        v.acceptBoolean(5, "collocated", row.collocated());
+        v.acceptBoolean(6, "removed", row.removed());
+    }
+
+    /** {@inheritDoc} */
+    @Override public int count() {
+        return 7;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index 48b9292..d79d881 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -135,6 +135,16 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
         setsMap = new ConcurrentHashMap<>(10);
     }
 
+    /** @return Sets map. */
+    public ConcurrentMap<IgniteUuid, GridCacheSetProxy> sets() {
+        return setsMap;
+    }
+
+    /** @return Queue map. */
+    public ConcurrentMap<IgniteUuid, GridCacheQueueProxy> queues() {
+        return queuesMap;
+    }
+
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
         try {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 16f7b91..8731e6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -56,6 +56,15 @@ import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.managers.systemview.walker.AtomicLongViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.AtomicReferenceViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.AtomicSequenceViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.AtomicStampedViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.CountDownLatchViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.QueueViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.ReentrantLockViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SemaphoreViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SetViewWalker;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.CacheType;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
@@ -71,9 +80,12 @@ import org.apache.ignite.internal.util.lang.IgniteClosureX;
 import org.apache.ignite.internal.util.lang.IgniteInClosureX;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
 import org.apache.ignite.internal.util.lang.IgnitePredicateX;
+import org.apache.ignite.internal.util.lang.gridfunc.PredicateCollectionView;
+import org.apache.ignite.internal.util.lang.gridfunc.TransformCollectionView;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.CX1;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -82,6 +94,15 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicLongView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicReferenceView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicSequenceView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicStampedView;
+import org.apache.ignite.spi.systemview.view.datastructures.CountDownLatchView;
+import org.apache.ignite.spi.systemview.view.datastructures.QueueView;
+import org.apache.ignite.spi.systemview.view.datastructures.ReentrantLockView;
+import org.apache.ignite.spi.systemview.view.datastructures.SemaphoreView;
+import org.apache.ignite.spi.systemview.view.datastructures.SetView;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
@@ -99,6 +120,7 @@ import static org.apache.ignite.internal.processors.datastructures.DataStructure
 import static org.apache.ignite.internal.processors.datastructures.DataStructureType.REENTRANT_LOCK;
 import static org.apache.ignite.internal.processors.datastructures.DataStructureType.SEMAPHORE;
 import static org.apache.ignite.internal.processors.datastructures.DataStructureType.SET;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
@@ -113,7 +135,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
     public static final String DEFAULT_VOLATILE_DS_GROUP_NAME = "default-volatile-ds-group";
 
     /** */
-    private static final String DEFAULT_DS_GROUP_NAME = "default-ds-group";
+    public static final String DEFAULT_DS_GROUP_NAME = "default-ds-group";
 
     /** */
     private static final String DS_CACHE_NAME_PREFIX = "datastructures_";
@@ -121,6 +143,60 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
     /** Atomics system cache name. */
     public static final String ATOMICS_CACHE_NAME = "ignite-sys-atomic-cache";
 
+    /** */
+    public static final String QUEUES_VIEW = metricName("ds", "queues");
+
+    /** */
+    public static final String SETS_VIEW = metricName("ds", "sets");
+
+    /** */
+    public static final String LOCKS_VIEW = metricName("ds", "reentrantlocks");
+
+    /** */
+    public static final String SEMAPHORES_VIEW = metricName("ds", "semaphores");
+
+    /** */
+    public static final String LATCHES_VIEW = metricName("ds", "countdownlatches");
+
+    /** */
+    public static final String STAMPED_VIEW = metricName("ds", "atomicstamped");
+
+    /** */
+    public static final String REFERENCES_VIEW = metricName("ds", "atomicreferences");
+
+    /** */
+    public static final String LONGS_VIEW = metricName("ds", "atomiclongs");
+
+    /** */
+    public static final String SEQUENCES_VIEW = metricName("ds", "atomicsequences");
+
+    /** */
+    private static final String QUEUES_VIEW_DESC = "Data structure queues";
+
+    /** */
+    private static final String SETS_VIEW_DESC = "Data structure sets";
+
+    /** */
+    private static final String LOCKS_VIEW_DESC = "Data structure reentrant locks";
+
+    /** */
+    private static final String SEMAPHORES_VIEW_DESC = "Data structure semaphores";
+
+    /** */
+    private static final String LATCHES_VIEW_DESC = "Data structure count down latches";
+
+    /** */
+    private static final String STAMPED_VIEW_DESC = "Data structure atomic stamped";
+
+    /** */
+    private static final String REFERENCES_VIEW_DESC = "Data structure atomic references";
+
+    /** */
+    private static final String LONGS_VIEW_DESC = "Data structure atomic longs";
+
+    /** */
+    private static final String SEQUENCES_VIEW_DESC = "Data structure atomic sequences";
+
     /** Non collocated IgniteSet will use separate cache if all nodes in cluster is not older then specified version. */
     private static final IgniteProductVersion SEPARATE_CACHE_PER_NON_COLLOCATED_SET_SINCE =
         IgniteProductVersion.fromString("2.7.0");
@@ -190,6 +266,86 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
         if (ctx.config().isDaemon() || !active)
             return;
 
+        ctx.systemView().registerView(
+            SEQUENCES_VIEW,
+            SEQUENCES_VIEW_DESC,
+            new AtomicSequenceViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteAtomicSequence),
+            AtomicSequenceView::new
+        );
+
+        ctx.systemView().registerView(
+            LONGS_VIEW,
+            LONGS_VIEW_DESC,
+            new AtomicLongViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteAtomicLong),
+            AtomicLongView::new
+        );
+
+        ctx.systemView().registerView(
+            REFERENCES_VIEW,
+            REFERENCES_VIEW_DESC,
+            new AtomicReferenceViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteAtomicReference),
+            AtomicReferenceView::new
+        );
+
+        ctx.systemView().registerView(
+            STAMPED_VIEW,
+            STAMPED_VIEW_DESC,
+            new AtomicStampedViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteAtomicStamped),
+            AtomicStampedView::new
+        );
+
+        ctx.systemView().registerView(
+            LATCHES_VIEW,
+            LATCHES_VIEW_DESC,
+            new CountDownLatchViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteCountDownLatch),
+            CountDownLatchView::new
+        );
+
+        ctx.systemView().registerView(
+            SEMAPHORES_VIEW,
+            SEMAPHORES_VIEW_DESC,
+            new SemaphoreViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteSemaphore),
+            SemaphoreView::new
+        );
+
+        ctx.systemView().registerView(
+            LOCKS_VIEW,
+            LOCKS_VIEW_DESC,
+            new ReentrantLockViewWalker(),
+            new PredicateCollectionView<>(dsMap.values(), v -> v instanceof IgniteLock),
+            ReentrantLockView::new
+        );
+
+        ctx.systemView().registerInnerCollectionView(
+            QUEUES_VIEW,
+            QUEUES_VIEW_DESC,
+            new QueueViewWalker(),
+            new TransformCollectionView<>(
+                ctx.cache().cacheDescriptors().values(),
+                desc -> ctx.cache().cache(desc.cacheName()).context().dataStructures(),
+                desc -> desc.cacheType() == CacheType.DATA_STRUCTURES),
+            cctx -> cctx.queues().values(),
+            (cctx, queue) -> new QueueView(queue)
+        );
+
+        ctx.systemView().registerInnerCollectionView(
+            SETS_VIEW,
+            SETS_VIEW_DESC,
+            new SetViewWalker(),
+            F.viewReadOnly(
+                ctx.cache().cacheDescriptors().values(),
+                desc -> ctx.cache().cache(desc.cacheName()).context().dataStructures(),
+                desc -> desc.cacheType() == CacheType.DATA_STRUCTURES),
+            cctx -> cctx.sets().values(),
+            (cctx, set) -> new SetView(set)
+        );
+
         onKernalStart0();
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueProxy.java
index 7a1b92b..a714243 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueProxy.java
@@ -416,6 +416,16 @@ public class GridCacheQueueProxy<T> implements IgniteQueue<T>, Externalizable {
         return delegate.name();
     }
 
+    /** @return Group name for queue. */
+    public String groupName() {
+        return cctx.group().name();
+    }
+
+    /** @return Group id for queue. */
+    public int groupId() {
+        return cctx.group().groupId();
+    }
+
     /** {@inheritDoc} */
     @Override public int capacity() {
         return delegate.capacity();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
index 541ca30..a1cb56b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
@@ -385,6 +385,16 @@ public class GridCacheSetProxy<T> implements IgniteSet<T>, Externalizable {
         return delegate.name();
     }
 
+    /** @return Group name for queue. */
+    public String groupName() {
+        return cctx.group().name();
+    }
+
+    /** @return Group id for queue. */
+    public int groupId() {
+        return cctx.group().groupId();
+    }
+
     /** {@inheritDoc} */
     @Override public boolean collocated() {
         return delegate.collocated();
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AbstractDataStructureView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AbstractDataStructureView.java
new file mode 100644
index 0000000..252d087
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AbstractDataStructureView.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.datastructures;
+
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.AtomicDataStructureProxy;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/** Abstract class for a {@link SystemView} representation of data structures. */
+abstract class AbstractDataStructureView<T extends AtomicDataStructureProxy> {
+    /** Data structure instance. */
+    protected final T ds;
+
+    /** @param ds Data structure instance. */
+    AbstractDataStructureView(T ds) {
+        this.ds = ds;
+    }
+
+    /** @return Name. */
+    @Order
+    public String name() {
+        return ds.name();
+    }
+
+    /** @return Group name. */
+    @Order(10)
+    public String groupName() {
+        return ds.key().groupName();
+    }
+
+    /** @return Group id. */
+    @Order(11)
+    public int groupId() {
+        return CU.cacheId(groupName());
+    }
+
+    /** @return {@code True} is data structure removed. */
+    @Order(12)
+    public boolean removed() {
+        return ds.removed();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicLongView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicLongView.java
new file mode 100644
index 0000000..61af3af
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicLongView.java
@@ -0,0 +1,48 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteAtomicLong;
+import org.apache.ignite.configuration.AtomicConfiguration;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * {@link IgniteAtomicLong} representation for a {@link SystemView}.
+ *
+ * @see Ignite#atomicLong(String, long, boolean)
+ * @see Ignite#atomicLong(String, AtomicConfiguration, long, boolean)
+ */
+public class AtomicLongView extends AbstractDataStructureView<GridCacheAtomicLongImpl> {
+    /** @param ds Data structure instance. */
+    public AtomicLongView(GridCacheRemovable ds) {
+        super((GridCacheAtomicLongImpl)ds);
+    }
+
+    /**
+     * @return Value.
+     * @see IgniteAtomicLong#get()
+     */
+    @Order(1)
+    public long value() {
+        return ds.get();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicReferenceView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicReferenceView.java
new file mode 100644
index 0000000..5c100c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicReferenceView.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.spi.systemview.view.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteAtomicReference;
+import org.apache.ignite.configuration.AtomicConfiguration;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicReferenceImpl;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+import static org.apache.ignite.internal.util.IgniteUtils.toStringSafe;
+
+/**
+ * {@link IgniteAtomicReference} representation for a {@link SystemView}.
+ *
+ * @see Ignite#atomicReference(String, Object, boolean)
+ * @see Ignite#atomicReference(String, AtomicConfiguration, Object, boolean)
+ */
+public class AtomicReferenceView extends AbstractDataStructureView<GridCacheAtomicReferenceImpl> {
+    /** @param ds Data structure instance. */
+    public AtomicReferenceView(GridCacheRemovable ds) {
+        super((GridCacheAtomicReferenceImpl)ds);
+    }
+
+    /**
+     * @return Value.
+     * @see IgniteAtomicReference#get()
+     */
+    @Order(1)
+    public String value() {
+        return toStringSafe(ds.get());
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicSequenceView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicSequenceView.java
new file mode 100644
index 0000000..38a1de8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicSequenceView.java
@@ -0,0 +1,57 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteAtomicSequence;
+import org.apache.ignite.configuration.AtomicConfiguration;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicSequenceImpl;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * {@link IgniteAtomicSequence} representation for a {@link SystemView}.
+ *
+ * @see Ignite#atomicSequence(String, long, boolean)
+ * @see Ignite#atomicSequence(String, AtomicConfiguration, long, boolean)
+ */
+public class AtomicSequenceView extends AbstractDataStructureView<GridCacheAtomicSequenceImpl> {
+    /** @param ds Data structure instance. */
+    public AtomicSequenceView(GridCacheRemovable ds) {
+        super((GridCacheAtomicSequenceImpl)ds);
+    }
+
+    /**
+     * @return Value.
+     * @see IgniteAtomicSequence#get()
+     */
+    @Order(1)
+    public long value() {
+        return ds.get();
+    }
+
+    /**
+     * @return Batch size.
+     * @see IgniteAtomicSequence#batchSize()
+     */
+    @Order(2)
+    public long batchSize() {
+        return ds.batchSize();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicStampedView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicStampedView.java
new file mode 100644
index 0000000..3586e0a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/AtomicStampedView.java
@@ -0,0 +1,59 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteAtomicStamped;
+import org.apache.ignite.configuration.AtomicConfiguration;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicStampedImpl;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+import static org.apache.ignite.internal.util.IgniteUtils.toStringSafe;
+
+/**
+ * {@link IgniteAtomicStamped} representation for a {@link SystemView}.
+ *
+ * @see Ignite#atomicStamped(String, Object, Object, boolean)
+ * @see Ignite#atomicStamped(String, AtomicConfiguration, Object, Object, boolean)
+ */
+public class AtomicStampedView extends AbstractDataStructureView<GridCacheAtomicStampedImpl> {
+    /** @param ds Data structure instance. */
+    public AtomicStampedView(GridCacheRemovable ds) {
+        super((GridCacheAtomicStampedImpl)ds);
+    }
+
+    /**
+     * @return Value.
+     * @see IgniteAtomicStamped#get()
+     */
+    @Order(1)
+    public String value() {
+        return toStringSafe(ds.value());
+    }
+
+    /**
+     * @return Stamp.
+     * @see IgniteAtomicStamped#stamp()
+     */
+    @Order(2)
+    public String stamp() {
+        return toStringSafe(ds.stamp());
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/CountDownLatchView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/CountDownLatchView.java
new file mode 100644
index 0000000..623a81c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/CountDownLatchView.java
@@ -0,0 +1,64 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCountDownLatch;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheCountDownLatchImpl;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * {@link IgniteCountDownLatch} representation for a {@link SystemView}.
+ *
+ * @see Ignite#countDownLatch(String, int, boolean, boolean)
+ */
+public class CountDownLatchView extends AbstractDataStructureView<GridCacheCountDownLatchImpl> {
+    /** @param ds Data structure instance. */
+    public CountDownLatchView(GridCacheRemovable ds) {
+        super((GridCacheCountDownLatchImpl)ds);
+    }
+
+    /**
+     * @return Count.
+     * @see IgniteCountDownLatch#count()
+     */
+    @Order(1)
+    public int count() {
+        return ds.count();
+    }
+
+    /**
+     * @return Initial count.
+     * @see IgniteCountDownLatch#initialCount()
+     */
+    @Order(2)
+    public int initialCount() {
+        return ds.initialCount();
+    }
+
+    /**
+     * @return {@code True} if latch is auto removed after counter down to 0.
+     * @see IgniteCountDownLatch#autoDelete()
+     */
+    @Order(3)
+    public boolean autoDelete() {
+        return ds.autoDelete();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/QueueView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/QueueView.java
new file mode 100644
index 0000000..2a97ba8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/QueueView.java
@@ -0,0 +1,89 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheQueueProxy;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * Queue representation for a {@link SystemView}.
+ */
+public class QueueView {
+    /** Queue. */
+    private final GridCacheQueueProxy<?> queue;
+
+    /**
+     * @param queue Queue to view.
+     */
+    public QueueView(GridCacheQueueProxy<?> queue) {
+        this.queue = queue;
+    }
+
+    /** @return Queue id. */
+    @Order
+    public IgniteUuid id() {
+        return queue.delegate().id();
+    }
+
+    /** @return Queue name. */
+    @Order(1)
+    public String name() {
+        return queue.name();
+    }
+
+    /** @return Queue capacity. */
+    @Order(2)
+    public int capacity() {
+        return queue.capacity();
+    }
+
+    /** @return Queue size. */
+    @Order(3)
+    public int size() {
+        return queue.size();
+    }
+
+    /** @return Cache group name where values for queue stored. */
+    @Order(4)
+    public String groupName() {
+        return queue.groupName();
+    }
+
+    /** @return Cache group id where values for queue stored. */
+    @Order(5)
+    public int groupId() {
+        return queue.groupId();
+    }
+
+    /** @return If {@code true} then queue capacity is bounded. */
+    public boolean bounded() {
+        return queue.bounded();
+    }
+
+    /** Collocated flag. */
+    public boolean collocated() {
+        return queue.collocated();
+    }
+
+    /** @return If {@code true} then this queue removed. */
+    public boolean removed() {
+        return queue.removed();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/ReentrantLockView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/ReentrantLockView.java
new file mode 100644
index 0000000..75e3573
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/ReentrantLockView.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.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteLock;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheLockImpl;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * {@link IgniteLock} representation for a {@link SystemView}.
+ *
+ * @see Ignite#reentrantLock(String, boolean, boolean, boolean)
+ */
+public class ReentrantLockView extends AbstractDataStructureView<GridCacheLockImpl> {
+    /** @param ds Data structure instance. */
+    public ReentrantLockView(GridCacheRemovable ds) {
+        super((GridCacheLockImpl)ds);
+    }
+
+    /**
+     * @return {@code True} if locked.
+     * @see IgniteLock#isLocked()
+     */
+    @Order(1)
+    public boolean locked() {
+        return ds.isLocked();
+    }
+
+    /**
+     * @return {@code True} if there may be other threads waiting to acquire the lock.
+     * @see IgniteLock#hasQueuedThreads()
+     */
+    @Order(2)
+    public boolean hasQueuedThreads() {
+        return ds.hasQueuedThreads();
+    }
+
+    /**
+     * @return {@code True} if this semaphore is failover safe.
+     * @see IgniteLock#isFailoverSafe()
+     */
+    @Order(3)
+    public boolean failoverSafe() {
+        return ds.isFailoverSafe();
+    }
+
+    /**
+     * @return {@code True} if this lock is fair.
+     * @see IgniteLock#isFair()
+     */
+    @Order(4)
+    public boolean fair() {
+        return ds.isFair();
+    }
+
+    /**
+     * @return {@code True} if a node failed on this semaphore and {@link #failoverSafe} flag was set to {@code false}, {@code false} otherwise.
+     * @see IgniteLock#isBroken()
+     */
+    @Order(5)
+    public boolean broken() {
+        return ds.isBroken();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/SemaphoreView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/SemaphoreView.java
new file mode 100644
index 0000000..ce36674
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/SemaphoreView.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.datastructures;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteSemaphore;
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheRemovable;
+import org.apache.ignite.internal.processors.datastructures.GridCacheSemaphoreImpl;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * {@link IgniteSemaphore} representation for a {@link SystemView}.
+ *
+ * @see Ignite#semaphore(String, int, boolean, boolean)
+ */
+public class SemaphoreView extends AbstractDataStructureView<GridCacheSemaphoreImpl> {
+    /** @param ds Data structure instance. */
+    public SemaphoreView(GridCacheRemovable ds) {
+        super((GridCacheSemaphoreImpl)ds);
+    }
+
+    /**
+     * @return Number of permits available.
+     * @see IgniteSemaphore#availablePermits()
+     */
+    @Order(1)
+    public long availablePermits() {
+        return ds.availablePermits();
+    }
+
+    /**
+     * @return {@code True} if there may be other threads waiting to acquire the lock.
+     * @see IgniteSemaphore#hasQueuedThreads()
+     */
+    @Order(2)
+    public boolean hasQueuedThreads() {
+        return ds.hasQueuedThreads();
+    }
+
+    /**
+     * @return The estimated number of nodes waiting for this lock.
+     * @see IgniteSemaphore#getQueueLength()
+     */
+    @Order(3)
+    public int queueLength() {
+        return ds.getQueueLength();
+    }
+
+    /**
+     * @return {@code True} if this semaphore is failover safe.
+     * @see IgniteSemaphore#isFailoverSafe()
+     */
+    @Order(4)
+    public boolean failoverSafe() {
+        return ds.isFailoverSafe();
+    }
+
+    /**
+     * @return {@code True} if a node failed on this semaphore and {@link #failoverSafe} flag was set to {@code false}, {@code false} otherwise.
+     * @see IgniteSemaphore#isBroken()
+     */
+    @Order(5)
+    public boolean broken() {
+        return ds.isBroken();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/SetView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/SetView.java
new file mode 100644
index 0000000..48414a8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/datastructures/SetView.java
@@ -0,0 +1,78 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.systemview.view.SystemView;
+
+/**
+ * Set representation for a {@link SystemView}.
+ */
+public class SetView {
+    /** Set. */
+    private final GridCacheSetProxy<?> set;
+
+    /**
+     * @param set Set to view.
+     */
+    public SetView(GridCacheSetProxy<?> set) {
+        this.set = set;
+    }
+
+    /** @return Set id. */
+    @Order
+    public IgniteUuid id() {
+        return set.delegate().id();
+    }
+
+    /** @return Set name. */
+    @Order(1)
+    public String name() {
+        return set.name();
+    }
+
+    /** @return Set size. */
+    @Order(2)
+    public int size() {
+        return set.size();
+    }
+
+    /** @return Cache group name where values for set stored. */
+    @Order(3)
+    public String groupName() {
+        return set.groupName();
+    }
+
+    /** @return Cache group id where values for set stored. */
+    @Order(4)
+    public int groupId() {
+        return set.groupId();
+    }
+
+    /** @return Collocated flag. */
+    public boolean collocated() {
+        return set.collocated();
+    }
+
+    /** @return If {@code true} then this set removed. */
+    public boolean removed() {
+        return set.removed();
+    }
+}
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 4b35f50..4d05258 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
@@ -35,10 +35,19 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import javax.cache.Cache;
+import org.apache.ignite.IgniteAtomicLong;
+import org.apache.ignite.IgniteAtomicReference;
+import org.apache.ignite.IgniteAtomicSequence;
+import org.apache.ignite.IgniteAtomicStamped;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteCountDownLatch;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteJdbcThinDriver;
+import org.apache.ignite.IgniteLock;
+import org.apache.ignite.IgniteQueue;
+import org.apache.ignite.IgniteSemaphore;
+import org.apache.ignite.IgniteSet;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
@@ -53,11 +62,14 @@ import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTask;
+import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.mutabletest.GridBinaryTestClasses.TestObjectAllTypes;
 import org.apache.ignite.internal.binary.mutabletest.GridBinaryTestClasses.TestObjectEnum;
 import org.apache.ignite.internal.client.thin.ProtocolVersion;
@@ -69,6 +81,7 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
 import org.apache.ignite.internal.processors.service.DummyService;
 import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteCallable;
@@ -92,6 +105,15 @@ import org.apache.ignite.spi.systemview.view.ServiceView;
 import org.apache.ignite.spi.systemview.view.StripedExecutorTaskView;
 import org.apache.ignite.spi.systemview.view.SystemView;
 import org.apache.ignite.spi.systemview.view.TransactionView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicLongView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicReferenceView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicSequenceView;
+import org.apache.ignite.spi.systemview.view.datastructures.AtomicStampedView;
+import org.apache.ignite.spi.systemview.view.datastructures.CountDownLatchView;
+import org.apache.ignite.spi.systemview.view.datastructures.QueueView;
+import org.apache.ignite.spi.systemview.view.datastructures.ReentrantLockView;
+import org.apache.ignite.spi.systemview.view.datastructures.SemaphoreView;
+import org.apache.ignite.spi.systemview.view.datastructures.SetView;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
@@ -99,6 +121,7 @@ import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 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.systemview.GridSystemViewManager.STREAM_POOL_QUEUE_VIEW;
 import static org.apache.ignite.internal.managers.systemview.GridSystemViewManager.SYS_POOL_QUEUE_VIEW;
@@ -113,6 +136,18 @@ import static org.apache.ignite.internal.processors.cache.persistence.GridCacheD
 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.continuous.GridContinuousProcessor.CQ_SYS_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.DEFAULT_DS_GROUP_NAME;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.DEFAULT_VOLATILE_DS_GROUP_NAME;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.LATCHES_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.LOCKS_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.LONGS_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.QUEUES_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.REFERENCES_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.SEMAPHORES_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.SEQUENCES_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.SETS_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.STAMPED_VIEW;
+import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.VOLATILE_DATA_REGION_NAME;
 import static org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageImpl.DISTRIBUTED_METASTORE_VIEW;
 import static org.apache.ignite.internal.processors.odbc.ClientListenerProcessor.CLI_CONN_VIEW;
 import static org.apache.ignite.internal.processors.service.IgniteServiceProcessor.SVCS_VIEW;
@@ -120,6 +155,7 @@ import static org.apache.ignite.internal.processors.task.GridTaskProcessor.TASKS
 import static org.apache.ignite.internal.util.IgniteUtils.toStringSafe;
 import static org.apache.ignite.internal.util.lang.GridFunc.alwaysTrue;
 import static org.apache.ignite.internal.util.lang.GridFunc.identity;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -911,6 +947,647 @@ public class SystemViewSelfTest extends GridCommonAbstractTest {
 
     /** */
     @Test
+    public void testAtomicSequence() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteAtomicSequence s1 = g0.atomicSequence("seq-1", 42, true);
+            IgniteAtomicSequence s2 = g0.atomicSequence("seq-2",
+                new AtomicConfiguration().setBackups(1).setGroupName("my-group"), 43, true);
+
+            s1.batchSize(42);
+
+            SystemView<AtomicSequenceView> seqs0 = g0.context().systemView().view(SEQUENCES_VIEW);
+            SystemView<AtomicSequenceView> seqs1 = g1.context().systemView().view(SEQUENCES_VIEW);
+
+            assertEquals(2, seqs0.size());
+            assertEquals(0, seqs1.size());
+
+            for (AtomicSequenceView s : seqs0) {
+                if ("seq-1".equals(s.name())) {
+                    assertEquals(42, s.value());
+                    assertEquals(42, s.batchSize());
+                    assertEquals(DEFAULT_DS_GROUP_NAME, s.groupName());
+                    assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), s.groupId());
+
+                    long val = s1.addAndGet(42);
+
+                    assertEquals(val, s.value());
+                    assertFalse(s.removed());
+                }
+                else {
+                    assertEquals("seq-2", s.name());
+                    assertEquals(DFLT_ATOMIC_SEQUENCE_RESERVE_SIZE, s.batchSize());
+                    assertEquals(43, s.value());
+                    assertEquals("my-group", s.groupName());
+                    assertEquals(CU.cacheId("my-group"), s.groupId());
+                    assertFalse(s.removed());
+
+                    s2.close();
+
+                    assertTrue(s.removed());
+                }
+            }
+
+            g1.atomicSequence("seq-1", 42, true);
+
+            assertEquals(1, seqs1.size());
+
+            AtomicSequenceView s = seqs1.iterator().next();
+
+            assertEquals(DFLT_ATOMIC_SEQUENCE_RESERVE_SIZE + 42, s.value());
+            assertEquals(DFLT_ATOMIC_SEQUENCE_RESERVE_SIZE, s.batchSize());
+            assertEquals(DEFAULT_DS_GROUP_NAME, s.groupName());
+            assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), s.groupId());
+            assertFalse(s.removed());
+
+            s1.close();
+
+            assertTrue(s.removed());
+
+            assertEquals(0, seqs0.size());
+            assertEquals(0, seqs1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testAtomicLongs() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteAtomicLong l1 = g0.atomicLong("long-1", 42, true);
+            IgniteAtomicLong l2 = g0.atomicLong("long-2",
+                new AtomicConfiguration().setBackups(1).setGroupName("my-group"), 43, true);
+
+            SystemView<AtomicLongView> longs0 = g0.context().systemView().view(LONGS_VIEW);
+            SystemView<AtomicLongView> longs1 = g1.context().systemView().view(LONGS_VIEW);
+
+            assertEquals(2, longs0.size());
+            assertEquals(0, longs1.size());
+
+            for (AtomicLongView l : longs0) {
+                if ("long-1".equals(l.name())) {
+                    assertEquals(42, l.value());
+                    assertEquals(DEFAULT_DS_GROUP_NAME, l.groupName());
+                    assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), l.groupId());
+
+                    long val = l1.addAndGet(42);
+
+                    assertEquals(val, l.value());
+                    assertFalse(l.removed());
+                }
+                else {
+                    assertEquals("long-2", l.name());
+                    assertEquals(43, l.value());
+                    assertEquals("my-group", l.groupName());
+                    assertEquals(CU.cacheId("my-group"), l.groupId());
+                    assertFalse(l.removed());
+
+                    l2.close();
+
+                    assertTrue(l.removed());
+                }
+            }
+
+            g1.atomicLong("long-1", 42, true);
+
+            assertEquals(1, longs1.size());
+
+            AtomicLongView l = longs1.iterator().next();
+
+            assertEquals(84, l.value());
+            assertEquals(DEFAULT_DS_GROUP_NAME, l.groupName());
+            assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), l.groupId());
+            assertFalse(l.removed());
+
+            l1.close();
+
+            assertTrue(l.removed());
+
+            assertEquals(0, longs0.size());
+            assertEquals(0, longs1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testAtomicReference() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteAtomicReference<String> l1 = g0.atomicReference("ref-1", "str1", true);
+            IgniteAtomicReference<Integer> l2 = g0.atomicReference("ref-2",
+                new AtomicConfiguration().setBackups(1).setGroupName("my-group"), 43, true);
+
+            SystemView<AtomicReferenceView> refs0 = g0.context().systemView().view(REFERENCES_VIEW);
+            SystemView<AtomicReferenceView> refs1 = g1.context().systemView().view(REFERENCES_VIEW);
+
+            assertEquals(2, refs0.size());
+            assertEquals(0, refs1.size());
+
+            for (AtomicReferenceView r : refs0) {
+                if ("ref-1".equals(r.name())) {
+                    assertEquals("str1", r.value());
+                    assertEquals(DEFAULT_DS_GROUP_NAME, r.groupName());
+                    assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), r.groupId());
+
+                    l1.set("str2");
+
+                    assertEquals("str2", r.value());
+                    assertFalse(r.removed());
+                }
+                else {
+                    assertEquals("ref-2", r.name());
+                    assertEquals("43", r.value());
+                    assertEquals("my-group", r.groupName());
+                    assertEquals(CU.cacheId("my-group"), r.groupId());
+                    assertFalse(r.removed());
+
+                    l2.close();
+
+                    assertTrue(r.removed());
+                }
+            }
+
+            g1.atomicReference("ref-1", "str3", true);
+
+            assertEquals(1, refs1.size());
+
+            AtomicReferenceView l = refs1.iterator().next();
+
+            assertEquals("str2", l.value());
+            assertEquals(DEFAULT_DS_GROUP_NAME, l.groupName());
+            assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), l.groupId());
+            assertFalse(l.removed());
+
+            l1.close();
+
+            assertTrue(l.removed());
+
+            assertEquals(0, refs0.size());
+            assertEquals(0, refs1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testAtomicStamped() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteAtomicStamped<String, Integer> s1 = g0.atomicStamped("s-1", "str0", 1, true);
+            IgniteAtomicStamped<String, Integer> s2 = g0.atomicStamped("s-2",
+                new AtomicConfiguration().setBackups(1).setGroupName("my-group"), "str1", 43, true);
+
+            SystemView<AtomicStampedView> stamps0 = g0.context().systemView().view(STAMPED_VIEW);
+            SystemView<AtomicStampedView> stamps1 = g1.context().systemView().view(STAMPED_VIEW);
+
+            assertEquals(2, stamps0.size());
+            assertEquals(0, stamps1.size());
+
+            for (AtomicStampedView s : stamps0) {
+                if ("s-1".equals(s.name())) {
+                    assertEquals("str0", s.value());
+                    assertEquals("1", s.stamp());
+                    assertEquals(DEFAULT_DS_GROUP_NAME, s.groupName());
+                    assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), s.groupId());
+
+                    s1.set("str2", 2);
+
+                    assertEquals("str2", s.value());
+                    assertEquals("2", s.stamp());
+                    assertFalse(s.removed());
+                }
+                else {
+                    assertEquals("s-2", s.name());
+                    assertEquals("str1", s.value());
+                    assertEquals("43", s.stamp());
+                    assertEquals("my-group", s.groupName());
+                    assertEquals(CU.cacheId("my-group"), s.groupId());
+                    assertFalse(s.removed());
+
+                    s2.close();
+
+                    assertTrue(s.removed());
+                }
+            }
+
+            g1.atomicStamped("s-1", "str3", 3, true);
+
+            assertEquals(1, stamps1.size());
+
+            AtomicStampedView l = stamps1.iterator().next();
+
+            assertEquals("str2", l.value());
+            assertEquals("2", l.stamp());
+            assertEquals(DEFAULT_DS_GROUP_NAME, l.groupName());
+            assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), l.groupId());
+            assertFalse(l.removed());
+
+            s1.close();
+
+            assertTrue(l.removed());
+
+            assertEquals(0, stamps0.size());
+            assertEquals(0, stamps1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testCountDownLatch() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteCountDownLatch l1 = g0.countDownLatch("c1", 3, false, true);
+            IgniteCountDownLatch l2 = g0.countDownLatch("c2", 1, true, true);
+
+            SystemView<CountDownLatchView> latches0 = g0.context().systemView().view(LATCHES_VIEW);
+            SystemView<CountDownLatchView> latches1 = g1.context().systemView().view(LATCHES_VIEW);
+
+            assertEquals(2, latches0.size());
+            assertEquals(0, latches1.size());
+
+            String grpName = DEFAULT_VOLATILE_DS_GROUP_NAME + "@" + VOLATILE_DATA_REGION_NAME;
+
+            for (CountDownLatchView l : latches0) {
+                if ("c1".equals(l.name())) {
+                    assertEquals(3, l.count());
+                    assertEquals(3, l.initialCount());
+                    assertFalse(l.autoDelete());
+
+                    l1.countDown();
+
+                    assertEquals(2, l.count());
+                    assertEquals(3, l.initialCount());
+                    assertFalse(l.removed());
+                }
+                else {
+                    assertEquals("c2", l.name());
+                    assertEquals(1, l.count());
+                    assertEquals(1, l.initialCount());
+                    assertTrue(l.autoDelete());
+                    assertFalse(l.removed());
+
+                    l2.countDown();
+                    l2.close();
+
+                    assertTrue(l.removed());
+                }
+
+                assertEquals(grpName, l.groupName());
+                assertEquals(CU.cacheId(grpName), l.groupId());
+            }
+
+            IgniteCountDownLatch l3 = g1.countDownLatch("c1", 10, true, true);
+
+            assertEquals(1, latches1.size());
+
+            CountDownLatchView l = latches1.iterator().next();
+
+            assertEquals(2, l.count());
+            assertEquals(3, l.initialCount());
+            assertEquals(grpName, l.groupName());
+            assertEquals(CU.cacheId(grpName), l.groupId());
+            assertFalse(l.removed());
+            assertFalse(l.autoDelete());
+
+            l3.countDown();
+            l3.countDown();
+            l3.close();
+
+            assertTrue(l.removed());
+
+            assertEquals(0, latches0.size());
+            assertEquals(0, latches1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testSemaphores() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteSemaphore s1 = g0.semaphore("s1", 3, false, true);
+            IgniteSemaphore s2 = g0.semaphore("s2", 1, true, true);
+
+            SystemView<SemaphoreView> semaphores0 = g0.context().systemView().view(SEMAPHORES_VIEW);
+            SystemView<SemaphoreView> semaphores1 = g1.context().systemView().view(SEMAPHORES_VIEW);
+
+            assertEquals(2, semaphores0.size());
+            assertEquals(0, semaphores1.size());
+
+            String grpName = DEFAULT_VOLATILE_DS_GROUP_NAME + "@" + VOLATILE_DATA_REGION_NAME;
+
+            IgniteInternalFuture<?> acquirePermitFut = null;
+
+            for (SemaphoreView s : semaphores0) {
+                if ("s1".equals(s.name())) {
+                    assertEquals(3, s.availablePermits());
+                    assertFalse(s.hasQueuedThreads());
+                    assertEquals(0, s.queueLength());
+                    assertFalse(s.failoverSafe());
+                    assertFalse(s.broken());
+                    assertFalse(s.removed());
+
+                    acquirePermitFut = runAsync(() -> {
+                        s1.acquire(2);
+
+                        try {
+                            Thread.sleep(getTestTimeout());
+                        }
+                        catch (InterruptedException e) {
+                            throw new RuntimeException(e);
+                        }
+                        finally {
+                            s1.release(2);
+                        }
+                    });
+
+                    assertTrue(waitForCondition(() -> s.availablePermits() == 1, getTestTimeout()));
+                    assertTrue(s.hasQueuedThreads());
+                    assertEquals(1, s.queueLength());
+                }
+                else {
+                    assertEquals(1, s.availablePermits());
+                    assertFalse(s.hasQueuedThreads());
+                    assertEquals(0, s.queueLength());
+                    assertTrue(s.failoverSafe());
+                    assertFalse(s.broken());
+                    assertFalse(s.removed());
+
+                    s2.close();
+
+                    assertTrue(s.removed());
+                }
+
+                assertEquals(grpName, s.groupName());
+                assertEquals(CU.cacheId(grpName), s.groupId());
+            }
+
+            IgniteSemaphore l3 = g1.semaphore("s1", 10, true, true);
+
+            assertEquals(1, semaphores1.size());
+
+            SemaphoreView s = semaphores1.iterator().next();
+
+            assertEquals(1, s.availablePermits());
+            assertTrue(s.hasQueuedThreads());
+            assertEquals(1, s.queueLength());
+            assertFalse(s.failoverSafe());
+            assertFalse(s.broken());
+            assertFalse(s.removed());
+
+            acquirePermitFut.cancel();
+            assertTrue(waitForCondition(() -> s.availablePermits() == 3, getTestTimeout()));
+
+            l3.close();
+
+            assertTrue(s.removed());
+
+            assertEquals(0, semaphores0.size());
+            assertEquals(0, semaphores1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testLocks() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+            IgniteLock l1 = g0.reentrantLock("l1", false, true, true);
+            IgniteLock l2 = g0.reentrantLock("l2", true, false, true);
+
+            SystemView<ReentrantLockView> locks0 = g0.context().systemView().view(LOCKS_VIEW);
+            SystemView<ReentrantLockView> locks1 = g1.context().systemView().view(LOCKS_VIEW);
+
+            assertEquals(2, locks0.size());
+            assertEquals(0, locks1.size());
+
+            String grpName = DEFAULT_VOLATILE_DS_GROUP_NAME + "@" + VOLATILE_DATA_REGION_NAME;
+
+            IgniteInternalFuture<?> lockFut = null;
+
+            Runnable lockNSleep = () -> {
+                l1.lock();
+
+                try {
+                    Thread.sleep(getTestTimeout());
+                }
+                catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+                finally {
+                    l1.unlock();
+                }
+            };
+
+            for (ReentrantLockView l : locks0) {
+                if ("l1".equals(l.name())) {
+                    assertFalse(l.locked());
+                    assertFalse(l.hasQueuedThreads());
+                    assertFalse(l.failoverSafe());
+                    assertTrue(l.fair());
+                    assertFalse(l.broken());
+                    assertFalse(l.removed());
+
+                    lockFut = runAsync(lockNSleep);
+
+                    assertTrue(waitForCondition(l::locked, getTestTimeout()));
+                }
+                else {
+                    assertFalse(l.hasQueuedThreads());
+                    assertTrue(l.failoverSafe());
+                    assertFalse(l.fair());
+                    assertFalse(l.broken());
+                    assertFalse(l.removed());
+
+                    l2.close();
+
+                    assertTrue(l.removed());
+                }
+
+                assertEquals(grpName, l.groupName());
+                assertEquals(CU.cacheId(grpName), l.groupId());
+            }
+
+            IgniteLock l3 = g1.reentrantLock("l1", true, false, true);
+
+            assertEquals(1, locks1.size());
+
+            ReentrantLockView s = locks1.iterator().next();
+
+            assertTrue(s.locked());
+            assertFalse(s.hasQueuedThreads());
+            assertFalse(s.failoverSafe());
+            assertTrue(s.fair());
+            assertFalse(s.broken());
+            assertFalse(s.removed());
+
+            lockFut.cancel();
+
+            assertTrue(waitForCondition(() -> !s.locked(), getTestTimeout()));
+            assertFalse(s.hasQueuedThreads());
+
+            l3.close();
+
+            assertTrue(s.removed());
+
+            assertEquals(0, locks0.size());
+            assertEquals(0, locks1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testQueue() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+
+            IgniteQueue<String> q0 = g0.queue("queue-1", 42, new CollectionConfiguration()
+                .setCollocated(true)
+                .setBackups(1)
+                .setGroupName("my-group"));
+            IgniteQueue<?> q1 = g0.queue("queue-2", 0, new CollectionConfiguration());
+
+            SystemView<QueueView> queues0 = g0.context().systemView().view(QUEUES_VIEW);
+            SystemView<QueueView> queues1 = g1.context().systemView().view(QUEUES_VIEW);
+
+            assertEquals(2, queues0.size());
+            assertEquals(0, queues1.size());
+
+            for (QueueView q : queues0) {
+                if ("queue-1".equals(q.name())) {
+                    assertNotNull(q.id());
+                    assertEquals("queue-1", q.name());
+                    assertEquals(42, q.capacity());
+                    assertTrue(q.bounded());
+                    assertTrue(q.collocated());
+                    assertEquals("my-group", q.groupName());
+                    assertEquals(CU.cacheId("my-group"), q.groupId());
+                    assertFalse(q.removed());
+                    assertEquals(0, q.size());
+
+                    q0.add("first");
+
+                    assertEquals(1, q.size());
+                }
+                else {
+                    assertNotNull(q.id());
+                    assertEquals("queue-2", q.name());
+                    assertEquals(Integer.MAX_VALUE, q.capacity());
+                    assertFalse(q.bounded());
+                    assertFalse(q.collocated());
+                    assertEquals(DEFAULT_DS_GROUP_NAME, q.groupName());
+                    assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), q.groupId());
+                    assertFalse(q.removed());
+                    assertEquals(0, q.size());
+
+                    q1.close();
+
+                    assertTrue(waitForCondition(q::removed, getTestTimeout()));
+                }
+            }
+
+            IgniteQueue<?> q2 = g1.queue("queue-1", 42, new CollectionConfiguration()
+                .setCollocated(true)
+                .setBackups(1)
+                .setGroupName("my-group"));
+
+            assertEquals(1, queues1.size());
+
+            QueueView q = queues1.iterator().next();
+
+            assertNotNull(q.id());
+            assertEquals("queue-1", q.name());
+            assertEquals(42, q.capacity());
+            assertTrue(q.bounded());
+            assertTrue(q.collocated());
+            assertEquals("my-group", q.groupName());
+            assertEquals(CU.cacheId("my-group"), q.groupId());
+            assertFalse(q.removed());
+            assertEquals(1, q.size());
+
+            q2.close();
+
+            assertTrue(waitForCondition(q::removed, getTestTimeout()));
+
+            assertEquals(0, queues0.size());
+            assertEquals(0, queues1.size());
+        }
+    }
+
+    /** */
+    @Test
+    public void testSet() throws Exception {
+        try (IgniteEx g0 = startGrid(0);
+             IgniteEx g1 = startGrid(1)) {
+
+            IgniteSet<String> s0 = g0.set("set-1", new CollectionConfiguration()
+                .setCollocated(true)
+                .setBackups(1)
+                .setGroupName("my-group"));
+            IgniteSet<?> s1 = g0.set("set-2", new CollectionConfiguration());
+
+            SystemView<SetView> sets0 = g0.context().systemView().view(SETS_VIEW);
+            SystemView<SetView> sets1 = g1.context().systemView().view(SETS_VIEW);
+
+            assertEquals(2, sets0.size());
+            assertEquals(0, sets1.size());
+
+            for (SetView q : sets0) {
+                if ("set-1".equals(q.name())) {
+                    assertNotNull(q.id());
+                    assertEquals("set-1", q.name());
+                    assertTrue(q.collocated());
+                    assertEquals("my-group", q.groupName());
+                    assertEquals(CU.cacheId("my-group"), q.groupId());
+                    assertFalse(q.removed());
+                    assertEquals(0, q.size());
+
+                    s0.add("first");
+
+                    assertEquals(1, q.size());
+                }
+                else {
+                    assertNotNull(q.id());
+                    assertEquals("set-2", q.name());
+                    assertFalse(q.collocated());
+                    assertEquals(DEFAULT_DS_GROUP_NAME, q.groupName());
+                    assertEquals(CU.cacheId(DEFAULT_DS_GROUP_NAME), q.groupId());
+                    assertFalse(q.removed());
+                    assertEquals(0, q.size());
+
+                    s1.close();
+
+                    assertTrue(waitForCondition(q::removed, getTestTimeout()));
+                }
+            }
+
+            IgniteSet<?> s2 = g1.set("set-1", new CollectionConfiguration()
+                .setCollocated(true)
+                .setBackups(1)
+                .setGroupName("my-group"));
+
+            assertEquals(1, sets1.size());
+
+            SetView s = sets1.iterator().next();
+
+            assertNotNull(s.id());
+            assertEquals("set-1", s.name());
+            assertTrue(s.collocated());
+            assertEquals("my-group", s.groupName());
+            assertEquals(CU.cacheId("my-group"), s.groupId());
+            assertFalse(s.removed());
+            assertEquals(1, s.size());
+
+            s2.close();
+
+            assertTrue(waitForCondition(s::removed, getTestTimeout()));
+
+            assertEquals(0, sets0.size());
+            assertEquals(0, sets1.size());
+        }
+    }
+
+    /** */
+    @Test
     public void testStripedExecutors() throws Exception {
         try (IgniteEx g = startGrid(0)) {
             checkStripeExecutorView(g.context().getStripedExecutorService(),