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

korlov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 779df2e460 IGNITE-20578 Implement scan over system view (#2678)
779df2e460 is described below

commit 779df2e4607e2f2e2b6280ee15ef58d8aee42b61
Author: korlov42 <[email protected]>
AuthorDate: Mon Oct 16 14:38:11 2023 +0300

    IGNITE-20578 Implement scan over system view (#2678)
---
 modules/catalog/build.gradle                       |   2 +
 .../internal/catalog/CatalogManagerImpl.java       | 111 +++++-
 .../descriptors/CatalogSystemViewDescriptor.java   |   4 +-
 .../internal/catalog/CatalogSystemViewTest.java    |   4 +-
 .../CreateSystemViewCommandValidationTest.java     |   2 +-
 .../ignite/internal/lang}/InternalTuple.java       |   4 +-
 .../ignite/internal/util/SubscriptionUtils.java    |  17 +
 .../subscription/IterableToPublisherAdapter.java   | 193 ++++++++++
 modules/distribution-zones/build.gradle            |   1 +
 modules/runner/build.gradle                        |   2 +
 .../internal/sql/engine/ItSystemViewsTest.java     | 182 ++++++++++
 .../org/apache/ignite/internal/app/IgniteImpl.java |   5 +-
 modules/schema/build.gradle                        |   1 +
 .../apache/ignite/internal/schema/BinaryTuple.java |   2 +-
 .../ignite/internal/schema/BinaryTuplePrefix.java  |   2 +-
 .../ignite/internal/schema/BinaryTupleSchema.java  |  52 ++-
 .../org/apache/ignite/internal/schema/row/Row.java |   1 +
 modules/sql-engine/build.gradle                    |   1 +
 .../internal/sql/engine/SqlQueryProcessor.java     |   2 +-
 .../internal/sql/engine/exec/RowHandler.java       |   2 +-
 .../sql/engine/exec/ScannableDataSource.java       |   2 +-
 .../internal/sql/engine/exec/SqlRowHandler.java    |   2 +-
 .../sql/engine/exec/TableRowConverterImpl.java     |   2 +-
 .../sql/engine/exec/rel/DataSourceScanNode.java    |   2 +-
 .../sql/engine/schema/CatalogSqlSchemaManager.java |   4 +-
 .../sql/engine/util/AbstractProjectedTuple.java    |   2 +-
 .../util/FieldDeserializingProjectedTuple.java     |   2 +-
 .../sql/engine/util/FormatAwareProjectedTuple.java |   2 +-
 .../sql/engine/exec/rel/AbstractExecutionTest.java |   2 +-
 .../exec/rel/DataSourceScanNodeSelfTest.java       |   2 +-
 .../sql/engine/framework/ArrayRowHandler.java      |   2 +-
 .../engine/schema/CatalogSqlSchemaManagerTest.java |   4 +-
 .../sql/engine/util/ProjectedTupleTest.java        |   2 +-
 modules/storage-api/build.gradle                   |   1 +
 .../build.gradle}                                  |  21 +-
 .../systemview/api}/ClusterSystemView.java         |  16 +-
 .../internal/systemview/api}/NodeSystemView.java   |  25 +-
 .../internal/systemview/api}/SystemView.java       |  48 ++-
 .../internal/systemview/api}/SystemViewColumn.java |   2 +-
 .../systemview/api}/SystemViewManager.java         |  12 +-
 .../systemview/api/SystemViewProvider.java}        |  15 +-
 .../internal/systemview/api}/SystemViews.java      |  10 +-
 .../internal/systemview/api/SystemViewTest.java    | 403 +++++++++++++++++++++
 modules/system-view/build.gradle                   |  17 +-
 .../internal/systemview/SystemViewManagerImpl.java |  91 ++++-
 .../internal/systemview/utils/SystemViewUtils.java |  57 ++-
 .../internal/systemview/SystemViewManagerTest.java | 149 ++++++--
 .../ignite/internal/systemview/SystemViewTest.java | 278 --------------
 modules/table/build.gradle                         |   1 +
 settings.gradle                                    |   2 +
 50 files changed, 1327 insertions(+), 441 deletions(-)

diff --git a/modules/catalog/build.gradle b/modules/catalog/build.gradle
index ab97262744..96c2e643fd 100644
--- a/modules/catalog/build.gradle
+++ b/modules/catalog/build.gradle
@@ -29,6 +29,7 @@ dependencies {
     implementation project(':ignite-configuration')
     implementation project(':ignite-metastorage-api')
     implementation project(':ignite-vault')
+    implementation project(':ignite-system-view-api')
 
     implementation libs.jetbrains.annotations
     implementation libs.auto.service.annotations
@@ -49,6 +50,7 @@ dependencies {
     testFixturesImplementation libs.mockito.core
     testFixturesImplementation libs.mockito.junit
     testFixturesImplementation libs.hamcrest.core
+    testFixturesImplementation project(':ignite-system-view-api')
     testFixturesImplementation(testFixtures(project(':ignite-core')))
     testFixturesImplementation(testFixtures(project(':ignite-vault')))
     testFixturesImplementation(testFixtures(project(':ignite-metastorage')))
diff --git 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
index faf1316d9b..aab4f0f777 100644
--- 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
+++ 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
@@ -36,6 +36,7 @@ import java.util.NavigableMap;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.Flow.Publisher;
 import java.util.function.LongSupplier;
 import org.apache.ignite.internal.catalog.commands.AlterZoneParams;
 import org.apache.ignite.internal.catalog.commands.CreateZoneParams;
@@ -45,6 +46,7 @@ import 
org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogObjectDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogSchemaDescriptor;
 import 
org.apache.ignite.internal.catalog.descriptors.CatalogSystemViewDescriptor;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
 import org.apache.ignite.internal.catalog.events.CatalogEvent;
@@ -66,7 +68,12 @@ import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.lang.IgniteInternalException;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViewProvider;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.type.NativeTypes;
 import org.apache.ignite.internal.util.PendingComparableValuesTracker;
+import org.apache.ignite.internal.util.SubscriptionUtils;
 import org.apache.ignite.lang.ErrorGroups.Common;
 import org.apache.ignite.lang.ErrorGroups.DistributionZones;
 import org.jetbrains.annotations.Nullable;
@@ -74,16 +81,20 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Catalog service implementation.
  */
-public class CatalogManagerImpl extends AbstractEventProducer<CatalogEvent, 
CatalogEventParameters> implements CatalogManager {
+public class CatalogManagerImpl extends AbstractEventProducer<CatalogEvent, 
CatalogEventParameters>
+        implements CatalogManager, SystemViewProvider {
     private static final int MAX_RETRY_COUNT = 10;
 
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
     /** Safe time to wait before new Catalog version activation. */
     private static final int DEFAULT_DELAY_DURATION = 0;
 
     /** Initial update token for a catalog descriptor, this token is valid 
only before the first call of
-     * {@link 
UpdateEntry#applyUpdate(org.apache.ignite.internal.catalog.Catalog, long)}.
-     * After that {@link CatalogObjectDescriptor#updateToken()} will be 
initialised with a causality token from
-     * {@link 
UpdateEntry#applyUpdate(org.apache.ignite.internal.catalog.Catalog, long)}
+     * {@link UpdateEntry#applyUpdate(Catalog, long)}.
+     *
+     * <p>After that {@link CatalogObjectDescriptor#updateToken()} will be 
initialised with a causality token from
+     * {@link UpdateEntry#applyUpdate(Catalog, long)}
      */
     public static final long INITIAL_CAUSALITY_TOKEN = 0L;
 
@@ -448,6 +459,14 @@ public class CatalogManagerImpl extends 
AbstractEventProducer<CatalogEvent, Cata
                 });
     }
 
+    @Override
+    public List<SystemView<?>> systemViews() {
+        return List.of(
+                createSystemViewsView(),
+                createSystemViewColumnsView()
+        );
+    }
+
     class OnUpdateHandlerImpl implements OnUpdateHandler {
         @Override
         public CompletableFuture<Void> handle(VersionedUpdate update, 
HybridTimestamp metaStorageUpdateTimestamp, long causalityToken) {
@@ -537,4 +556,88 @@ public class CatalogManagerImpl extends 
AbstractEventProducer<CatalogEvent, Cata
             return bulkUpdateEntries;
         }
     }
+
+    private SystemView<?> createSystemViewsView() {
+        Iterable<SchemaAwareDescriptor<CatalogSystemViewDescriptor>> viewData 
= () -> {
+            int version = latestCatalogVersion();
+
+            Catalog catalog = catalog(version);
+
+            return catalog.schemas().stream()
+                    .flatMap(schema -> Arrays.stream(schema.systemViews())
+                            .map(viewDescriptor -> new 
SchemaAwareDescriptor<>(viewDescriptor, schema.name()))
+                    )
+                    .iterator();
+        };
+
+        Publisher<SchemaAwareDescriptor<CatalogSystemViewDescriptor>> 
viewDataPublisher = SubscriptionUtils.fromIterable(viewData);
+
+        return 
SystemViews.<SchemaAwareDescriptor<CatalogSystemViewDescriptor>>clusterViewBuilder()
+                .name("SYSTEM_VIEWS")
+                .addColumn("ID", NativeTypes.INT32, entry -> 
entry.descriptor.id())
+                .addColumn("SCHEMA", 
NativeTypes.stringOf(SYSTEM_VIEW_STRING_COLUMN_LENGTH),
+                        entry -> entry.schema)
+                .addColumn("NAME", 
NativeTypes.stringOf(SYSTEM_VIEW_STRING_COLUMN_LENGTH),
+                        entry -> entry.descriptor.name())
+                .addColumn("TYPE", 
NativeTypes.stringOf(SYSTEM_VIEW_STRING_COLUMN_LENGTH),
+                        entry -> entry.descriptor.systemViewType().name())
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    private SystemView<?> createSystemViewColumnsView() {
+        Iterable<ParentIdAwareDescriptor<CatalogTableColumnDescriptor>> 
viewData = () -> {
+            int version = latestCatalogVersion();
+
+            Catalog catalog = catalog(version);
+
+            return catalog.schemas().stream()
+                    .flatMap(schema -> Arrays.stream(schema.systemViews()))
+                    .flatMap(viewDescriptor -> 
viewDescriptor.columns().stream()
+                            .map(columnDescriptor -> new 
ParentIdAwareDescriptor<>(columnDescriptor, viewDescriptor.id()))
+                    )
+                    .iterator();
+        };
+
+        Publisher<ParentIdAwareDescriptor<CatalogTableColumnDescriptor>> 
viewDataPublisher = SubscriptionUtils.fromIterable(viewData);
+
+        return 
SystemViews.<ParentIdAwareDescriptor<CatalogTableColumnDescriptor>>clusterViewBuilder()
+                .name("SYSTEM_VIEW_COLUMNS")
+                .addColumn("VIEW_ID", NativeTypes.INT32, entry -> entry.id)
+                .addColumn("NAME", 
NativeTypes.stringOf(SYSTEM_VIEW_STRING_COLUMN_LENGTH), entry -> 
entry.descriptor.name())
+                .addColumn("TYPE", 
NativeTypes.stringOf(SYSTEM_VIEW_STRING_COLUMN_LENGTH), entry -> 
entry.descriptor.type().name())
+                .addColumn("NULLABLE", NativeTypes.BOOLEAN, entry -> 
entry.descriptor.nullable())
+                .addColumn("PRECISION", NativeTypes.INT32, entry -> 
entry.descriptor.precision())
+                .addColumn("SCALE", NativeTypes.INT32, entry -> 
entry.descriptor.scale())
+                .addColumn("LENGTH", NativeTypes.INT32, entry -> 
entry.descriptor.length())
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    /**
+     * A container that keeps given descriptor along with name of the schema 
this
+     * descriptor belongs to.
+     */
+    private static class SchemaAwareDescriptor<T> {
+        private final T descriptor;
+        private final String schema;
+
+        SchemaAwareDescriptor(T descriptor, String schema) {
+            this.descriptor = descriptor;
+            this.schema = schema;
+        }
+    }
+
+    /**
+     * A container that keeps given descriptor along with its parent's id.
+     */
+    private static class ParentIdAwareDescriptor<T> {
+        private final T descriptor;
+        private final int id;
+
+        ParentIdAwareDescriptor(T descriptor, int id) {
+            this.descriptor = descriptor;
+            this.id = id;
+        }
+    }
 }
diff --git 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSystemViewDescriptor.java
 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSystemViewDescriptor.java
index 9a5af26a55..37c5379a3f 100644
--- 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSystemViewDescriptor.java
+++ 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/CatalogSystemViewDescriptor.java
@@ -98,10 +98,10 @@ public class CatalogSystemViewDescriptor extends 
CatalogObjectDescriptor {
         /**
          * Node system view.
          */
-        LOCAL,
+        NODE,
         /**
          * Cluster-wide system view.
          */
-        GLOBAL
+        CLUSTER
     }
 }
diff --git 
a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogSystemViewTest.java
 
b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogSystemViewTest.java
index 48ee7f9dbb..4a47983c61 100644
--- 
a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogSystemViewTest.java
+++ 
b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogSystemViewTest.java
@@ -201,13 +201,13 @@ public class CatalogSystemViewTest extends 
BaseCatalogManagerTest {
             return CreateSystemViewCommand.builder()
                     .name(SYS_VIEW_NAME)
                     .columns(COLUMNS)
-                    .type(SystemViewType.LOCAL);
+                    .type(SystemViewType.NODE);
         }
 
         void apply(CreateSystemViewCommandBuilder builder) {
             switch (this) {
                 case CHANGE_TYPE: {
-                    builder.type(SystemViewType.GLOBAL);
+                    builder.type(SystemViewType.CLUSTER);
                     break;
                 }
                 case ADD_COLUMN: {
diff --git 
a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateSystemViewCommandValidationTest.java
 
b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateSystemViewCommandValidationTest.java
index 88e7255cc8..83c2a279cf 100644
--- 
a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateSystemViewCommandValidationTest.java
+++ 
b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/commands/CreateSystemViewCommandValidationTest.java
@@ -107,6 +107,6 @@ public class CreateSystemViewCommandValidationTest extends 
AbstractCommandValida
         ColumnParams column = 
ColumnParams.builder().name("C").type(ColumnType.INT8).build();
         List<ColumnParams> columns = List.of(column);
 
-        return 
builder.name("view").columns(columns).type(SystemViewType.LOCAL);
+        return builder.name("view").columns(columns).type(SystemViewType.NODE);
     }
 }
diff --git 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/InternalTuple.java
 b/modules/core/src/main/java/org/apache/ignite/internal/lang/InternalTuple.java
similarity index 96%
rename from 
modules/schema/src/main/java/org/apache/ignite/internal/schema/row/InternalTuple.java
rename to 
modules/core/src/main/java/org/apache/ignite/internal/lang/InternalTuple.java
index c3e242e7bc..596e3a3a18 100644
--- 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/InternalTuple.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/lang/InternalTuple.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.schema.row;
+package org.apache.ignite.internal.lang;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -26,11 +26,9 @@ import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.BitSet;
 import java.util.UUID;
-import org.apache.ignite.internal.schema.InvalidTypeException;
 
 /**
  * General interface to describe tuples outside of their data layout and 
column schemas.
- * Accessor methods may or may not throw {@link InvalidTypeException} 
depending on the implementation.
  */
 public interface InternalTuple {
     /**
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/util/SubscriptionUtils.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/util/SubscriptionUtils.java
index 00073ad656..41b4f81d6b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/util/SubscriptionUtils.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/util/SubscriptionUtils.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.Flow.Publisher;
 import org.apache.ignite.internal.util.subscription.ConcatenatedPublisher;
+import org.apache.ignite.internal.util.subscription.IterableToPublisherAdapter;
 import org.apache.ignite.internal.util.subscription.OrderedMergePublisher;
 
 /**
@@ -81,4 +82,20 @@ public class SubscriptionUtils {
     public static <T> Publisher<T> orderedMerge(Comparator<T> comparator, int 
prefetch, Publisher<? extends T>... sources) {
         return new OrderedMergePublisher<>(comparator, prefetch, sources);
     }
+
+    /**
+     * Creates a publisher from the given iterable.
+     *
+     * <p>A new iterator will be issued for every new subscription.
+     *
+     * <p>This particular adapter will drain iterator on the same thread that 
requested the
+     * entries, so it's better to avoid using long or blocking operations 
inside provided iterable.
+     *
+     * @param iterable An iterable to create adapter for.
+     * @param <T> Type of the entries this publisher will emit.
+     * @return Publisher created from the given iterable.
+     */
+    public static <T> Publisher<T> fromIterable(Iterable<T> iterable) {
+        return new IterableToPublisherAdapter<>(iterable, Runnable::run, 
Integer.MAX_VALUE);
+    }
 }
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/util/subscription/IterableToPublisherAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/util/subscription/IterableToPublisherAdapter.java
new file mode 100644
index 0000000000..60670d3924
--- /dev/null
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/util/subscription/IterableToPublisherAdapter.java
@@ -0,0 +1,193 @@
+/*
+ * 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.util.subscription;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodHandles.Lookup;
+import java.lang.invoke.VarHandle;
+import java.util.Iterator;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
+
+/**
+ * An adapter that issues a new iterator for every subscription and drains 
that iterator on a given
+ * executor, emitting up to given {@code batchSize} entries at time. The drain 
task will be re-scheduled
+ * until demand is fulfilled or iterator has no more items.
+ *
+ * @param <T> The type of the entry this publisher will emit.
+ */
+public class IterableToPublisherAdapter<T> implements Publisher<T> {
+    private final Iterable<T> iterable;
+    private final Executor executor;
+    private final int batchSize;
+
+    /**
+     * Constructor.
+     *
+     * @param iterable An iterable to issue iterator for every incoming 
subscription.
+     * @param executor This executor will be used to drain iterator and supply 
entries to the subscription.
+     * @param batchSize An amount of entries to supply during a single 
iteration. It's always good idea
+     *      to provide some reasonable value here in order to give am ability 
to other publishers which share the same
+     *      executor to make progress.
+     */
+    public IterableToPublisherAdapter(Iterable<T> iterable, Executor executor, 
int batchSize) {
+        this.iterable = iterable;
+        this.executor = executor;
+        this.batchSize = batchSize;
+    }
+
+    @Override
+    public void subscribe(Subscriber<? super T> subscriber) {
+        Iterator<T> it = iterable.iterator();
+
+        Subscription subscription = new SubscriptionImpl<>(it, subscriber, 
executor, batchSize);
+
+        subscriber.onSubscribe(subscription);
+    }
+
+    @SuppressWarnings("FieldMayBeFinal")
+    private static class SubscriptionImpl<T> implements Subscription {
+        private static final VarHandle CANCELLED_HANDLE;
+        private static final VarHandle REQUESTED_HANDLE;
+        private static final VarHandle WIP_HANDLE;
+
+        static {
+            try {
+                Lookup lookup = MethodHandles.lookup();
+
+                CANCELLED_HANDLE = 
lookup.findVarHandle(SubscriptionImpl.class, "cancelled", boolean.class);
+                REQUESTED_HANDLE = 
lookup.findVarHandle(SubscriptionImpl.class, "requested", long.class);
+                WIP_HANDLE = lookup.findVarHandle(SubscriptionImpl.class, 
"wip", boolean.class);
+            } catch (NoSuchFieldException | IllegalAccessException e) {
+                throw new ExceptionInInitializerError(e);
+            }
+        }
+
+        private final Iterator<T> it;
+        private final Subscriber<? super T> subscriber;
+        private final Executor executor;
+        private final int batchSize;
+
+        private boolean cancelled = false;
+        private long requested = 0;
+        private boolean wip = false;
+
+        SubscriptionImpl(Iterator<T> it, Subscriber<? super T> subscriber, 
Executor executor, int batchSize) {
+            this.it = it;
+            this.subscriber = subscriber;
+            this.executor = executor;
+            this.batchSize = batchSize;
+        }
+
+        @Override
+        public void request(long n) {
+            if (n <= 0) {
+                notifyError(new IllegalArgumentException("N should be 
positive:" + n));
+
+                return;
+            }
+
+            if ((boolean) CANCELLED_HANDLE.getAcquire(this)) {
+                return;
+            }
+
+            long oldValue;
+            long newValue;
+            do {
+                oldValue = (long) REQUESTED_HANDLE.getAcquire(this);
+
+                newValue = oldValue + n;
+
+                if (newValue < 0) {
+                    newValue = Long.MAX_VALUE;
+                }
+            } while (!REQUESTED_HANDLE.compareAndSet(this, oldValue, 
newValue));
+
+            // the task may be scheduled several times, but it's ok, since we 
will deal with this
+            // inside task itself
+            executor.execute(this::drain);
+        }
+
+        @Override
+        public void cancel() {
+            CANCELLED_HANDLE.setRelease(this, true);
+        }
+
+        private void drain() {
+            if (!WIP_HANDLE.compareAndSet(this, false, true)) {
+                return;
+            }
+
+            if ((boolean) CANCELLED_HANDLE.getAcquire(this)) {
+                return;
+            }
+
+            long amount = amountToDrain();
+
+            try {
+                while (amount-- > 0 && it.hasNext()) {
+                    subscriber.onNext(it.next());
+                }
+            } catch (Throwable th) {
+                notifyError(th);
+            }
+
+            if (amount > 0) {
+                // according to javadoc, no need to send onComplete signal if 
subscription has been cancelled
+                if (CANCELLED_HANDLE.compareAndSet(this, false, true)) {
+                    subscriber.onComplete();
+                }
+
+                return;
+            }
+
+            WIP_HANDLE.setRelease(this, false);
+
+            if (((long) REQUESTED_HANDLE.getAcquire(this)) > 0) {
+                executor.execute(this::drain);
+            }
+        }
+
+        private long amountToDrain() {
+            long oldRequested;
+            long newRequested;
+            do {
+                oldRequested = (long) REQUESTED_HANDLE.getAcquire(this);
+
+                if (oldRequested <= batchSize) {
+                    newRequested = 0;
+                } else {
+                    newRequested = oldRequested - batchSize;
+                }
+            } while (!REQUESTED_HANDLE.compareAndSet(this, oldRequested, 
newRequested));
+
+            return oldRequested - newRequested;
+        }
+
+        private void notifyError(Throwable th) {
+            // according to javadoc, no need to send onError signal if 
subscription has been cancelled
+            if (!CANCELLED_HANDLE.compareAndSet(this, false, true)) {
+                return;
+            }
+
+            subscriber.onError(th);
+        }
+    }
+}
diff --git a/modules/distribution-zones/build.gradle 
b/modules/distribution-zones/build.gradle
index cb41a61ff0..09c49f11c1 100644
--- a/modules/distribution-zones/build.gradle
+++ b/modules/distribution-zones/build.gradle
@@ -52,6 +52,7 @@ dependencies {
     testImplementation project(':ignite-metastorage-api')
     testImplementation project(':ignite-metastorage')
     testImplementation project(':ignite-runner')
+    testImplementation project(':ignite-system-view-api')
 
     testImplementation(testFixtures(project(':ignite-core')))
     testImplementation(testFixtures(project(':ignite-configuration')))
diff --git a/modules/runner/build.gradle b/modules/runner/build.gradle
index 6ca384c363..d8ee68f3ed 100644
--- a/modules/runner/build.gradle
+++ b/modules/runner/build.gradle
@@ -77,6 +77,7 @@ dependencies {
     implementation project(':ignite-security-api')
     implementation project(':ignite-security')
     implementation project(':ignite-catalog')
+    implementation project(':ignite-system-view-api')
     implementation project(':ignite-system-view')
     implementation libs.jetbrains.annotations
     implementation libs.micronaut.inject
@@ -148,6 +149,7 @@ dependencies {
     integrationTestImplementation project(':ignite-placement-driver')
     integrationTestImplementation project(':ignite-distribution-zones')
     integrationTestImplementation project(':ignite-system-view')
+    integrationTestImplementation project(':ignite-system-view-api')
     integrationTestImplementation testFixtures(project(":ignite-api"))
     integrationTestImplementation testFixtures(project(':ignite-core'))
     integrationTestImplementation 
testFixtures(project(':ignite-configuration'))
diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java
new file mode 100644
index 0000000000..d73782450a
--- /dev/null
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSystemViewsTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.sql.engine;
+
+import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
+import static 
org.apache.ignite.internal.sql.engine.ItSystemViewsTest.KnownSystemView.SYSTEM_VIEWS;
+import static 
org.apache.ignite.internal.sql.engine.ItSystemViewsTest.KnownSystemView.SYSTEM_VIEW_COLUMNS;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+
+import java.util.List;
+import org.apache.ignite.internal.sql.engine.util.MetadataMatcher;
+import org.apache.ignite.sql.ColumnType;
+import org.apache.ignite.sql.ResultSet;
+import org.apache.ignite.sql.ResultSetMetadata;
+import org.apache.ignite.sql.Session;
+import org.apache.ignite.sql.SqlRow;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+/**
+ * End-to-end tests to verify system views.
+ */
+@SuppressWarnings("DataFlowIssue")
+public class ItSystemViewsTest extends ClusterPerClassIntegrationTest {
+    enum KnownSystemView {
+        SYSTEM_VIEWS("SYSTEM", "SYSTEM_VIEWS"),
+        SYSTEM_VIEW_COLUMNS("SYSTEM", "SYSTEM_VIEW_COLUMNS"),
+        ;
+
+        private final String schema;
+        private final String name;
+
+        KnownSystemView(String schema, String name) {
+            this.schema = schema;
+            this.name = name;
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(KnownSystemView.class)
+    public void systemViewWithGivenNameExists(KnownSystemView view) {
+        try (Session session = 
CLUSTER_NODES.get(0).sql().sessionBuilder().defaultSchema(view.schema).build()) 
{
+            ResultSet<SqlRow> rs = session.execute(null, format("SELECT 
count(*) FROM {}", view.name));
+
+            // for this test it's enough to check presence of the row,
+            // because we are interested in whether the view is available for
+            // querying at all
+            assertThat(rs.hasNext(), is(true));
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(KnownSystemView.class)
+    public void 
systemViewWithGivenNamePresentedInSystemViewsView(KnownSystemView view) {
+        try (Session session = 
CLUSTER_NODES.get(0).sql().sessionBuilder().defaultSchema(SYSTEM_VIEWS.schema).build())
 {
+            ResultSet<SqlRow> rs = session.execute(null,
+                    format("SELECT count(*) FROM {} WHERE schema = '{}' AND 
name = '{}'",
+                            SYSTEM_VIEWS.name, view.schema, view.name));
+
+            assertThat(rs.hasNext(), is(true));
+
+            SqlRow row = rs.next();
+
+            assertThat(row.value(0), is(1L));
+        }
+    }
+
+    @Test
+    public void systemViewsViewMetadataTest() {
+        KnownSystemView view = SYSTEM_VIEWS;
+
+        try (Session session = 
CLUSTER_NODES.get(0).sql().sessionBuilder().defaultSchema(view.schema).build()) 
{
+            ResultSet<SqlRow> rs = session.execute(null, format("SELECT * FROM 
{}", view.name));
+
+            ResultSetMetadata metadata = rs.metadata();
+
+            assertMetadata(metadata, List.of(
+                    new MetadataMatcher()
+                            .name("ID")
+                            .type(ColumnType.INT32)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("SCHEMA")
+                            .type(ColumnType.STRING)
+                            .precision(Short.MAX_VALUE)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("NAME")
+                            .type(ColumnType.STRING)
+                            .precision(Short.MAX_VALUE)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("TYPE")
+                            .type(ColumnType.STRING)
+                            .precision(Short.MAX_VALUE)
+                            .nullable(true)
+            ));
+        }
+    }
+
+    @Test
+    public void systemViewColumnsViewMetadataTest() {
+        KnownSystemView view = SYSTEM_VIEW_COLUMNS;
+
+        try (Session session = 
CLUSTER_NODES.get(0).sql().sessionBuilder().defaultSchema(view.schema).build()) 
{
+            ResultSet<SqlRow> rs = session.execute(null, format("SELECT * FROM 
{}", view.name));
+
+            ResultSetMetadata metadata = rs.metadata();
+
+            assertMetadata(metadata, List.of(
+                    new MetadataMatcher()
+                            .name("VIEW_ID")
+                            .type(ColumnType.INT32)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("NAME")
+                            .type(ColumnType.STRING)
+                            .precision(Short.MAX_VALUE)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("TYPE")
+                            .type(ColumnType.STRING)
+                            .precision(Short.MAX_VALUE)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("NULLABLE")
+                            .type(ColumnType.BOOLEAN)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("PRECISION")
+                            .type(ColumnType.INT32)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("SCALE")
+                            .type(ColumnType.INT32)
+                            .nullable(true),
+
+                    new MetadataMatcher()
+                            .name("LENGTH")
+                            .type(ColumnType.INT32)
+                            .nullable(true)
+            ));
+
+        }
+    }
+
+    private static void assertMetadata(ResultSetMetadata actual, 
List<MetadataMatcher> matchers) {
+        assertThat(actual, notNullValue());
+        assertThat(actual.columns(), hasSize(matchers.size()));
+
+        for (int i = 0; i < matchers.size(); i++) {
+            matchers.get(i).check(actual.columns().get(i));
+        }
+    }
+}
diff --git 
a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java 
b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 99cb691bd9..5cf0ace69a 100644
--- 
a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ 
b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -508,7 +508,7 @@ public class IgniteImpl implements Ignite {
 
         LongSupplier delayDurationMsSupplier = () -> 
schemaSyncConfig.delayDuration().value();
 
-        catalogManager = new CatalogManagerImpl(
+        CatalogManagerImpl catalogManager = new CatalogManagerImpl(
                 new UpdateLogImpl(metaStorageMgr),
                 clockWaiter,
                 delayDurationMsSupplier
@@ -517,6 +517,9 @@ public class IgniteImpl implements Ignite {
         systemViewManager = new SystemViewManagerImpl(name, catalogManager);
         nodeAttributesCollector.register(systemViewManager);
         logicalTopology.addEventListener(systemViewManager);
+        systemViewManager.register(catalogManager);
+
+        this.catalogManager = catalogManager;
 
         raftMgr.appendEntriesRequestInterceptor(new 
CheckCatalogVersionOnAppendEntries(catalogManager));
         raftMgr.actionRequestInterceptor(new 
CheckCatalogVersionOnActionRequest(catalogManager));
diff --git a/modules/schema/build.gradle b/modules/schema/build.gradle
index 227e7890a0..92cdd24d10 100644
--- a/modules/schema/build.gradle
+++ b/modules/schema/build.gradle
@@ -44,6 +44,7 @@ dependencies {
     testImplementation project(':ignite-configuration')
     testImplementation project(':ignite-core')
     testImplementation project(':ignite-vault')
+    testImplementation project(':ignite-system-view-api')
     testImplementation(testFixtures(project(':ignite-core')))
     testImplementation(testFixtures(project(':ignite-configuration')))
     testImplementation(testFixtures(project(':ignite-metastorage')))
diff --git 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuple.java
 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuple.java
index 26884b68e1..2d8da1d448 100644
--- 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuple.java
+++ 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuple.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.schema;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 
 /**
  * Utility for access to binary tuple elements as typed values and with schema 
knowledge that allows to read
diff --git 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java
 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java
index 0c437a74f4..cecbb140d4 100644
--- 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java
+++ 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java
@@ -22,7 +22,7 @@ import static 
org.apache.ignite.internal.binarytuple.BinaryTupleCommon.PREFIX_FL
 import java.nio.ByteBuffer;
 import org.apache.ignite.internal.binarytuple.BinaryTuplePrefixBuilder;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 
 /**
  * Class that represents a Binary Tuple Prefix.
diff --git 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTupleSchema.java
 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTupleSchema.java
index 8962617474..551c61d373 100644
--- 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTupleSchema.java
+++ 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTupleSchema.java
@@ -18,8 +18,17 @@
 package org.apache.ignite.internal.schema;
 
 import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.BitSet;
+import java.util.UUID;
+import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
+import org.apache.ignite.internal.binarytuple.BinaryTupleFormatException;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.type.DecimalNativeType;
 import org.apache.ignite.internal.type.NativeType;
 import org.apache.ignite.internal.type.NativeTypeSpec;
@@ -315,4 +324,45 @@ public class BinaryTupleSchema {
             default: throw new InvalidTypeException("Unknown element type: " + 
element.typeSpec);
         }
     }
+
+    /**
+     * Helper method that adds value to the binary tuple builder.
+     *
+     * @param builder Binary tuple builder.
+     * @param index Field index to write.
+     * @param value Value to add.
+     * @return Binary tuple builder.
+     */
+    public BinaryTupleBuilder appendValue(BinaryTupleBuilder builder, int 
index, @Nullable Object value) {
+        Element element = element(index);
+
+        if (value == null) {
+            if (!element.nullable()) {
+                throw new BinaryTupleFormatException("NULL value for 
non-nullable column in binary tuple builder.");
+            }
+
+            return builder.appendNull();
+        }
+
+        switch (element.typeSpec()) {
+            case BOOLEAN: return builder.appendBoolean((boolean) value);
+            case INT8: return builder.appendByte((byte) value);
+            case INT16: return builder.appendShort((short) value);
+            case INT32: return builder.appendInt((int) value);
+            case INT64: return builder.appendLong((long) value);
+            case FLOAT: return builder.appendFloat((float) value);
+            case DOUBLE: return builder.appendDouble((double) value);
+            case NUMBER: return builder.appendNumberNotNull((BigInteger) 
value);
+            case DECIMAL: return builder.appendDecimalNotNull((BigDecimal) 
value, element.decimalScale());
+            case UUID: return builder.appendUuidNotNull((UUID) value);
+            case BYTES: return builder.appendBytesNotNull((byte[]) value);
+            case STRING: return builder.appendStringNotNull((String) value);
+            case BITMASK: return builder.appendBitmaskNotNull((BitSet) value);
+            case DATE: return builder.appendDateNotNull((LocalDate) value);
+            case TIME: return builder.appendTimeNotNull((LocalTime) value);
+            case DATETIME: return 
builder.appendDateTimeNotNull((LocalDateTime) value);
+            case TIMESTAMP: return builder.appendTimestampNotNull((Instant) 
value);
+            default: throw new InvalidTypeException("Unknown element type: " + 
element.typeSpec);
+        }
+    }
 }
diff --git 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java
index 7335173221..6294815237 100644
--- 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java
+++ 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/Row.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import org.apache.ignite.internal.binarytuple.BinaryTupleContainer;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.BinaryRowEx;
 import org.apache.ignite.internal.schema.BinaryTuple;
diff --git a/modules/sql-engine/build.gradle b/modules/sql-engine/build.gradle
index d95e7bb899..bd83e64856 100644
--- a/modules/sql-engine/build.gradle
+++ b/modules/sql-engine/build.gradle
@@ -38,6 +38,7 @@ dependencies {
     implementation project(':ignite-catalog')
     implementation project(':ignite-metrics')
     implementation project(':ignite-cluster-management')
+    implementation project(':ignite-system-view-api')
     implementation project(':ignite-system-view')
     implementation libs.jetbrains.annotations
     implementation libs.fastutil.core
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
index bf072ca3a3..f37ca76be3 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java
@@ -91,7 +91,7 @@ import org.apache.ignite.internal.sql.engine.util.TypeUtils;
 import org.apache.ignite.internal.sql.engine.util.cache.CaffeineCacheFactory;
 import org.apache.ignite.internal.sql.metrics.SqlClientMetricSource;
 import org.apache.ignite.internal.storage.DataStorageManager;
-import org.apache.ignite.internal.systemview.SystemViewManager;
+import org.apache.ignite.internal.systemview.api.SystemViewManager;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.table.distributed.schema.SchemaSyncService;
 import org.apache.ignite.internal.tx.InternalTransaction;
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/RowHandler.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/RowHandler.java
index 406b0dc7aa..302be058c9 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/RowHandler.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/RowHandler.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.sql.engine.exec;
 
 import java.nio.ByteBuffer;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.sql.engine.exec.row.RowSchema;
 import org.jetbrains.annotations.Nullable;
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
index 7453c899fb..6c97208744 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.sql.engine.exec;
 
 import java.util.concurrent.Flow.Publisher;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 
 /**
  * Provides read operations over an abstract data source.
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/SqlRowHandler.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/SqlRowHandler.java
index 50cc0f3306..51b941ffa1 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/SqlRowHandler.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/SqlRowHandler.java
@@ -32,9 +32,9 @@ import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.lang.IgniteStringBuilder;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.schema.InvalidTypeException;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 import org.apache.ignite.internal.sql.engine.exec.SqlRowHandler.RowWrapper;
 import org.apache.ignite.internal.sql.engine.exec.row.RowSchema;
 import org.apache.ignite.internal.sql.engine.exec.row.RowSchema.Builder;
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
index 9af3e25aa9..7e81b9cf72 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterImpl.java
@@ -18,12 +18,12 @@
 package org.apache.ignite.internal.sql.engine.exec;
 
 import java.util.BitSet;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.schema.BinaryTupleSchema;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaRegistry;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 import org.apache.ignite.internal.sql.engine.schema.ColumnDescriptor;
 import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
 import 
org.apache.ignite.internal.sql.engine.util.FieldDeserializingProjectedTuple;
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNode.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNode.java
index 24da64c175..f8d68cbf92 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNode.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNode.java
@@ -21,8 +21,8 @@ import java.util.BitSet;
 import java.util.concurrent.Flow.Publisher;
 import java.util.function.Function;
 import java.util.function.Predicate;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryTupleSchema;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler;
 import org.apache.ignite.internal.sql.engine.exec.ScannableDataSource;
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManager.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManager.java
index e10656d9da..fd8ab94ef0 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManager.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManager.java
@@ -225,11 +225,11 @@ public class CatalogSqlSchemaManager implements 
SqlSchemaManager {
         SystemViewType systemViewType = descriptor.systemViewType();
 
         switch (systemViewType) {
-            case LOCAL:
+            case NODE:
                 // node name is always the first column.
                 distribution = IgniteDistributions.identity(0);
                 break;
-            case GLOBAL:
+            case CLUSTER:
                 distribution = IgniteDistributions.single();
                 break;
             default:
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/AbstractProjectedTuple.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/AbstractProjectedTuple.java
index dbcf66a3c2..2ca099ad12 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/AbstractProjectedTuple.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/AbstractProjectedTuple.java
@@ -26,7 +26,7 @@ import java.time.LocalDateTime;
 import java.time.LocalTime;
 import java.util.BitSet;
 import java.util.UUID;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 
 /**
  * Projected Tuple is a facade that creates projection of the given tuple.
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FieldDeserializingProjectedTuple.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FieldDeserializingProjectedTuple.java
index 926fc89b47..22b0406851 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FieldDeserializingProjectedTuple.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FieldDeserializingProjectedTuple.java
@@ -18,11 +18,11 @@
 package org.apache.ignite.internal.sql.engine.util;
 
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryRowConverter;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.schema.BinaryTupleSchema;
 import org.apache.ignite.internal.schema.BinaryTupleSchema.Element;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 
 /**
  * A projected tuple that doesn't require delegate to be in particular format.
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FormatAwareProjectedTuple.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FormatAwareProjectedTuple.java
index 5ee256b9c9..5e6b24ea4e 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FormatAwareProjectedTuple.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/FormatAwareProjectedTuple.java
@@ -21,8 +21,8 @@ import java.nio.ByteBuffer;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.binarytuple.BinaryTupleParser;
 import org.apache.ignite.internal.binarytuple.BinaryTupleParser.Sink;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryTuple;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 
 /**
  * A projected tuple that aware of the format of delegate.
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractExecutionTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractExecutionTest.java
index 28ffb1549b..49947228b1 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractExecutionTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/AbstractExecutionTest.java
@@ -39,10 +39,10 @@ import java.util.stream.Stream;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryRowConverter;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.schema.BinaryTupleSchema;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
 import org.apache.ignite.internal.sql.engine.exec.QueryTaskExecutorImpl;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler;
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNodeSelfTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNodeSelfTest.java
index b41b44b178..1adb68c924 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNodeSelfTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/DataSourceScanNodeSelfTest.java
@@ -36,9 +36,9 @@ import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryTupleSchema;
 import org.apache.ignite.internal.schema.BinaryTupleSchema.Element;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/ArrayRowHandler.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/ArrayRowHandler.java
index 9ff00ae9fa..21c9f69e9f 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/ArrayRowHandler.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/ArrayRowHandler.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.sql.engine.framework;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler;
 import org.apache.ignite.internal.sql.engine.exec.row.RowSchema;
 import org.apache.ignite.internal.util.ArrayUtils;
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManagerTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManagerTest.java
index 88372fd4f2..cf0b2198f1 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManagerTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/CatalogSqlSchemaManagerTest.java
@@ -483,8 +483,8 @@ public class CatalogSqlSchemaManagerTest extends 
BaseIgniteAbstractTest {
 
     private static Stream<Arguments> systemViewDistributions() {
         return Stream.of(
-                Arguments.of(SystemViewType.LOCAL, 
IgniteDistributions.identity(0)),
-                Arguments.of(SystemViewType.GLOBAL, 
IgniteDistributions.single())
+                Arguments.of(SystemViewType.NODE, 
IgniteDistributions.identity(0)),
+                Arguments.of(SystemViewType.CLUSTER, 
IgniteDistributions.single())
         );
     }
 
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/ProjectedTupleTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/ProjectedTupleTest.java
index 29e8c8a1f9..5a97e4ca26 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/ProjectedTupleTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/ProjectedTupleTest.java
@@ -29,6 +29,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.schema.BinaryRowConverter;
@@ -36,7 +37,6 @@ import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.schema.BinaryTupleSchema;
 import org.apache.ignite.internal.schema.BinaryTupleSchema.Element;
 import org.apache.ignite.internal.schema.SchemaTestUtils;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 import org.apache.ignite.internal.type.NativeTypeSpec;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
diff --git a/modules/storage-api/build.gradle b/modules/storage-api/build.gradle
index 81555f0db7..f64dd596df 100644
--- a/modules/storage-api/build.gradle
+++ b/modules/storage-api/build.gradle
@@ -54,6 +54,7 @@ dependencies {
     testFixturesImplementation project(':ignite-schema')
     testFixturesImplementation project(':ignite-api')
     testFixturesImplementation project(':ignite-catalog')
+    testFixturesImplementation project(':ignite-system-view-api')
     testFixturesImplementation(testFixtures(project(':ignite-core')))
     testFixturesImplementation(testFixtures(project(':ignite-configuration')))
     testFixturesImplementation(testFixtures(project(':ignite-schema')))
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
 b/modules/system-view-api/build.gradle
similarity index 63%
copy from 
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
copy to modules/system-view-api/build.gradle
index 7453c899fb..3006a5429e 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
+++ b/modules/system-view-api/build.gradle
@@ -15,15 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.sql.engine.exec;
+apply from: "$rootDir/buildscripts/java-core.gradle"
+apply from: "$rootDir/buildscripts/publishing.gradle"
+apply from: "$rootDir/buildscripts/java-junit5.gradle"
+apply from: "$rootDir/buildscripts/java-test-fixtures.gradle"
 
-import java.util.concurrent.Flow.Publisher;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+dependencies {
+    implementation project(':ignite-core')
 
-/**
- * Provides read operations over an abstract data source.
- */
-@FunctionalInterface
-public interface ScannableDataSource {
-    Publisher<InternalTuple> scan();
+    implementation libs.jetbrains.annotations
+
+    testImplementation libs.hamcrest.core
+    testImplementation(testFixtures(project(':ignite-core')))
 }
+
+description = 'ignite-system-view-api'
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/ClusterSystemView.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/ClusterSystemView.java
similarity index 84%
rename from 
modules/system-view/src/main/java/org/apache/ignite/internal/systemview/ClusterSystemView.java
rename to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/ClusterSystemView.java
index cee6c7ba9f..078112ba34 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/ClusterSystemView.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/ClusterSystemView.java
@@ -15,13 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.systemview;
+package org.apache.ignite.internal.systemview.api;
 
 import java.util.List;
-import java.util.function.Supplier;
-import 
org.apache.ignite.internal.catalog.descriptors.CatalogSystemViewDescriptor.SystemViewType;
+import java.util.concurrent.Flow.Publisher;
 import org.apache.ignite.internal.tostring.S;
-import org.apache.ignite.internal.util.AsyncCursor;
 
 /**
  * Cluster wide view definition.
@@ -49,19 +47,13 @@ public class ClusterSystemView<T> extends SystemView<T> {
      * @param columns List of columns.
      * @param dataProvider Data provider.
      */
-    ClusterSystemView(String name,
+    private ClusterSystemView(String name,
             List<SystemViewColumn<T, ?>> columns,
-            Supplier<AsyncCursor<T>> dataProvider) {
+            Publisher<T> dataProvider) {
 
         super(name, columns, dataProvider);
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public SystemViewType type() {
-        return SystemViewType.GLOBAL;
-    }
-
     /** {@inheritDoc} */
     @Override
     public String toString() {
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/NodeSystemView.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/NodeSystemView.java
similarity index 85%
rename from 
modules/system-view/src/main/java/org/apache/ignite/internal/systemview/NodeSystemView.java
rename to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/NodeSystemView.java
index e1c4c7bd64..0468491520 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/NodeSystemView.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/NodeSystemView.java
@@ -15,13 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.systemview;
+package org.apache.ignite.internal.systemview.api;
 
 import java.util.List;
-import java.util.function.Supplier;
-import 
org.apache.ignite.internal.catalog.descriptors.CatalogSystemViewDescriptor.SystemViewType;
+import java.util.concurrent.Flow.Publisher;
 import org.apache.ignite.internal.tostring.S;
-import org.apache.ignite.internal.util.AsyncCursor;
 import org.apache.ignite.internal.util.StringUtils;
 
 /**
@@ -53,9 +51,9 @@ public class NodeSystemView<T> extends SystemView<T> {
      * @param dataProvider Data provider.
      * @param nodeNameColumnAlias Node name column alias.
      */
-    NodeSystemView(String name,
+    private NodeSystemView(String name,
             List<SystemViewColumn<T, ?>> columns,
-            Supplier<AsyncCursor<T>> dataProvider,
+            Publisher<T> dataProvider,
             String nodeNameColumnAlias) {
         super(name, columns, dataProvider);
 
@@ -63,6 +61,10 @@ public class NodeSystemView<T> extends SystemView<T> {
             throw new IllegalArgumentException("Node name column alias can not 
be null or blank");
         }
 
+        if (columns.stream().anyMatch(c -> 
nodeNameColumnAlias.equals(c.name()))) {
+            throw new IllegalArgumentException("Node name column alias must 
distinct from column names");
+        }
+
         this.nodeNameColumnAlias = nodeNameColumnAlias;
     }
 
@@ -75,12 +77,6 @@ public class NodeSystemView<T> extends SystemView<T> {
         return nodeNameColumnAlias;
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public SystemViewType type() {
-        return SystemViewType.LOCAL;
-    }
-
     /** {@inheritDoc} */
     @Override
     public String toString() {
@@ -104,11 +100,12 @@ public class NodeSystemView<T> extends SystemView<T> {
         /**
          * Sets an alias for a node name column. Should only be set for node 
system views.
          *
-         * @param alias Node name column alias.
+         * @param alias Node name column alias. Must contain only latin 
letters, digits and underscore.
+         *      The first character must be a letter.
          * @return this.
          */
         public Builder<T> nodeNameColumnAlias(String alias) {
-            this.nodeNameColumnAlias = alias;
+            this.nodeNameColumnAlias = normalizeIdentifier(alias);
             return this;
         }
 
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemView.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemView.java
similarity index 78%
rename from 
modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemView.java
rename to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemView.java
index 8d9a65e55d..01dc430807 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemView.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemView.java
@@ -15,20 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.systemview;
+package org.apache.ignite.internal.systemview.api;
 
 import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.util.StringUtils.nullOrBlank;
 
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.Flow.Publisher;
 import java.util.function.Function;
 import java.util.function.Predicate;
-import java.util.function.Supplier;
-import 
org.apache.ignite.internal.catalog.descriptors.CatalogSystemViewDescriptor.SystemViewType;
+import java.util.regex.Pattern;
 import org.apache.ignite.internal.type.NativeType;
-import org.apache.ignite.internal.util.AsyncCursor;
-import org.apache.ignite.internal.util.StringUtils;
 
 /**
  * Base class for system view definitions.
@@ -56,12 +56,13 @@ import org.apache.ignite.internal.util.StringUtils;
  * @param <T> System view data type.
  */
 public abstract class SystemView<T> {
+    private static final Pattern LETTER_AND_UNDERSCORE = 
Pattern.compile("^[a-zA-Z][a-zA-Z0-9_]*");
 
     private final String name;
 
     private final List<SystemViewColumn<T, ?>> columns;
 
-    private final Supplier<AsyncCursor<T>> dataProvider;
+    private final Publisher<T> dataProvider;
 
     /**
      * Constructor.
@@ -72,9 +73,9 @@ public abstract class SystemView<T> {
      */
     SystemView(String name,
             List<SystemViewColumn<T, ?>> columns,
-            Supplier<AsyncCursor<T>> dataProvider) {
+            Publisher<T> dataProvider) {
 
-        if (StringUtils.nullOrBlank(name)) {
+        if (nullOrBlank(name)) {
             throw new IllegalArgumentException("Name can not be null or 
blank");
         }
 
@@ -122,15 +123,10 @@ public abstract class SystemView<T> {
      *
      * @return The data provider.
      */
-    public Supplier<AsyncCursor<T>> dataProvider() {
+    public Publisher<T> dataProvider() {
         return dataProvider;
     }
 
-    /**
-     * Returns the {@link SystemViewType type} of the system view.
-     */
-    public abstract SystemViewType type();
-
     /**
      * System view builder.
      *
@@ -144,7 +140,7 @@ public abstract class SystemView<T> {
 
         protected String name;
 
-        protected Supplier<AsyncCursor<T>> dataProvider;
+        protected Publisher<T> dataProvider;
 
         /** Constructor. */
         SystemViewBuilder() {
@@ -154,25 +150,25 @@ public abstract class SystemView<T> {
         /**
          * Sets view name.
          *
-         * @param name View name.
+         * @param name View name. Must contain only latin letters, digits and 
underscore. The first character must be a letter.
          * @return this.
          */
         public BuilderT name(String name) {
-            this.name = name;
+            this.name = normalizeIdentifier(name);
             return (BuilderT) this;
         }
 
         /**
          * Adds a column.
          *
-         * @param name Column name.
+         * @param name Column name. Must contain only latin letters, digits 
and underscore. The first character must be a letter.
          * @param type Type of a column value.
          * @param value Function that extracts value of this column from a 
system view data record.
          * @param <C> Type of a column value.
          * @return this.
          */
         public <C> BuilderT addColumn(String name, NativeType type, 
Function<T, C> value) {
-            columns.add(new SystemViewColumn<>(name, type, value));
+            columns.add(new SystemViewColumn<>(normalizeIdentifier(name), 
type, value));
             return (BuilderT) this;
         }
 
@@ -182,7 +178,7 @@ public abstract class SystemView<T> {
          * @param dataProvider Function that produces data for this view.
          * @return this.
          */
-        public BuilderT dataProvider(Supplier<AsyncCursor<T>> dataProvider) {
+        public BuilderT dataProvider(Publisher<T> dataProvider) {
             this.dataProvider = dataProvider;
             return (BuilderT) this;
         }
@@ -194,5 +190,17 @@ public abstract class SystemView<T> {
          */
         public abstract ViewT build();
     }
+
+    static String normalizeIdentifier(String identifier) {
+        if (nullOrBlank(identifier)) {
+            throw new IllegalArgumentException("Identifier must not be null or 
blank");
+        }
+
+        if (!LETTER_AND_UNDERSCORE.matcher(identifier).matches()) {
+            throw new IllegalArgumentException("Identifier must be 
alphanumeric with underscore and start with letter. Was: " + identifier);
+        }
+
+        return identifier.toUpperCase(Locale.ROOT);
+    }
 }
 
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewColumn.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewColumn.java
similarity index 98%
rename from 
modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewColumn.java
rename to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewColumn.java
index cd3c4a6b53..43c9e8101a 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewColumn.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewColumn.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.systemview;
+package org.apache.ignite.internal.systemview.api;
 
 import java.util.function.Function;
 import org.apache.ignite.internal.tostring.S;
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManager.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewManager.java
similarity index 83%
rename from 
modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManager.java
rename to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewManager.java
index 5ebdfa4121..fade863d0d 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManager.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewManager.java
@@ -15,27 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.systemview;
+package org.apache.ignite.internal.systemview.api;
 
 import java.util.List;
 import java.util.concurrent.Flow.Publisher;
 import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.manager.IgniteComponent;
-import org.apache.ignite.internal.schema.row.InternalTuple;
 
 /**
  * The system view manager is responsible for registering system views in the 
cluster.
  */
 public interface SystemViewManager extends IgniteComponent {
     /**
-     * Registers a system view.
+     * Registers a view provider into view manager.
      *
      * <p>Registration of views is completed when the system view manager 
starts. Therefore,
-     * it is necessary for other components to register the views before the 
manager is started.
+     * it is necessary for other components to register itself as a view 
provider before the manager is started.
      *
-     * @param view System view to register.
+     * @param viewProvider System view to register.
      */
-    void register(SystemView<?> view);
+    void register(SystemViewProvider viewProvider);
 
     /**
      * Returns a list of nodes a view with given name can be found on.
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewProvider.java
similarity index 65%
copy from 
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
copy to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewProvider.java
index 7453c899fb..ca51135fd7 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableDataSource.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViewProvider.java
@@ -15,15 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.sql.engine.exec;
+package org.apache.ignite.internal.systemview.api;
 
-import java.util.concurrent.Flow.Publisher;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import java.util.List;
 
 /**
- * Provides read operations over an abstract data source.
+ * Denotes a component that willing to expose system views.
+ *
+ * <p>Do not forget to register provider at {@link SystemViewManager} by 
calling
+ * {@link SystemViewManager#register(SystemViewProvider)} before the manager 
got started.
  */
 @FunctionalInterface
-public interface ScannableDataSource {
-    Publisher<InternalTuple> scan();
+public interface SystemViewProvider {
+    /** Returns a list of a system views to expose. */
+    List<SystemView<?>> systemViews();
 }
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViews.java
 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViews.java
similarity index 73%
rename from 
modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViews.java
rename to 
modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViews.java
index fe46a5d96c..1d9137e400 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViews.java
+++ 
b/modules/system-view-api/src/main/java/org/apache/ignite/internal/systemview/api/SystemViews.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.systemview;
+package org.apache.ignite.internal.systemview.api;
 
 /**
  * Provides static factory methods for system view builders.
@@ -32,6 +32,9 @@ public final class SystemViews {
     /**
      * Creates an instance of a builder to construct cluster-wide system views.
      *
+     * <p>Use this view to expose the data from a common source, like 
distributed meta storage,
+     * table, or anything, so every node will provide the same copy of dataset.
+     *
      * @param <T> Type of elements returned by a system view.
      * @return Returns a builder to construct cluster-wide system views.
      */
@@ -42,6 +45,11 @@ public final class SystemViews {
     /**
      * Creates an instance of a builder to construct node system views.
      *
+     * <p>Use this view to expose data that is unique to a particular node, 
and can be acquired
+     * only on the node itself. As example, this type of view is suitable to 
expose client connection to
+     * the node, or any metadata from task coordinator (like query 
coordinator, or tx coordinator;
+     * assuming there is no distributed registry for such kind of metadata).
+     *
      * @param <T> Type of elements returned by a system view.
      * @return Returns a builder to construct node system views.
      */
diff --git 
a/modules/system-view-api/src/test/java/org/apache/ignite/internal/systemview/api/SystemViewTest.java
 
b/modules/system-view-api/src/test/java/org/apache/ignite/internal/systemview/api/SystemViewTest.java
new file mode 100644
index 0000000000..644bd806f6
--- /dev/null
+++ 
b/modules/system-view-api/src/test/java/org/apache/ignite/internal/systemview/api/SystemViewTest.java
@@ -0,0 +1,403 @@
+/*
+ * 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.systemview.api;
+
+import static 
org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+
+import java.util.List;
+import java.util.concurrent.Flow.Publisher;
+import java.util.stream.Stream;
+import org.apache.ignite.internal.systemview.api.NodeSystemView.Builder;
+import org.apache.ignite.internal.type.NativeType;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+/**
+ * Tests for {@link SystemView}.
+ */
+@SuppressWarnings("ThrowableNotThrown")
+public class SystemViewTest {
+
+    /** Builds a cluster view. */
+    @Test
+    public void buildClusterView() {
+        Publisher<Dummy> dataProvider = dataProvider();
+
+        ClusterSystemView<Dummy> view = SystemViews.<Dummy>clusterViewBuilder()
+                .name("view")
+                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
+                .dataProvider(dataProvider)
+                .build();
+
+        assertEquals("VIEW", view.name(), "name");
+        assertEquals(2, view.columns().size(), "columns");
+
+        expectColumn(view.columns().get(0), "C1", NativeTypes.INT32);
+        expectColumn(view.columns().get(1), "C2", NativeTypes.INT64);
+
+        assertSame(dataProvider, view.dataProvider(), "data provider");
+    }
+
+    /** Builds a node view. */
+    @Test
+    public void buildNodeView() {
+        Publisher<Dummy> dataProvider = dataProvider();
+
+        NodeSystemView<Dummy> view = SystemViews.<Dummy>nodeViewBuilder()
+                .name("view")
+                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
+                .nodeNameColumnAlias("node_name")
+                .dataProvider(dataProvider)
+                .build();
+
+        assertEquals("VIEW", view.name(), "name");
+        assertEquals(2, view.columns().size(), "columns");
+
+        expectColumn(view.columns().get(0), "C1", NativeTypes.INT32);
+        expectColumn(view.columns().get(1), "C2", NativeTypes.INT64);
+
+        assertSame(dataProvider, view.dataProvider(), "data provider");
+        assertEquals("NODE_NAME", view.nodeNameColumnAlias(), "node name 
column alias");
+    }
+
+    /** Reject a node view without node name alias. */
+    @Test
+    public void rejectNodeViewWithoutNodeNameColumnAlias() {
+        assertThrowsWithCause(
+                () -> {
+                    SystemViews.<Dummy>nodeViewBuilder()
+                            .name("name")
+                            .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                            .dataProvider(dataProvider())
+                            .build();
+                },
+                IllegalArgumentException.class,
+                "Node name column alias can not be null or blank"
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("nullOrBlankNames")
+    public void rejectNodeViewWithBlankNodeNameColumnAlias(String name) {
+        assertThrowsWithCause(
+                () -> {
+                    SystemViews.<Dummy>nodeViewBuilder()
+                            .name("name")
+                            .nodeNameColumnAlias(name)
+                            .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                            .dataProvider(dataProvider())
+                            .build();
+                },
+                IllegalArgumentException.class,
+                "Identifier must not be null or blank"
+        );
+    }
+
+    @ParameterizedTest
+    @MethodSource("illegalCharsNames")
+    public void rejectNodeViewWithInvalidNodeNameColumnAlias(String name) {
+        assertThrowsWithCause(
+                () -> {
+                    SystemViews.<Dummy>nodeViewBuilder()
+                            .name("name")
+                            .nodeNameColumnAlias(name)
+                            .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                            .dataProvider(dataProvider())
+                            .build();
+                },
+                IllegalArgumentException.class,
+                "Identifier must be alphanumeric with underscore and start 
with letter"
+        );
+    }
+
+    @Test
+    public void rejectNodeViewIfColumnDuplicatesNodeNameAlias() {
+        assertThrowsWithCause(
+                () -> {
+                    SystemViews.<Dummy>nodeViewBuilder()
+                            .name("dummy")
+                            .nodeNameColumnAlias("c1")
+                            .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                            .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
+                            .dataProvider(dataProvider())
+                            .build();
+                },
+                IllegalArgumentException.class,
+                "Node name column alias must distinct from column names"
+        );
+    }
+
+    @Test
+    public void rejectNodeViewIfNodeNameAliasDuplicatesColumn() {
+        assertThrowsWithCause(
+                () -> {
+                    SystemViews.<Dummy>nodeViewBuilder()
+                            .name("dummy")
+                            .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                            .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
+                            .nodeNameColumnAlias("c1")
+                            .dataProvider(dataProvider())
+                            .build();
+                },
+                IllegalArgumentException.class,
+                "Node name column alias must distinct from column names"
+        );
+    }
+
+    /**
+     * Tests for {@link NodeSystemView.Builder}.
+     */
+    @Nested
+    public class NodeViewBuilderTest extends 
BuilderTest<NodeSystemView<Dummy>, Builder<Dummy>> {
+
+        @Override
+        protected NodeSystemView.Builder<Dummy> newBuilder() {
+            return SystemViews.nodeViewBuilder();
+        }
+    }
+
+    /**
+     * Tests for {@link ClusterSystemView.Builder}.
+     */
+    @Nested
+    public class ClusterViewBuilderTest extends 
BuilderTest<ClusterSystemView<Dummy>, ClusterSystemView.Builder<Dummy>> {
+
+        @Override
+        protected ClusterSystemView.Builder<Dummy> newBuilder() {
+            return SystemViews.clusterViewBuilder();
+        }
+    }
+
+    /**
+     * Common tests for view builder classes.
+     *
+     * @param <V> View type.
+     * @param <B> Builder type.
+     */
+    @SuppressWarnings({"ThrowableNotThrown", "DataFlowIssue"})
+    public abstract static class BuilderTest<V extends SystemView<Dummy>,
+            B extends SystemView.SystemViewBuilder<V, Dummy, B>> {
+
+        protected abstract B newBuilder();
+
+        /** Reject a view with {@code null} name. */
+        @ParameterizedTest
+        
@MethodSource("org.apache.ignite.internal.systemview.api.SystemViewTest#nullOrBlankNames")
+        public void rejectViewWithBlankName(String name) {
+            assertThrowsWithCause(() -> {
+                        newBuilder()
+                                .name(name)
+                                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Identifier must not be null or blank"
+            );
+        }
+
+        @ParameterizedTest
+        
@MethodSource("org.apache.ignite.internal.systemview.api.SystemViewTest#illegalCharsNames")
+        public void rejectViewWithInvalidName(String name) {
+            assertThrowsWithCause(
+                    () -> {
+                        newBuilder()
+                                .name(name)
+                                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Identifier must be alphanumeric with underscore and start 
with letter"
+            );
+        }
+
+        /** Reject a view without name. */
+        @Test
+        public void rejectViewWithUnspecifiedName() {
+            assertThrowsWithCause(
+                    () -> {
+                        newBuilder()
+                                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Name can not be null or blank"
+            );
+        }
+
+        /** Reject a view without columns. */
+        @Test
+        public void rejectViewWithoutColumns() {
+            assertThrowsWithCause(() -> {
+                        newBuilder()
+                                .name("dummy")
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Columns can not be empty"
+            );
+        }
+
+        /** Reject a view with duplicate column names. */
+        @Test
+        public void rejectViewWithDuplicateColumns() {
+            assertThrowsWithCause(
+                    () -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                                .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
+                                .addColumn("c1", NativeTypes.stringOf(16), (d) 
-> "3")
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Column names must be unique. Duplicates: [C1]"
+            );
+        }
+
+        /** Reject a view with {@code null} column name. */
+        @ParameterizedTest
+        
@MethodSource("org.apache.ignite.internal.systemview.api.SystemViewTest#nullOrBlankNames")
+        public void rejectViewWithBlankColumnName(String name) {
+            assertThrowsWithCause(() -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn(name, NativeTypes.INT32, (d) -> 0)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Identifier must not be null or blank"
+            );
+        }
+
+        /** Reject a view with invalid column name. */
+        @ParameterizedTest
+        
@MethodSource("org.apache.ignite.internal.systemview.api.SystemViewTest#illegalCharsNames")
+        public void rejectViewWithIllegalColumnName(String name) {
+            assertThrowsWithCause(
+                    () -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn(name, NativeTypes.INT32, (d) -> 0)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Identifier must be alphanumeric with underscore and start 
with letter"
+            );
+        }
+
+        /** Reject a view with {@code null} column type. */
+        @Test
+        public void rejectViewWithNullColumnType() {
+            assertThrowsWithCause(() -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn("c1", null, (d) -> 0)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Column type can not be null"
+            );
+        }
+
+        /** Reject a view with {@code null} column value function. */
+        @Test
+        public void rejectViewWithNullColumnFunction() {
+            assertThrowsWithCause(
+                    () -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn("c1", NativeTypes.INT32, null)
+                                .dataProvider(dataProvider())
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "Column value can not be null"
+            );
+        }
+
+        /** Reject a view without data provider. */
+        @Test
+        public void rejectViewWithoutDataProvider() {
+            assertThrowsWithCause(() -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "DataProvider can not be null"
+            );
+        }
+
+        /** Reject a view with {@code null} data provider. */
+        @Test
+        public void rejectViewWithNullDataProvider() {
+            assertThrowsWithCause(
+                    () -> {
+                        newBuilder()
+                                .name("dummy")
+                                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
+                                .dataProvider(null)
+                                .build();
+                    },
+                    IllegalArgumentException.class,
+                    "DataProvider can not be null"
+            );
+        }
+    }
+
+    private static void expectColumn(SystemViewColumn<?, ?> col, String name, 
NativeType type) {
+        assertEquals(name, col.name(), "name");
+        assertSame(type, col.type(), "type");
+        assertNotNull(col.value(), "value");
+    }
+
+    /** Dummy system view record. */
+    public static final class Dummy {
+
+    }
+
+    private static Publisher<Dummy> dataProvider() {
+        return SubscriptionUtils.fromIterable(List.of());
+    }
+
+    private static Stream<String> nullOrBlankNames() {
+        return Stream.of(null, "", " ", "  ");
+    }
+
+    private static Stream<String> illegalCharsNames() {
+        return Stream.of("ASASD!@#", "_ASD", "1C", "ASD,ASD");
+    }
+}
diff --git a/modules/system-view/build.gradle b/modules/system-view/build.gradle
index 27c48b030f..a865f1f1e9 100644
--- a/modules/system-view/build.gradle
+++ b/modules/system-view/build.gradle
@@ -21,37 +21,24 @@ apply from: "$rootDir/buildscripts/java-junit5.gradle"
 apply from: "$rootDir/buildscripts/java-test-fixtures.gradle"
 
 dependencies {
-    annotationProcessor project(':ignite-configuration-annotation-processor')
-    annotationProcessor libs.auto.service
+    implementation project(':ignite-system-view-api')
 
     implementation project(':ignite-api')
     implementation project(':ignite-core')
-    implementation project(':ignite-configuration')
-    implementation project(':ignite-metastorage-api')
-    implementation project(':ignite-vault')
     implementation project(':ignite-catalog')
     implementation project(':ignite-schema')
+    implementation project(':ignite-network-api')
     implementation project(':ignite-cluster-management')
 
     implementation libs.jetbrains.annotations
-    implementation libs.auto.service.annotations
 
     testImplementation(testFixtures(project(':ignite-core')))
     testImplementation(testFixtures(project(':ignite-metastorage')))
     testImplementation(testFixtures(project(':ignite-vault')))
     testImplementation(testFixtures(project(':ignite-schema')))
-    testImplementation project(':ignite-sql-engine')
     testImplementation libs.mockito.junit
     testImplementation libs.mockito.core
     testImplementation libs.hamcrest.core
-
-    testFixturesImplementation libs.jetbrains.annotations
-    testFixturesImplementation libs.mockito.core
-    testFixturesImplementation libs.mockito.junit
-    testFixturesImplementation libs.hamcrest.core
-    testFixturesImplementation(testFixtures(project(':ignite-core')))
-    testFixturesImplementation(testFixtures(project(':ignite-vault')))
-    testFixturesImplementation(testFixtures(project(':ignite-metastorage')))
 }
 
 description = 'ignite-system-view'
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManagerImpl.java
 
b/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManagerImpl.java
index c3d6e55aa9..d0711f6aab 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManagerImpl.java
+++ 
b/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/SystemViewManagerImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.systemview;
 
 import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
+import static 
org.apache.ignite.internal.systemview.utils.SystemViewUtils.tupleSchemaForView;
 import static org.apache.ignite.internal.util.IgniteUtils.inBusyLock;
 
 import java.util.ArrayList;
@@ -30,6 +31,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Flow.Publisher;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
+import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.catalog.CatalogCommand;
 import org.apache.ignite.internal.catalog.CatalogManager;
 import org.apache.ignite.internal.cluster.management.NodeAttributesProvider;
@@ -37,12 +39,20 @@ import 
org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
 import 
org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyEventListener;
 import 
org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
 import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.lang.NodeStoppingException;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
-import org.apache.ignite.internal.schema.row.InternalTuple;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTupleSchema;
+import org.apache.ignite.internal.systemview.api.NodeSystemView;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViewColumn;
+import org.apache.ignite.internal.systemview.api.SystemViewManager;
+import org.apache.ignite.internal.systemview.api.SystemViewProvider;
 import org.apache.ignite.internal.systemview.utils.SystemViewUtils;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.internal.util.subscription.TransformingPublisher;
 import org.apache.ignite.lang.ErrorGroups.Common;
 
 /**
@@ -77,6 +87,8 @@ public class SystemViewManagerImpl implements 
SystemViewManager, NodeAttributesP
     /** Future which is completed when system views are registered in the 
catalog. */
     private final CompletableFuture<Void> viewsRegistrationFuture = new 
CompletableFuture<>();
 
+    private volatile Map<String, ScannableView<?>> scannableViews = Map.of();
+
     private volatile Map<String, List<String>> owningNodesByViewName = 
Map.of();
 
     /** Creates a system view manager. */
@@ -98,6 +110,8 @@ public class SystemViewManagerImpl implements 
SystemViewManager, NodeAttributesP
                 return;
             }
 
+            scannableViews = toScannableViews(localNodeName, views);
+
             List<CatalogCommand> commands = views.values().stream()
                     .map(SystemViewUtils::toSystemViewCreateCommand)
                     .collect(Collectors.toList());
@@ -134,30 +148,23 @@ public class SystemViewManagerImpl implements 
SystemViewManager, NodeAttributesP
 
     @Override
     public Publisher<InternalTuple> scanView(String name) {
-        if (views.get(name) == null) {
+        ScannableView<?> scannableView = scannableViews.get(name);
+
+        if (scannableView == null) {
             throw new IgniteInternalException(
                     Common.INTERNAL_ERR,
                     format("View with name '{}' not found on node '{}'", name, 
localNodeName)
             );
         }
 
-        throw new 
UnsupportedOperationException("https://issues.apache.org/jira/browse/IGNITE-20578";);
+        return scannableView.scan();
     }
 
-    /** {@inheritDoc} */
     @Override
-    public void register(SystemView<?> view) {
-        if (views.containsKey(view.name())) {
-            throw new IllegalArgumentException(format("The view with name '{}' 
already registered", view.name()));
-        }
-
-        inBusyLock(busyLock, () -> {
-            if (startGuard.get()) {
-                throw new IllegalStateException(format("Unable to register 
view '{}', manager already started", view.name()));
-            }
+    public void register(SystemViewProvider viewProvider) {
+        List<SystemView<?>> views = viewProvider.systemViews();
 
-            views.put(view.name(), view);
-        });
+        views.forEach(this::registerView);
     }
 
     /** {@inheritDoc} */
@@ -188,6 +195,20 @@ public class SystemViewManagerImpl implements 
SystemViewManager, NodeAttributesP
         return viewsRegistrationFuture;
     }
 
+    private void registerView(SystemView<?> view) {
+        if (views.containsKey(view.name())) {
+            throw new IllegalArgumentException(format("The view with name '{}' 
already registered", view.name()));
+        }
+
+        inBusyLock(busyLock, () -> {
+            if (startGuard.get()) {
+                throw new IllegalStateException(format("Unable to register 
view '{}', manager already started", view.name()));
+            }
+
+            views.put(view.name(), view);
+        });
+    }
+
     private void processNewTopology(LogicalTopologySnapshot topology) {
         Map<String, List<String>> owningNodesByViewName = new HashMap<>();
 
@@ -215,4 +236,44 @@ public class SystemViewManagerImpl implements 
SystemViewManager, NodeAttributesP
 
         this.owningNodesByViewName = Map.copyOf(owningNodesByViewName);
     }
+
+    private static Map<String, ScannableView<?>> toScannableViews(String 
localNodeName, Map<String, SystemView<?>> views) {
+        Map<String, ScannableView<?>> scannableViews = new HashMap<>();
+
+        for (SystemView<?> view : views.values()) {
+            scannableViews.put(view.name(), new ScannableView<>(localNodeName, 
(SystemView<Object>) view));
+        }
+
+        return Map.copyOf(scannableViews);
+    }
+
+    private static class ScannableView<T> {
+        private final Publisher<InternalTuple> publisher;
+
+        private ScannableView(String localNodeName, SystemView<T> view) {
+            BinaryTupleSchema schema = tupleSchemaForView(view);
+
+            this.publisher = new TransformingPublisher<>(view.dataProvider(), 
object -> {
+                BinaryTupleBuilder builder = new 
BinaryTupleBuilder(schema.elementCount());
+
+                int offset = 0;
+                if (view instanceof NodeSystemView) {
+                    builder.appendString(localNodeName);
+                    offset++;
+                }
+
+                for (int i = 0; i < view.columns().size(); i++) {
+                    SystemViewColumn<T, ?> column = view.columns().get(i);
+
+                    schema.appendValue(builder, i + offset, 
column.value().apply(object));
+                }
+
+                return new BinaryTuple(schema.elementCount(), builder.build());
+            });
+        }
+
+        Publisher<InternalTuple> scan() {
+            return publisher;
+        }
+    }
 }
diff --git 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/utils/SystemViewUtils.java
 
b/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/utils/SystemViewUtils.java
index 48038dbae5..4b5de1032c 100644
--- 
a/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/utils/SystemViewUtils.java
+++ 
b/modules/system-view/src/main/java/org/apache/ignite/internal/systemview/utils/SystemViewUtils.java
@@ -24,13 +24,17 @@ import 
org.apache.ignite.internal.catalog.commands.ColumnParams;
 import org.apache.ignite.internal.catalog.commands.ColumnParams.Builder;
 import org.apache.ignite.internal.catalog.commands.CreateSystemViewCommand;
 import 
org.apache.ignite.internal.catalog.descriptors.CatalogSystemViewDescriptor.SystemViewType;
-import org.apache.ignite.internal.systemview.NodeSystemView;
-import org.apache.ignite.internal.systemview.SystemView;
-import org.apache.ignite.internal.systemview.SystemViewColumn;
+import org.apache.ignite.internal.schema.BinaryTupleSchema;
+import org.apache.ignite.internal.schema.BinaryTupleSchema.Element;
+import org.apache.ignite.internal.systemview.api.ClusterSystemView;
+import org.apache.ignite.internal.systemview.api.NodeSystemView;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViewColumn;
 import org.apache.ignite.internal.type.BitmaskNativeType;
 import org.apache.ignite.internal.type.DecimalNativeType;
 import org.apache.ignite.internal.type.NativeType;
 import org.apache.ignite.internal.type.NativeTypeSpec;
+import org.apache.ignite.internal.type.NativeTypes;
 import org.apache.ignite.internal.type.NumberNativeType;
 import org.apache.ignite.internal.type.TemporalNativeType;
 import org.apache.ignite.internal.type.VarlenNativeType;
@@ -40,7 +44,39 @@ import org.apache.ignite.sql.ColumnType;
  * System views utils.
  */
 public class SystemViewUtils {
-    private static final int NODE_NAME_FIELD_LENGTH = 
CatalogUtils.DEFAULT_VARLEN_LENGTH;
+    private static final int STRING_FIELD_LENGTH = 
CatalogUtils.DEFAULT_VARLEN_LENGTH;
+
+    /**
+     * Creates the {@link BinaryTupleSchema binary tuple schema} for the given 
system view definition.
+     *
+     * <p>The schema created will reflect the actual rows emitted by the scan 
publisher. That is, for
+     * node views it will include column representing node name at 0 position.
+     *
+     * @param view A definition of the view to create schema for.
+     * @return A schema representing rows of the given view.
+     */
+    public static BinaryTupleSchema tupleSchemaForView(SystemView<?> view) {
+        int viewColumn = view.columns().size();
+
+        boolean nodeView = view instanceof NodeSystemView;
+
+        Element[] elements = new Element[viewColumn + (nodeView ? 1 : 0)];
+
+        int offset = 0;
+        if (nodeView) {
+            // for node view we should inject column representing local node 
name at the very beginning
+            // of the tuple
+            elements[offset++] = new 
Element(NativeTypes.stringOf(STRING_FIELD_LENGTH), false);
+        }
+
+        for (int i = 0; i < viewColumn; i++) {
+            NativeType type = view.columns().get(i).type();
+
+            elements[i + offset] = new Element(type, true);
+        }
+
+        return BinaryTupleSchema.create(elements);
+    }
 
     /**
      * Converts {@link SystemViewColumn} to a {@link CreateSystemViewCommand 
catalog command} to create a system view.
@@ -48,15 +84,22 @@ public class SystemViewUtils {
     public static CreateSystemViewCommand 
toSystemViewCreateCommand(SystemView<?> view) {
         List<ColumnParams> columnParams = new 
ArrayList<>(view.columns().size());
 
-        if (view.type() == SystemViewType.LOCAL) {
+        SystemViewType viewType;
+        if (view instanceof NodeSystemView) {
             columnParams.add(
                     ColumnParams.builder()
                             .name(((NodeSystemView<?>) 
view).nodeNameColumnAlias())
                             .type(ColumnType.STRING)
-                            .length(NODE_NAME_FIELD_LENGTH)
+                            .length(STRING_FIELD_LENGTH)
                             .nullable(false)
                             .build()
             );
+
+            viewType = SystemViewType.NODE;
+        } else {
+            assert view instanceof ClusterSystemView : 
view.getClass().getCanonicalName();
+
+            viewType = SystemViewType.CLUSTER;
         }
 
         for (SystemViewColumn<?, ?> col : view.columns()) {
@@ -66,7 +109,7 @@ public class SystemViewUtils {
         return CreateSystemViewCommand.builder()
                 .name(view.name())
                 .columns(columnParams)
-                .type(view.type())
+                .type(viewType)
                 .build();
     }
 
diff --git 
a/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewManagerTest.java
 
b/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewManagerTest.java
index 2619670fb7..fbc36b3665 100644
--- 
a/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewManagerTest.java
+++ 
b/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewManagerTest.java
@@ -22,13 +22,17 @@ import static 
java.util.concurrent.CompletableFuture.failedFuture;
 import static 
org.apache.ignite.internal.systemview.SystemViewManagerImpl.NODE_ATTRIBUTES_KEY;
 import static 
org.apache.ignite.internal.systemview.SystemViewManagerImpl.NODE_ATTRIBUTES_LIST_SEPARATOR;
 import static 
org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.await;
 import static 
org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowFast;
 import static 
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe;
 import static org.apache.ignite.internal.util.CollectionUtils.first;
+import static org.apache.ignite.internal.util.SubscriptionUtils.fromIterable;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.aMapWithSize;
 import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -42,28 +46,34 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.Flow.Subscriber;
+import java.util.concurrent.Flow.Subscription;
 import java.util.function.Function;
 import org.apache.ignite.internal.catalog.CatalogManager;
 import org.apache.ignite.internal.catalog.CatalogValidationException;
 import org.apache.ignite.internal.cluster.management.topology.api.LogicalNode;
 import 
org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologySnapshot;
+import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.lang.NodeStoppingException;
 import org.apache.ignite.internal.schema.SchemaTestUtils;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
 import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.internal.type.NativeType;
 import org.apache.ignite.internal.type.NativeTypeSpec;
 import org.apache.ignite.internal.type.NativeTypes;
-import org.apache.ignite.internal.util.AsyncCursor;
 import org.apache.ignite.network.ClusterNode;
 import org.apache.ignite.network.ClusterNodeImpl;
 import org.apache.ignite.network.NetworkAddress;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.EnumSource;
-import org.mockito.InjectMocks;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.junit.jupiter.MockitoExtension;
@@ -73,12 +83,18 @@ import org.mockito.junit.jupiter.MockitoExtension;
  */
 @ExtendWith(MockitoExtension.class)
 public class SystemViewManagerTest extends BaseIgniteAbstractTest {
+    private static final String LOCAL_NODE_NAME = "LOCAL_NODE_NAME";
+
     @Mock
     private CatalogManager catalog;
 
-    @InjectMocks
     private SystemViewManagerImpl viewMgr;
 
+    @BeforeEach
+    void setUp() {
+        viewMgr = new SystemViewManagerImpl(LOCAL_NODE_NAME, catalog);
+    }
+
     @Test
     public void registerDuplicateNameFails() {
         String name = "testView1";
@@ -86,9 +102,9 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
         SystemView<?> view = dummyView(name);
         SystemView<?> viewWithSameName = dummyView(name);
 
-        viewMgr.register(view);
+        viewMgr.register(() -> List.of(view));
 
-        assertThrows(IllegalArgumentException.class, () -> 
viewMgr.register(viewWithSameName));
+        assertThrows(IllegalArgumentException.class, () -> viewMgr.register(() 
-> List.of(viewWithSameName)));
         verifyNoInteractions(catalog);
     }
 
@@ -96,7 +112,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
     public void registerAfterStartFails() {
         viewMgr.start();
 
-        assertThrows(IllegalStateException.class, () -> 
viewMgr.register(dummyView("test")));
+        assertThrows(IllegalStateException.class, () -> viewMgr.register(() -> 
List.of(dummyView("test"))));
         verifyNoInteractions(catalog);
     }
 
@@ -104,7 +120,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
     public void startAfterStartFails() {
         
Mockito.when(catalog.execute(anyList())).thenReturn(completedFuture(null));
 
-        viewMgr.register(dummyView("test"));
+        viewMgr.register(() -> List.of(dummyView("test")));
 
         viewMgr.start();
 
@@ -131,7 +147,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
 
         
Mockito.when(catalog.execute(anyList())).thenReturn(completedFuture(null));
 
-        viewMgr.register(dummyView("test", type));
+        viewMgr.register(() -> List.of(dummyView("test", type)));
         viewMgr.start();
 
         verify(catalog, only()).execute(anyList());
@@ -144,7 +160,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
 
         
Mockito.when(catalog.execute(anyList())).thenReturn(failedFuture(expected));
 
-        viewMgr.register(dummyView("test"));
+        viewMgr.register(() -> List.of(dummyView("test")));
 
         viewMgr.start();
 
@@ -160,8 +176,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
         String name1 = "view1";
         String name2 = "view2";
 
-        viewMgr.register(dummyView(name1));
-        viewMgr.register(dummyView(name2));
+        viewMgr.register(() -> List.of(dummyView(name1), dummyView(name2)));
 
         assertThat(viewMgr.nodeAttributes(), aMapWithSize(0));
 
@@ -170,7 +185,8 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
         verify(catalog, only()).execute(anyList());
         verifyNoMoreInteractions(catalog);
 
-        assertThat(viewMgr.nodeAttributes(), is(Map.of(NODE_ATTRIBUTES_KEY, 
String.join(NODE_ATTRIBUTES_LIST_SEPARATOR, name1, name2))));
+        assertThat(viewMgr.nodeAttributes(), is(Map.of(NODE_ATTRIBUTES_KEY, 
String.join(NODE_ATTRIBUTES_LIST_SEPARATOR, name1.toUpperCase(
+                Locale.ROOT), name2.toUpperCase(Locale.ROOT)))));
     }
 
     @Test
@@ -193,7 +209,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
         viewMgr.stop();
 
         //noinspection ThrowableNotThrown
-        assertThrowsWithCause(() -> viewMgr.register(dummyView("test")), 
NodeStoppingException.class);
+        assertThrowsWithCause(() -> viewMgr.register(() -> 
List.of(dummyView("test"))), NodeStoppingException.class);
     }
 
     @Test
@@ -230,6 +246,75 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
         assertThat(viewMgr.owningNodes(viewName), hasItem("C"));
     }
 
+    @Test
+    void viewScanTest() {
+        
Mockito.when(catalog.execute(anyList())).thenReturn(completedFuture(null));
+
+        String nodeView = "NODE_VIEW";
+        String clusterView = "CLUSTER_VIEW";
+
+        class Pojo {
+            private final int c1;
+            private final int c2;
+
+            private Pojo(int c1, int c2) {
+                this.c1 = c1;
+                this.c2 = c2;
+            }
+        }
+
+        Publisher<Pojo> dataSet = fromIterable(List.of(new Pojo(1, 1), new 
Pojo(2, 2)));
+
+        viewMgr.register(() -> List.of(
+                SystemViews.<Pojo>nodeViewBuilder()
+                        .name(nodeView)
+                        .nodeNameColumnAlias("NODE")
+                        .addColumn("C1", NativeTypes.INT32, p -> p.c1)
+                        .addColumn("C2", NativeTypes.INT32, p -> p.c2)
+                        .dataProvider(dataSet)
+                        .build(),
+                SystemViews.<Pojo>clusterViewBuilder()
+                        .name(clusterView)
+                        .addColumn("C1", NativeTypes.INT32, p -> p.c1)
+                        .addColumn("C2", NativeTypes.INT32, p -> p.c2)
+                        .dataProvider(dataSet)
+                        .build()
+        ));
+
+        viewMgr.start();
+
+        {
+            DrainAllSubscriber<InternalTuple> subs = new 
DrainAllSubscriber<>();
+
+            viewMgr.scanView(clusterView).subscribe(subs);
+
+            List<InternalTuple> entries = await(subs.completion);
+
+            assertThat(entries, hasSize(2));
+            assertThat(entries.get(0).intValue(0), equalTo(1));
+            assertThat(entries.get(0).intValue(1), equalTo(1));
+            assertThat(entries.get(1).intValue(0), equalTo(2));
+            assertThat(entries.get(1).intValue(1), equalTo(2));
+        }
+
+        {
+            DrainAllSubscriber<InternalTuple> subs = new 
DrainAllSubscriber<>();
+
+            viewMgr.scanView(nodeView).subscribe(subs);
+
+            List<InternalTuple> entries = await(subs.completion);
+
+            assertThat(entries, hasSize(2));
+            assertThat(entries.get(0).stringValue(0), 
equalTo(LOCAL_NODE_NAME));
+            assertThat(entries.get(0).intValue(1), equalTo(1));
+            assertThat(entries.get(0).intValue(2), equalTo(1));
+            assertThat(entries.get(1).stringValue(0), 
equalTo(LOCAL_NODE_NAME));
+            assertThat(entries.get(1).intValue(1), equalTo(2));
+            assertThat(entries.get(1).intValue(2), equalTo(2));
+        }
+
+    }
+
     private static SystemView<?> dummyView(String name) {
         return dummyView(name, NativeTypes.INT32);
     }
@@ -239,17 +324,7 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
                 .nodeNameColumnAlias("NODE")
                 .name(name)
                 .addColumn("c1", type, (Function<Object, T>) 
Function.identity())
-                .dataProvider(() -> new AsyncCursor<>() {
-                    @Override
-                    public CompletableFuture<BatchedResult<Object>> 
requestNextAsync(int rows) {
-                        return completedFuture(null);
-                    }
-
-                    @Override
-                    public CompletableFuture<Void> closeAsync() {
-                        return completedFuture(null);
-                    }
-                })
+                .dataProvider(fromIterable(List.of()))
                 .build();
     }
 
@@ -279,4 +354,30 @@ public class SystemViewManagerTest extends 
BaseIgniteAbstractTest {
 
         return new LogicalTopologySnapshot(1, topology);
     }
+
+    static class DrainAllSubscriber<T> implements Subscriber<T> {
+        private final List<T> entries = new ArrayList<>();
+
+        CompletableFuture<List<T>> completion = new CompletableFuture<>();
+
+        @Override
+        public void onSubscribe(Subscription subscription) {
+            subscription.request(Long.MAX_VALUE);
+        }
+
+        @Override
+        public void onNext(T item) {
+            entries.add(item);
+        }
+
+        @Override
+        public void onError(Throwable throwable) {
+            completion.completeExceptionally(throwable);
+        }
+
+        @Override
+        public void onComplete() {
+            completion.complete(entries);
+        }
+    }
 }
diff --git 
a/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewTest.java
 
b/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewTest.java
deleted file mode 100644
index 7e504ece51..0000000000
--- 
a/modules/system-view/src/test/java/org/apache/ignite/internal/systemview/SystemViewTest.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.systemview;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertSame;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Supplier;
-import java.util.stream.Stream;
-import org.apache.ignite.internal.systemview.NodeSystemView.Builder;
-import org.apache.ignite.internal.type.NativeType;
-import org.apache.ignite.internal.type.NativeTypes;
-import org.apache.ignite.internal.util.AsyncCursor;
-import org.junit.jupiter.api.Nested;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.function.Executable;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.MethodSource;
-
-/**
- * Tests for {@link SystemView}.
- */
-public class SystemViewTest {
-
-    /** Builds a cluster view. */
-    @Test
-    public void buildClusterView() {
-        Supplier<AsyncCursor<Dummy>> dataProvider = dataProvider();
-
-        ClusterSystemView<Dummy> view = SystemViews.<Dummy>clusterViewBuilder()
-                .name("view")
-                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
-                .dataProvider(dataProvider)
-                .build();
-
-        assertEquals("view", view.name(), "name");
-        assertEquals(2, view.columns().size(), "columns");
-
-        expectColumn(view.columns().get(0), "c1", NativeTypes.INT32);
-        expectColumn(view.columns().get(1), "c2", NativeTypes.INT64);
-
-        assertSame(dataProvider, view.dataProvider(), "data provider");
-    }
-
-    /** Builds a node view. */
-    @Test
-    public void buildNodeView() {
-        Supplier<AsyncCursor<Dummy>> dataProvider = dataProvider();
-
-        NodeSystemView<Dummy> view = SystemViews.<Dummy>nodeViewBuilder()
-                .name("view")
-                .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
-                .nodeNameColumnAlias("node_name")
-                .dataProvider(dataProvider)
-                .build();
-
-        assertEquals("view", view.name(), "name");
-        assertEquals(2, view.columns().size(), "columns");
-
-        expectColumn(view.columns().get(0), "c1", NativeTypes.INT32);
-        expectColumn(view.columns().get(1), "c2", NativeTypes.INT64);
-
-        assertSame(dataProvider, view.dataProvider(), "data provider");
-        assertEquals("node_name", view.nodeNameColumnAlias(), "node name 
column alias");
-    }
-
-    /** Reject a node view without node name alias. */
-    @Test
-    public void rejectNodeViewWithoutNodeNameColumnAlias() {
-        expectThrows(IllegalArgumentException.class, () -> {
-            SystemViews.<Dummy>nodeViewBuilder()
-                    .name("name")
-                    .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                    .dataProvider(dataProvider())
-                    .build();
-        }, "Node name column alias can not be null or blank");
-    }
-
-    /**
-     * Tests for {@link NodeSystemView.Builder}.
-     */
-    @Nested
-    public class NodeViewBuilderTest extends 
BuilderTest<NodeSystemView<Dummy>, Builder<Dummy>> {
-
-        @Override
-        protected NodeSystemView.Builder<Dummy> newBuilder() {
-            return SystemViews.nodeViewBuilder();
-        }
-    }
-
-    /**
-     * Tests for {@link ClusterSystemView.Builder}.
-     */
-    @Nested
-    public class ClusterViewBuilderTest extends 
BuilderTest<ClusterSystemView<Dummy>, ClusterSystemView.Builder<Dummy>> {
-
-        @Override
-        protected ClusterSystemView.Builder<Dummy> newBuilder() {
-            return SystemViews.clusterViewBuilder();
-        }
-    }
-
-    /**
-     * Common tests for view builder classes.
-     *
-     * @param <V> View type.
-     * @param <B> Builder type.
-     */
-    public abstract static class BuilderTest<V extends SystemView<Dummy>,
-            B extends SystemView.SystemViewBuilder<V, Dummy, B>> {
-
-        protected abstract B newBuilder();
-
-        /** Reject a view with {@code null} name. */
-        @ParameterizedTest
-        @MethodSource("invalidNames")
-        public void rejectViewWithNullName(String name) {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name(name)
-                        .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Name can not be null or blank");
-        }
-
-        /** Reject a view without name. */
-        @Test
-        public void rejectViewWithUnspecifiedName() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Name can not be null or blank");
-        }
-
-        /** Reject a view without columns. */
-        @Test
-        public void rejectViewWithoutColumns() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Columns can not be empty");
-        }
-
-        /** Reject a view without columns. */
-        @Test
-        public void rejectViewWithDuplicateColumns() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                        .addColumn("c2", NativeTypes.INT64, (d) -> 1L)
-                        .addColumn("c1", NativeTypes.stringOf(16), (d) -> "3")
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Column names must be unique. Duplicates: [c1]");
-        }
-
-        /** Reject a view with {@code null} column name. */
-        @ParameterizedTest
-        @MethodSource("invalidNames")
-        public void rejectViewWithNullColumnName(String name) {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .addColumn(name, NativeTypes.INT32, (d) -> 0)
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Column name can not be null or blank");
-        }
-
-        /** Reject a view with {@code null} column type. */
-        @Test
-        public void rejectViewWithNullColumnType() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .addColumn("c1", null, (d) -> 0)
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Column type can not be null");
-        }
-
-        /** Reject a view with {@code null} column value function. */
-        @Test
-        public void rejectViewWithNullColumnFunction() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .addColumn("c1", NativeTypes.INT32, null)
-                        .dataProvider(dataProvider())
-                        .build();
-            }, "Column value can not be null");
-        }
-
-        /** Reject a view without data provider. */
-        @Test
-        public void rejectViewWithoutDataProvider() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                        .build();
-            }, "DataProvider can not be null");
-        }
-
-        /** Reject a view with {@code null} data provider. */
-        @Test
-        public void rejectViewWithoutNullDataProvider() {
-            expectThrows(IllegalArgumentException.class, () -> {
-                newBuilder()
-                        .name("dummy")
-                        .addColumn("c1", NativeTypes.INT32, (d) -> 0)
-                        .dataProvider(null)
-                        .build();
-            }, "DataProvider can not be null");
-        }
-
-        static Stream<String> invalidNames() {
-            return Stream.of(null, "", " ", "  ");
-        }
-    }
-
-    private static void expectColumn(SystemViewColumn<?, ?> col, String name, 
NativeType type) {
-        assertEquals(name, col.name(), "name");
-        assertSame(type, col.type(), "type");
-        assertNotNull(col.value(), "value");
-    }
-
-    /** Dummy system view record. */
-    public static final class Dummy {
-
-    }
-
-    private static void expectThrows(Class<? extends RuntimeException> 
errorClass, Executable action, String errorMessage) {
-        RuntimeException t = assertThrows(errorClass, action);
-        assertEquals(errorMessage, t.getMessage(), "error message");
-    }
-
-    private static Supplier<AsyncCursor<Dummy>> dataProvider() {
-        return () -> new AsyncCursor<>() {
-            @Override
-            public CompletableFuture<BatchedResult<Dummy>> 
requestNextAsync(int rows) {
-                return CompletableFuture.completedFuture(new 
BatchedResult<>(List.of(), false));
-            }
-
-            @Override
-            public CompletableFuture<Void> closeAsync() {
-                return CompletableFuture.completedFuture(null);
-            }
-        };
-    }
-}
diff --git a/modules/table/build.gradle b/modules/table/build.gradle
index 829bf2b615..b71e1c14e1 100644
--- a/modules/table/build.gradle
+++ b/modules/table/build.gradle
@@ -59,6 +59,7 @@ dependencies {
     testImplementation project(':ignite-page-memory')
     testImplementation project(':ignite-storage-rocksdb')
     testImplementation project(':ignite-placement-driver-api')
+    testImplementation project(':ignite-system-view-api')
     testImplementation(testFixtures(project(':ignite-core')))
     testImplementation(testFixtures(project(':ignite-schema')))
     testImplementation(testFixtures(project(':ignite-configuration')))
diff --git a/settings.gradle b/settings.gradle
index 194ad55fb0..caa3c80365 100644
--- a/settings.gradle
+++ b/settings.gradle
@@ -74,6 +74,7 @@ include(':ignite-catalog')
 include(":ignite-arch-test")
 include(":ignite-file-transfer")
 include(":ignite-system-view")
+include(":ignite-system-view-api")
 
 project(":ignite-examples").projectDir = file('examples')
 project(":ignite-page-memory").projectDir = file('modules/page-memory')
@@ -133,6 +134,7 @@ project(":ignite-catalog").projectDir = 
file('modules/catalog')
 project(":ignite-arch-test").projectDir = file('modules/arch-test')
 project(":ignite-file-transfer").projectDir = file('modules/file-transfer')
 project(":ignite-system-view").projectDir = file('modules/system-view')
+project(":ignite-system-view-api").projectDir = file('modules/system-view-api')
 
 ext.isCiServer = System.getenv().containsKey("IGNITE_CI")
 

Reply via email to