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

mpochatkin 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 ea60e9de8e IGNITE-22152 Support partition filter in SQL (#4091)
ea60e9de8e is described below

commit ea60e9de8ec5d5d26e0be2c850f4e02c506a9fae
Author: Mikhail <[email protected]>
AuthorDate: Thu Jul 25 13:50:38 2024 +0300

    IGNITE-22152 Support partition filter in SQL (#4091)
---
 .../ignite/table/criteria/CriteriaVisitor.java     |   9 -
 .../ignite/table/criteria/PartitionCriteria.java   |  32 --
 .../internal/table/criteria/ColumnValidator.java   |   6 -
 .../internal/table/criteria/SqlSerializer.java     |   7 -
 .../internal/sql/engine/ItCreateTableDdlTest.java  |  78 ++++-
 .../ignite/internal/sql/engine/ItDmlTest.java      |  11 +
 .../ignite/internal/sql/engine/ItMetadataTest.java |   2 +
 .../internal/sql/engine/ItSecondaryIndexTest.java  |   2 +-
 ...rom.test => rename_columns_in_from.test_ignore} |   1 +
 .../sql/syscolumns/system_columns.test             |  48 +++
 .../engine/exec/ExecutableTableRegistryImpl.java   |   2 +-
 .../sql/engine/exec/LogicalRelImplementor.java     |   1 +
 .../exec/ProjectedTableRowConverterImpl.java       |  30 +-
 .../sql/engine/exec/ScannableTableImpl.java        |  26 +-
 .../sql/engine/exec/TableRowConverterFactory.java  |   4 +
 .../engine/exec/TableRowConverterFactoryImpl.java  |  46 ++-
 .../internal/sql/engine/exec/UpdatableTable.java   |   4 +-
 .../sql/engine/exec/UpdatableTableImpl.java        |  11 +-
 .../internal/sql/engine/exec/VirtualColumn.java    |  61 ++++
 .../internal/sql/engine/exec/rel/ModifyNode.java   |  14 +-
 .../sql/engine/prepare/IgniteSqlValidator.java     |  53 +++-
 .../prepare/ddl/DdlSqlToCommandConverter.java      |  18 ++
 .../sql/engine/schema/ColumnDescriptor.java        |   5 +
 .../sql/engine/schema/ColumnDescriptorImpl.java    |  10 +
 .../internal/sql/engine/schema/IgniteTable.java    |   8 +
 .../sql/engine/schema/IgniteTableImpl.java         |  17 +-
 .../sql/engine/schema/SqlSchemaManagerImpl.java    |  53 ++--
 .../sql/engine/schema/TableDescriptorImpl.java     |  25 +-
 .../sql/engine/util/AbstractProjectedTuple.java    |   4 +-
 .../ignite/internal/sql/engine/util/Commons.java   |   1 +
 .../ExtendedFieldDeserializingProjectedTuple.java  | 329 +++++++++++++++++++++
 .../util/FieldDeserializingProjectedTuple.java     |   2 +-
 .../internal/sql/engine/util/RowTypeUtils.java     |  69 +++++
 .../exec/ExecutableTableRegistrySelfTest.java      |   2 +
 .../exec/ProjectedTableRowConverterSelfTest.java   |   3 +-
 .../sql/engine/exec/TableRowConverterSelfTest.java |   5 +-
 .../engine/exec/rel/ModifyNodeExecutionTest.java   |   3 +-
 .../sql/engine/framework/TestBuilders.java         |  21 +-
 .../planner/CorrelatedSubqueryPlannerTest.java     |   2 +
 .../internal/sql/engine/planner/PlannerTest.java   |  23 +-
 .../sql/engine/prepare/TypeCoercionTest.java       |   5 +
 .../engine/schema/SqlSchemaManagerImplTest.java    |   5 +-
 .../internal/sql/engine/util/QueryCheckerTest.java |   2 +-
 .../resources/mapping/test_partition_pruning.test  |  24 --
 44 files changed, 916 insertions(+), 168 deletions(-)

diff --git 
a/modules/api/src/main/java/org/apache/ignite/table/criteria/CriteriaVisitor.java
 
b/modules/api/src/main/java/org/apache/ignite/table/criteria/CriteriaVisitor.java
index d8fe77f404..a40742b0cb 100644
--- 
a/modules/api/src/main/java/org/apache/ignite/table/criteria/CriteriaVisitor.java
+++ 
b/modules/api/src/main/java/org/apache/ignite/table/criteria/CriteriaVisitor.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.table.criteria;
 
-import org.apache.ignite.table.partition.Partition;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -50,14 +49,6 @@ public interface CriteriaVisitor<C> {
      */
     <T> void visit(Expression expression, @Nullable C context);
 
-    /**
-     * Visit a {@link Partition} instance with the given context.
-     *
-     * @param partition Partition to visit.
-     * @param context context of the visit or {@code null}, if not used.
-     */
-    void visit(PartitionCriteria partition, @Nullable C context);
-
     /**
      * Visit a {@link Criteria} instance with the given context.
      *
diff --git 
a/modules/api/src/main/java/org/apache/ignite/table/criteria/PartitionCriteria.java
 
b/modules/api/src/main/java/org/apache/ignite/table/criteria/PartitionCriteria.java
deleted file mode 100644
index 3f6ff0e5ef..0000000000
--- 
a/modules/api/src/main/java/org/apache/ignite/table/criteria/PartitionCriteria.java
+++ /dev/null
@@ -1,32 +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.table.criteria;
-
-import org.apache.ignite.table.partition.Partition;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Represents a partition reference for criteria query.
- */
-// TODO: IGNITE-22153
-public class PartitionCriteria implements Partition, Criteria {
-    @Override
-    public <C> void accept(CriteriaVisitor<C> v, @Nullable C context) {
-        v.visit(this, context);
-    }
-}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/ColumnValidator.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/ColumnValidator.java
index b94e116de9..3ae839861e 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/ColumnValidator.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/ColumnValidator.java
@@ -25,7 +25,6 @@ import org.apache.ignite.table.criteria.Criteria;
 import org.apache.ignite.table.criteria.CriteriaVisitor;
 import org.apache.ignite.table.criteria.Expression;
 import org.apache.ignite.table.criteria.Parameter;
-import org.apache.ignite.table.criteria.PartitionCriteria;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -58,11 +57,6 @@ class ColumnValidator implements 
CriteriaVisitor<Collection<String>> {
         }
     }
 
-    @Override
-    public void visit(PartitionCriteria partition, @Nullable 
Collection<String> context) {
-        // No-op.
-    }
-
     @Override
     public <T> void visit(Criteria criteria, @Nullable Collection<String> 
context) {
         criteria.accept(this, context);
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/SqlSerializer.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/SqlSerializer.java
index ea50c90e22..9eedbc603e 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/SqlSerializer.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/table/criteria/SqlSerializer.java
@@ -37,7 +37,6 @@ import org.apache.ignite.table.criteria.CriteriaVisitor;
 import org.apache.ignite.table.criteria.Expression;
 import org.apache.ignite.table.criteria.Operator;
 import org.apache.ignite.table.criteria.Parameter;
-import org.apache.ignite.table.criteria.PartitionCriteria;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -125,12 +124,6 @@ public class SqlSerializer implements 
CriteriaVisitor<Void> {
         }
     }
 
-    @Override
-    // TODO: IGNITE-22153
-    public void visit(PartitionCriteria partition, @Nullable Void context) {
-        throw new UnsupportedOperationException("This operation doesn't 
implemented yet.");
-    }
-
     /** {@inheritDoc} */
     @Override
     public <T> void visit(Criteria criteria, @Nullable Void context) {
diff --git 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
index 7781bcbb70..41baf10ce0 100644
--- 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
+++ 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItCreateTableDdlTest.java
@@ -41,6 +41,8 @@ import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
 import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.sql.BaseSqlIntegrationTest;
+import org.apache.ignite.internal.table.partition.HashPartition;
+import org.apache.ignite.internal.testframework.WithSystemProperty;
 import org.apache.ignite.sql.SqlException;
 import org.apache.ignite.table.Table;
 import org.apache.ignite.table.Tuple;
@@ -183,7 +185,7 @@ public class ItCreateTableDdlTest extends 
BaseSqlIntegrationTest {
     }
 
     /**
-     * Check implicit colocation columns configuration (defined by PK)..
+     * Check implicit colocation columns configuration (defined by PK).
      */
     @Test
     public void implicitColocationColumns() {
@@ -196,6 +198,76 @@ public class ItCreateTableDdlTest extends 
BaseSqlIntegrationTest {
         assertEquals("ID0", colocationColumns.get(1).name());
     }
 
+    @Test
+    @WithSystemProperty(key = "IMPLICIT_PK_ENABLED", value = "true")
+    public void reservedColumnNames() {
+        assertThrowsSqlException(
+                STMT_VALIDATION_ERR,
+                "Failed to validate query. Column '__p_key' is reserved name.",
+                () -> sql("CREATE TABLE T0(\"__p_key\" INT PRIMARY KEY, VAL 
INT)")
+        );
+
+        assertThrowsSqlException(
+                STMT_VALIDATION_ERR,
+                "Failed to validate query. Column '__part' is reserved name.",
+                () -> sql("CREATE TABLE T0(\"__part\" INT PRIMARY KEY, VAL 
INT)")
+        );
+
+        sql("CREATE TABLE T0(id INT PRIMARY KEY)");
+
+        assertThrowsSqlException(
+                STMT_VALIDATION_ERR,
+                "Failed to validate query. Column '__p_key' is reserved name.",
+                () -> sql("ALTER TABLE T0 ADD COLUMN \"__p_key\" INT")
+        );
+
+        assertThrowsSqlException(
+                STMT_VALIDATION_ERR,
+                "Failed to validate query. Column '__part' is reserved name.",
+                () -> sql("ALTER TABLE T0 ADD COLUMN \"__part\" INT")
+        );
+    }
+
+    /**
+     * Check implicit partition column configuration (defined by PK).
+     */
+    @Test
+    public void implicitPartitionColumn() throws Exception {
+        sql("CREATE TABLE T0(ID BIGINT PRIMARY KEY, VAL VARCHAR)");
+
+        List<Column> columns = 
unwrapTableViewInternal(table("T0")).schemaView().lastKnownSchema().columns();
+
+        assertEquals(2, columns.size());
+        assertEquals("ID", columns.get(0).name());
+        assertEquals("VAL", columns.get(1).name());
+
+        Tuple key1 = Tuple.create().set("id", 101L);
+        Tuple key2 = Tuple.create().set("id", 102L);
+
+        // Add data
+        sql("insert into t0 values (101, 'v1')");
+
+        Table table = CLUSTER.node(0).tables().table("T0");
+        table.keyValueView().put(null, Tuple.create().set("id", 102L), 
Tuple.create().set("val", "v2"));
+
+        assertEquals(Tuple.create().set("val", "v1"), 
table.keyValueView().get(null, Tuple.create().set("id", 101L)));
+
+        assertQuery("SELECT * FROM t0")
+                .returns(101L, "v1")
+                .returns(102L, "v2")
+                .check();
+
+        assertQuery("SELECT \"__part\" FROM t0")
+                .returns(partitionForKey(table, key1))
+                .returns(partitionForKey(table, key2))
+                .check();
+
+        assertQuery("SELECT \"__part\", id FROM t0")
+                .returns(partitionForKey(table, key1), 101L)
+                .returns(partitionForKey(table, key2), 102L)
+                .check();
+    }
+
     /**
      * Check explicit colocation columns configuration.
      */
@@ -484,4 +556,8 @@ public class ItCreateTableDdlTest extends 
BaseSqlIntegrationTest {
 
         return Objects.requireNonNull(catalog.defaultZone());
     }
+
+    private static int partitionForKey(Table table, Tuple keyTuple) throws 
Exception {
+        return ((HashPartition) 
table.partitionManager().partitionAsync(keyTuple).get()).partitionId();
+    }
 }
diff --git 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
index cada2e17b0..895ea74f1e 100644
--- 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
+++ 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItDmlTest.java
@@ -493,6 +493,17 @@ public class ItDmlTest extends BaseSqlIntegrationTest {
         assertEquals(3, pkVals.size());
     }
 
+    @Test
+    @WithSystemProperty(key = "IMPLICIT_PK_ENABLED", value = "true")
+    public void invalidAliases() {
+        sql("CREATE TABLE T(VAL INT)");
+
+        assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, "Illegal alias. 
__p_key is reserved name",
+                () -> sql("select val as \"__p_key\" from t"));
+        assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, "Illegal alias. 
__part is reserved name",
+                () -> sql("select val as \"__part\" from t"));
+    }
+
     private static Stream<DefaultValueArg> defaultValueArgs() {
         Stream<DefaultValueArg> vals = Stream.of(
                 new DefaultValueArg("BOOLEAN", "TRUE", Boolean.TRUE),
diff --git 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java
 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java
index 318f759e62..217aed0c61 100644
--- 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java
+++ 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMetadataTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.sql.ColumnType;
 import org.apache.ignite.sql.ResultSet;
 import org.apache.ignite.sql.SqlRow;
 import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
@@ -89,6 +90,7 @@ public class ItMetadataTest extends BaseSqlIntegrationTest {
     }
 
     @Test
+    @Disabled("IGNITE-22772")
     public void renameColumnsInFrom() {
         assertQuery("select NEW_PERSON.NEW_ID, NEW_NAME, NEW_persON.New_salary 
from person NEW_PERSON(NeW_Id, NeW_NaMe, New_SaLaRy)")
                 .columnNames("NEW_ID", "NEW_NAME", "NEW_SALARY").check();
diff --git 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
index cbbe218bab..af238b68c2 100644
--- 
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
+++ 
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
@@ -838,7 +838,7 @@ public class ItSecondaryIndexTest extends 
BaseSqlIntegrationTest {
 
         // Not nullable column, filter is always - false.
         assertQuery("SELECT * FROM T1 WHERE id IS NULL")
-                .matches(QueryChecker.matches(".*filters=\\[false\\].*"))
+                .matches(QueryChecker.matches(".*tuples=\\[\\[\\]\\].*"))
                 .check();
     }
 
diff --git 
a/modules/sql-engine/src/integrationTest/sql/basic_queries/rename_columns_in_from.test
 
b/modules/sql-engine/src/integrationTest/sql/basic_queries/rename_columns_in_from.test_ignore
similarity index 95%
rename from 
modules/sql-engine/src/integrationTest/sql/basic_queries/rename_columns_in_from.test
rename to 
modules/sql-engine/src/integrationTest/sql/basic_queries/rename_columns_in_from.test_ignore
index 702f5e7c79..130f934d00 100644
--- 
a/modules/sql-engine/src/integrationTest/sql/basic_queries/rename_columns_in_from.test
+++ 
b/modules/sql-engine/src/integrationTest/sql/basic_queries/rename_columns_in_from.test_ignore
@@ -2,6 +2,7 @@
 # description: SQL feature E051-9 (Basic query specification. Rename columns 
in the FROM clause)
 # feature: E051-09
 # group: [basic_queries]
+# Ignore: https://issues.apache.org/jira/browse/IGNITE-22772
 
 statement ok
 CREATE TABLE tab ( A INT PRIMARY KEY, B INT )
diff --git 
a/modules/sql-engine/src/integrationTest/sql/syscolumns/system_columns.test 
b/modules/sql-engine/src/integrationTest/sql/syscolumns/system_columns.test
new file mode 100644
index 0000000000..a7bc0c296e
--- /dev/null
+++ b/modules/sql-engine/src/integrationTest/sql/syscolumns/system_columns.test
@@ -0,0 +1,48 @@
+# name: test/sql/basic_queries/system_columns.test
+# description:  Test Ignite 3 system columns.
+# group: [basic_queries]
+
+statement ok
+CREATE TABLE T0( ID BIGINT PRIMARY KEY, VAL VARCHAR );
+
+# Add
+statement ok
+insert into t0 values (101, 'val1'), (102, 'val2');
+
+statement ok
+insert into t0 values (103, 'val3');
+
+# statement error: Failed to validate query:
+statement error
+----
+insert into t0 (id, val, "__part") values (104, 'val4', 1)
+
+
+# Select partition system column
+query IT rowsort
+select * from t0
+----
+101    val1
+102    val2
+103    val3
+
+query II rowsort
+select "__part", id from t0
+----
+19     101
+17     102
+11     103
+
+query I rowsort
+select "__part" from t0
+----
+19
+17
+11
+
+
+# Partition system column in WHERE
+query IT
+select * from t0 WHERE "__part" = 17
+----
+102    val2
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistryImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistryImpl.java
index 5d85417746..71a4c74db3 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistryImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistryImpl.java
@@ -89,7 +89,7 @@ public class ExecutableTableRegistryImpl implements 
ExecutableTableRegistry {
                     SchemaRegistry schemaRegistry = 
schemaManager.schemaRegistry(sqlTable.id());
                     SchemaDescriptor schemaDescriptor = 
schemaRegistry.schema(sqlTable.version());
                     TableRowConverterFactory converterFactory = new 
TableRowConverterFactoryImpl(
-                            sqlTable.keyColumns(), schemaRegistry, 
schemaDescriptor
+                            tableDescriptor, schemaRegistry, schemaDescriptor
                     );
 
                     InternalTable internalTable = table.internalTable();
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
index 96893bae04..b74cb14ffe 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
@@ -463,6 +463,7 @@ public class LogicalRelImplementor<RowT> implements 
IgniteRelVisitor<Node<RowT>>
             return new ScanNode<>(ctx, Collections.emptyList());
         }
 
+        // TODO: IGNITE-22822 fix required columns.
         RowSchema rowSchema = 
rowSchemaFromRelTypes(RelOptUtil.getFieldTypeList(rowType));
         RowFactory<RowT> rowFactory = ctx.rowHandler().factory(rowSchema);
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
index d7ece09da3..99a19bdacb 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.sql.engine.exec;
 
 import java.util.BitSet;
+import java.util.List;
 import org.apache.ignite.internal.lang.InternalTuple;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.BinaryTuple;
@@ -26,6 +27,7 @@ import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
+import 
org.apache.ignite.internal.sql.engine.util.ExtendedFieldDeserializingProjectedTuple;
 import 
org.apache.ignite.internal.sql.engine.util.FieldDeserializingProjectedTuple;
 import org.apache.ignite.internal.sql.engine.util.FormatAwareProjectedTuple;
 
@@ -40,16 +42,20 @@ public class ProjectedTableRowConverterImpl extends 
TableRowConverterImpl {
 
     private final BinaryTupleSchema fullTupleSchema;
 
+    private final List<VirtualColumn> virtualColumns;
+
     /** Constructor. */
     ProjectedTableRowConverterImpl(
             SchemaRegistry schemaRegistry,
             BinaryTupleSchema fullTupleSchema,
             SchemaDescriptor schemaDescriptor,
-            BitSet requiredColumns
+            BitSet requiredColumns,
+            List<VirtualColumn> extraColumns
     ) {
         super(schemaRegistry, schemaDescriptor);
 
         this.fullTupleSchema = fullTupleSchema;
+        this.virtualColumns = extraColumns;
 
         int size = requiredColumns.cardinality();
 
@@ -61,23 +67,27 @@ public class ProjectedTableRowConverterImpl extends 
TableRowConverterImpl {
                 requiredColumnsMapping[requiredIndex++] = 
column.positionInRow();
             }
         }
+
+        for (VirtualColumn col : extraColumns) {
+            requiredColumnsMapping[requiredIndex++] = col.columnIndex();
+        }
     }
 
     @Override
     public <RowT> RowT toRow(ExecutionContext<RowT> ectx, BinaryRow tableRow, 
RowFactory<RowT> factory) {
         InternalTuple tuple;
-        if (tableRow.schemaVersion() == schemaDescriptor.version()) {
-            BinaryTuple tableTuple = new 
BinaryTuple(schemaDescriptor.length(), tableRow.tupleSlice());
+        boolean rowSchemaMatches = tableRow.schemaVersion() == 
schemaDescriptor.version();
 
+        InternalTuple tableTuple = rowSchemaMatches
+                ? new BinaryTuple(schemaDescriptor.length(), 
tableRow.tupleSlice())
+                : schemaRegistry.resolve(tableRow, schemaDescriptor);
+
+        if (!virtualColumns.isEmpty()) {
+            tuple = new 
ExtendedFieldDeserializingProjectedTuple(fullTupleSchema, tableTuple, 
requiredColumnsMapping, virtualColumns);
+        } else if (rowSchemaMatches) {
             tuple = new FormatAwareProjectedTuple(tableTuple, 
requiredColumnsMapping);
         } else {
-            InternalTuple tableTuple = schemaRegistry.resolve(tableRow, 
schemaDescriptor);
-
-            tuple = new FieldDeserializingProjectedTuple(
-                    fullTupleSchema,
-                    tableTuple,
-                    requiredColumnsMapping
-            );
+            tuple = new FieldDeserializingProjectedTuple(fullTupleSchema, 
tableTuple, requiredColumnsMapping);
         }
 
         return factory.create(tuple);
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java
index ec41c9be2b..d02c6eb240 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ScannableTableImpl.java
@@ -61,18 +61,20 @@ public class ScannableTableImpl implements ScannableTable {
         Publisher<BinaryRow> pub;
         TxAttributes txAttributes = ctx.txAttributes();
 
+        int partId = partWithConsistencyToken.partId();
+
         if (txAttributes.readOnly()) {
             HybridTimestamp readTime = txAttributes.time();
 
             assert readTime != null;
 
-            pub = internalTable.scan(partWithConsistencyToken.partId(), 
txAttributes.id(), readTime, ctx.localNode(),
+            pub = internalTable.scan(partId, txAttributes.id(), readTime, 
ctx.localNode(),
                     txAttributes.coordinatorId());
         } else {
             PrimaryReplica recipient = new PrimaryReplica(ctx.localNode(), 
partWithConsistencyToken.enlistmentConsistencyToken());
 
             pub = internalTable.scan(
-                    partWithConsistencyToken.partId(),
+                    partId,
                     txAttributes.id(),
                     txAttributes.commitPartition(),
                     txAttributes.coordinatorId(),
@@ -85,7 +87,7 @@ public class ScannableTableImpl implements ScannableTable {
             );
         }
 
-        TableRowConverter rowConverter = 
converterFactory.create(requiredColumns);
+        TableRowConverter rowConverter = 
converterFactory.create(requiredColumns, partId);
 
         return new TransformingPublisher<>(pub, item -> 
rowConverter.toRow(ctx, item, rowFactory));
     }
@@ -122,13 +124,15 @@ public class ScannableTableImpl implements ScannableTable 
{
             flags |= (cond.upperInclude()) ? LESS_OR_EQUAL : 0;
         }
 
+        int partId = partWithConsistencyToken.partId();
+
         if (txAttributes.readOnly()) {
             HybridTimestamp readTime = txAttributes.time();
 
             assert readTime != null;
 
             pub = internalTable.scan(
-                    partWithConsistencyToken.partId(),
+                    partId,
                     txAttributes.id(),
                     readTime,
                     ctx.localNode(),
@@ -141,7 +145,7 @@ public class ScannableTableImpl implements ScannableTable {
             );
         } else {
             pub = internalTable.scan(
-                    partWithConsistencyToken.partId(),
+                    partId,
                     txAttributes.id(),
                     txAttributes.commitPartition(),
                     txAttributes.coordinatorId(),
@@ -154,7 +158,7 @@ public class ScannableTableImpl implements ScannableTable {
             );
         }
 
-        TableRowConverter rowConverter = 
converterFactory.create(requiredColumns);
+        TableRowConverter rowConverter = 
converterFactory.create(requiredColumns, partId);
 
         return new TransformingPublisher<>(pub, item -> 
rowConverter.toRow(ctx, item, rowFactory));
     }
@@ -179,13 +183,15 @@ public class ScannableTableImpl implements ScannableTable 
{
         assert keyTuple.elementCount() == columns.size()
                 : format("Key should contain exactly {} fields, but was {}", 
columns.size(), handler.toString(key));
 
+        int partId = partWithConsistencyToken.partId();
+
         if (txAttributes.readOnly()) {
             HybridTimestamp readTime = txAttributes.time();
 
             assert readTime != null;
 
             pub = internalTable.lookup(
-                    partWithConsistencyToken.partId(),
+                    partId,
                     txAttributes.id(),
                     readTime,
                     ctx.localNode(),
@@ -196,7 +202,7 @@ public class ScannableTableImpl implements ScannableTable {
             );
         } else {
             pub = internalTable.lookup(
-                    partWithConsistencyToken.partId(),
+                    partId,
                     txAttributes.id(),
                     txAttributes.commitPartition(),
                     txAttributes.coordinatorId(),
@@ -207,7 +213,7 @@ public class ScannableTableImpl implements ScannableTable {
             );
         }
 
-        TableRowConverter rowConverter = 
converterFactory.create(requiredColumns);
+        TableRowConverter rowConverter = 
converterFactory.create(requiredColumns, partId);
 
         return new TransformingPublisher<>(pub, item -> 
rowConverter.toRow(ctx, item, rowFactory));
     }
@@ -218,7 +224,7 @@ public class ScannableTableImpl implements ScannableTable {
             @Nullable InternalTransaction explicitTx,
             RowFactory<RowT> rowFactory,
             RowT key,
-            @Nullable BitSet requiredColumns
+            BitSet requiredColumns
     ) {
         TableRowConverter converter = converterFactory.create(requiredColumns);
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactory.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactory.java
index e465fef494..c8ff3f8ad9 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactory.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactory.java
@@ -27,4 +27,8 @@ import org.jetbrains.annotations.Nullable;
 @FunctionalInterface
 public interface TableRowConverterFactory {
     TableRowConverter create(@Nullable BitSet requiredColumns);
+
+    default TableRowConverter create(BitSet requiredColumns, int partId) {
+        return create(requiredColumns);
+    }
 }
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactoryImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactoryImpl.java
index 8ce0c7ab28..1cac952417 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactoryImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterFactoryImpl.java
@@ -18,10 +18,15 @@
 package org.apache.ignite.internal.sql.engine.exec;
 
 import java.util.BitSet;
-import org.apache.calcite.util.ImmutableIntList;
+import java.util.List;
+import java.util.function.IntFunction;
 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.sql.engine.schema.ColumnDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.type.NativeTypes;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -32,19 +37,20 @@ public class TableRowConverterFactoryImpl implements 
TableRowConverterFactory {
     private final SchemaDescriptor schemaDescriptor;
     private final BinaryTupleSchema fullTupleSchema;
     private final TableRowConverter fullRowConverter;
+    private final BitSet tableColumnSet;
+    private IntFunction<VirtualColumn> virtualColumnFactory;
 
     /**
      * Creates a factory from given schema and indexes of primary key.
      *
-     * @param primaryKeyLogicalIndexes Indexes of a primary key column in a 
logical order. Used to
-     *      properly build key only rows.
+     * @param tableDescriptor Table descriptor.
      * @param schemaRegistry Registry of all schemas known so far. Used in 
case table returned
      *      a row in older version than required to make an upgrade.
      * @param schemaDescriptor Actual schema descriptor. Used as a target 
schema to convert
      *      rows from sql format to one accepted by underlying table.
      */
     public TableRowConverterFactoryImpl(
-            ImmutableIntList primaryKeyLogicalIndexes,
+            TableDescriptor tableDescriptor,
             SchemaRegistry schemaRegistry,
             SchemaDescriptor schemaDescriptor
     ) {
@@ -56,11 +62,38 @@ public class TableRowConverterFactoryImpl implements 
TableRowConverterFactory {
                 schemaRegistry,
                 schemaDescriptor
         );
+
+        tableColumnSet = new BitSet();
+        tableColumnSet.set(0, tableDescriptor.columnsCount());
+
+        ColumnDescriptor columnDescriptor = 
tableDescriptor.columnDescriptor(Commons.PART_COL_NAME);
+
+        if (columnDescriptor != null) {
+            assert columnDescriptor.virtual();
+
+            virtualColumnFactory = (partId) -> new 
VirtualColumn(columnDescriptor.logicalIndex(), NativeTypes.INT32, false, 
partId);
+        }
     }
 
     @Override
     public TableRowConverter create(@Nullable BitSet requiredColumns) {
-        if (requiredColumns == null || requiredColumns.cardinality() == 
schemaDescriptor.length()) {
+        // TODO: IGNITE-22823 fix this. UpdatableTable must pass the bitset 
with updatable columns.
+        if (requiredColumns == null) {
+            return fullRowConverter;
+        }
+
+        return create(requiredColumns, -1);
+    }
+
+    @Override
+    public TableRowConverter create(@Nullable BitSet requiredColumns, int 
partId) {
+        if (requiredColumns == null) {
+            requiredColumns = tableColumnSet;
+        }
+
+        boolean requireVirtualColumn = 
requiredColumns.nextSetBit(schemaDescriptor.length()) != -1;
+
+        if (!requireVirtualColumn && requiredColumns.cardinality() == 
schemaDescriptor.length()) {
             return fullRowConverter;
         }
 
@@ -68,7 +101,8 @@ public class TableRowConverterFactoryImpl implements 
TableRowConverterFactory {
                 schemaRegistry,
                 fullTupleSchema,
                 schemaDescriptor,
-                requiredColumns
+                requiredColumns,
+                requireVirtualColumn ? 
List.of(virtualColumnFactory.apply(partId)) : List.of()
         );
     }
 }
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTable.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTable.java
index 24f679b53b..79fee23eda 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTable.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTable.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.sql.engine.exec;
 
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.ignite.internal.sql.engine.exec.mapping.ColocationGroup;
 import org.apache.ignite.internal.sql.engine.exec.rel.ModifyNode;
 import org.apache.ignite.internal.sql.engine.schema.IgniteTable;
@@ -69,8 +68,7 @@ public interface UpdatableTable {
     /**
      * Updates rows if they are exists, inserts the rows otherwise.
      *
-     * <p>The rows passed should match the full row type defined by the 
table's {@link #descriptor() descriptor}
-     * (see {@link TableDescriptor#rowType(IgniteTypeFactory, 
ImmutableBitSet)}).
+     * <p>The rows passed should match the full row type defined by the 
table's {@link #descriptor() descriptor}.
      *
      * @param ectx An execution context.
      * @param rows Rows to upsert.
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTableImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTableImpl.java
index 05fbd824d9..0cc734588e 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTableImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/UpdatableTableImpl.java
@@ -21,6 +21,7 @@ import static 
org.apache.ignite.internal.partition.replicator.network.replicatio
 import static 
org.apache.ignite.internal.partition.replicator.network.replication.RequestType.RW_INSERT_ALL;
 import static 
org.apache.ignite.internal.partition.replicator.network.replication.RequestType.RW_UPSERT_ALL;
 import static 
org.apache.ignite.internal.replicator.message.ReplicaMessageUtils.toTablePartitionIdMessage;
+import static org.apache.ignite.internal.sql.engine.util.RowTypeUtils.rowType;
 import static 
org.apache.ignite.internal.sql.engine.util.TypeUtils.rowSchemaFromRelTypes;
 import static 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.collectRejectedRowsResponsesWithRestoreOrder;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
@@ -123,7 +124,7 @@ public final class UpdatableTableImpl implements 
UpdatableTable {
 
         validateNotNullConstraint(ectx.rowHandler(), rows);
 
-        RelDataType rowType = descriptor().rowType(ectx.getTypeFactory(), 
null);
+        RelDataType rowType = rowType(descriptor(), ectx.getTypeFactory());
         Supplier<RowSchema> schemaSupplier = makeSchemaSupplier(ectx);
 
         rows = validateCharactersOverflowAndTrimIfPossible(rowType, 
ectx.rowHandler(), rows, schemaSupplier);
@@ -204,7 +205,7 @@ public final class UpdatableTableImpl implements 
UpdatableTable {
     ) {
         validateNotNullConstraint(ectx.rowHandler(), row);
 
-        RelDataType rowType = descriptor().rowType(ectx.getTypeFactory(), 
null);
+        RelDataType rowType = rowType(descriptor(), ectx.getTypeFactory());
         Supplier<RowSchema> schemaSupplier = makeSchemaSupplier(ectx);
 
         RowT validatedRow = 
TypeUtils.validateCharactersOverflowAndTrimIfPossible(rowType, 
ectx.rowHandler(), row, schemaSupplier);
@@ -235,7 +236,7 @@ public final class UpdatableTableImpl implements 
UpdatableTable {
 
         validateNotNullConstraint(ectx.rowHandler(), rows);
 
-        RelDataType rowType = descriptor().rowType(ectx.getTypeFactory(), 
null);
+        RelDataType rowType = rowType(descriptor(), ectx.getTypeFactory());
         Supplier<RowSchema> schemaSupplier = makeSchemaSupplier(ectx);
 
         rows = validateCharactersOverflowAndTrimIfPossible(rowType, 
ectx.rowHandler(), rows, schemaSupplier);
@@ -354,7 +355,7 @@ public final class UpdatableTableImpl implements 
UpdatableTable {
 
                     RowHandler<RowT> handler = ectx.rowHandler();
                     IgniteTypeFactory typeFactory = ectx.getTypeFactory();
-                    RowSchema rowSchema = 
rowSchemaFromRelTypes(RelOptUtil.getFieldTypeList(desc.rowType(typeFactory, 
null)));
+                    RowSchema rowSchema = 
rowSchemaFromRelTypes(RelOptUtil.getFieldTypeList(rowType(desc, typeFactory)));
                     RowHandler.RowFactory<RowT> rowFactory = 
handler.factory(rowSchema);
 
                     ArrayList<String> conflictRows = new 
ArrayList<>(response.size());
@@ -420,7 +421,7 @@ public final class UpdatableTableImpl implements 
UpdatableTable {
                 return rowSchema;
             }
 
-            RelDataType rowType = descriptor().rowType(ectx.getTypeFactory(), 
null);
+            RelDataType rowType = rowType(descriptor(), ectx.getTypeFactory());
             rowSchema = 
rowSchemaFromRelTypes(RelOptUtil.getFieldTypeList(rowType));
             return rowSchema;
         };
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/VirtualColumn.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/VirtualColumn.java
new file mode 100644
index 0000000000..fb643469b1
--- /dev/null
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/VirtualColumn.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine.exec;
+
+import org.apache.ignite.internal.tostring.IgniteToStringExclude;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.type.NativeType;
+
+/**
+ * Virtual column implementation.
+ */
+public class VirtualColumn {
+    private final int columnIndex;
+    private final NativeType type;
+    private final boolean nullable;
+    @IgniteToStringExclude
+    private final Object value;
+
+    VirtualColumn(int columnIndex, NativeType type, boolean nullable, Object 
value) {
+        this.columnIndex = columnIndex;
+        this.value = value;
+        this.type = type;
+        this.nullable = nullable;
+    }
+
+    public int columnIndex() {
+        return columnIndex;
+    }
+
+    public NativeType type() {
+        return type;
+    }
+
+    public boolean isNullable() {
+        return nullable;
+    }
+
+    public <T> T value() {
+        return (T) value;
+    }
+
+    @Override
+    public String toString() {
+        return S.toString(VirtualColumn.class, this);
+    }
+}
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNode.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNode.java
index 6eeee41ad1..0251bd27ed 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNode.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNode.java
@@ -17,12 +17,14 @@
 
 package org.apache.ignite.internal.sql.engine.exec.rel;
 
+import static 
org.apache.ignite.internal.sql.engine.util.RowTypeUtils.storedRowsCount;
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-import java.util.stream.IntStream;
+import java.util.function.Predicate;
+import java.util.stream.StreamSupport;
 import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
@@ -120,7 +122,11 @@ public class ModifyNode<RowT> extends AbstractNode<RowT> 
implements SingleNode<R
         this.updateColumns = updateColumns;
 
         this.mapping = mapping(table.descriptor(), updateColumns);
-        this.insertRowMapping = IntStream.range(0, 
table.descriptor().columnsCount()).toArray();
+
+        this.insertRowMapping = 
StreamSupport.stream(table.descriptor().spliterator(), false)
+                        .filter(Predicate.not(ColumnDescriptor::virtual))
+                        .mapToInt(ColumnDescriptor::logicalIndex)
+                        .toArray();
     }
 
     /** {@inheritDoc} */
@@ -432,7 +438,7 @@ public class ModifyNode<RowT> extends AbstractNode<RowT> 
implements SingleNode<R
             return null;
         }
 
-        int columnCount = descriptor.columnsCount();
+        int columnCount = storedRowsCount(descriptor);
 
         int[] mapping = new int[columnCount];
 
@@ -444,6 +450,8 @@ public class ModifyNode<RowT> extends AbstractNode<RowT> 
implements SingleNode<R
             String columnName = updateColumns.get(i);
             ColumnDescriptor columnDescriptor = 
descriptor.columnDescriptor(columnName);
 
+            assert !columnDescriptor.virtual() : "Virtual column can't be 
updated";
+
             mapping[columnDescriptor.logicalIndex()] = columnCount + i;
         }
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlValidator.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlValidator.java
index 54b4bc193c..15c31df07b 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlValidator.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/IgniteSqlValidator.java
@@ -33,6 +33,7 @@ import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
@@ -44,6 +45,7 @@ import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.schema.impl.ModifiableViewTable;
 import org.apache.calcite.sql.JoinConditionType;
 import org.apache.calcite.sql.SqlAggFunction;
@@ -381,6 +383,42 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
         }
     }
 
+    @Override
+    protected RelDataType createTargetRowType(SqlValidatorTable table,
+            @Nullable SqlNodeList targetColumnList, boolean append) {
+        RelDataType baseRowType = 
table.unwrap(IgniteTable.class).rowTypeForInsert((IgniteTypeFactory) 
typeFactory);
+        if (targetColumnList == null) {
+            return baseRowType;
+        }
+        List<RelDataTypeField> targetFields = baseRowType.getFieldList();
+        final PairList<String, RelDataType> fields = PairList.of();
+        if (append) {
+            for (RelDataTypeField targetField : targetFields) {
+                fields.add(SqlUtil.deriveAliasFromOrdinal(fields.size()),
+                        targetField.getType());
+            }
+        }
+        final Set<Integer> assignedFields = new HashSet<>();
+        final RelOptTable relOptTable = table instanceof RelOptTable
+                ? ((RelOptTable) table) : null;
+        for (SqlNode node : targetColumnList) {
+            SqlIdentifier id = (SqlIdentifier) node;
+            RelDataTypeField targetField =
+                    SqlValidatorUtil.getTargetField(
+                            baseRowType, typeFactory, id, getCatalogReader(), 
relOptTable);
+            if (targetField == null) {
+                throw newValidationError(id,
+                        RESOURCE.unknownTargetColumn(id.toString()));
+            }
+            if (!assignedFields.add(targetField.getIndex())) {
+                throw newValidationError(id,
+                        RESOURCE.duplicateTargetColumn(targetField.getName()));
+            }
+            fields.add(targetField);
+        }
+        return typeFactory.createStructType(fields);
+    }
+
     /** {@inheritDoc} */
     @Override
     protected SqlSelect createSourceSelectForUpdate(SqlUpdate call) {
@@ -392,7 +430,7 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
         SqlIdentifier alias = call.getAlias() != null ? call.getAlias() :
                 new SqlIdentifier(deriveAlias(targetTable, 0), 
SqlParserPos.ZERO);
 
-        igniteTable.getRowType(typeFactory)
+        igniteTable.rowTypeForUpdate((IgniteTypeFactory) typeFactory)
                 .getFieldNames().stream()
                 .map(name -> alias.plus(name, SqlParserPos.ZERO))
                 .forEach(selectList::add);
@@ -876,8 +914,9 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
         return (IgniteTypeFactory) typeFactory;
     }
 
-    private boolean isSystemFieldName(String alias) {
-        return Commons.implicitPkEnabled() && 
Commons.IMPLICIT_PK_COL_NAME.equals(alias);
+    public static boolean isSystemFieldName(String alias) {
+        return (Commons.implicitPkEnabled() && 
Commons.IMPLICIT_PK_COL_NAME.equals(alias))
+                || alias.equals(Commons.PART_COL_NAME);
     }
 
     // We use these scopes to filter out valid usages of a ROW operator.
@@ -930,6 +969,14 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
     /** {@inheritDoc} */
     @Override
     public void validateCall(SqlCall call, SqlValidatorScope scope) {
+        if (call.getKind() == SqlKind.AS) {
+            String alias = deriveAlias(call, 0);
+
+            if (isSystemFieldName(alias)) {
+                throw newValidationError(call, 
IgniteResource.INSTANCE.illegalAlias(alias));
+            }
+        }
+
         CallScope callScope = callScopes.peek();
         boolean validatingRowOperator = call.getOperator() == 
SqlStdOperatorTable.ROW;
         boolean insideValues = callScope == CallScope.VALUES;
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
index 22ab2f73d3..53974370e7 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
@@ -111,6 +111,7 @@ import 
org.apache.ignite.internal.catalog.commands.TablePrimaryKey;
 import org.apache.ignite.internal.catalog.commands.TableSortedPrimaryKey;
 import org.apache.ignite.internal.catalog.descriptors.CatalogColumnCollation;
 import org.apache.ignite.internal.sql.engine.prepare.IgnitePlanner;
+import org.apache.ignite.internal.sql.engine.prepare.IgniteSqlValidator;
 import org.apache.ignite.internal.sql.engine.prepare.PlanningContext;
 import org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterColumn;
 import org.apache.ignite.internal.sql.engine.sql.IgniteSqlAlterTableAddColumn;
@@ -290,6 +291,17 @@ public class DdlSqlToCommandConverter {
                 .map(IgniteSqlPrimaryKeyConstraint.class::cast)
                 .collect(Collectors.toList());
 
+        for (SqlNode sqlNode : createTblNode.columnList().getList()) {
+            if (sqlNode instanceof SqlColumnDeclaration) {
+                String colName = ((SqlColumnDeclaration) 
sqlNode).name.getSimple();
+
+                if (IgniteSqlValidator.isSystemFieldName(colName)) {
+                    throw new SqlException(STMT_VALIDATION_ERR, "Failed to 
validate query. "
+                            + "Column '" + colName + "' is reserved name.");
+                }
+            }
+        }
+
         if (pkConstraints.isEmpty() && Commons.implicitPkEnabled()) {
             SqlIdentifier colName = new 
SqlIdentifier(Commons.IMPLICIT_PK_COL_NAME, SqlParserPos.ZERO);
 
@@ -444,6 +456,12 @@ public class DdlSqlToCommandConverter {
             SqlColumnDeclaration col = (SqlColumnDeclaration) colNode;
             Boolean nullable = col.dataType.getNullable();
 
+            String colName = col.name.getSimple();
+            if (IgniteSqlValidator.isSystemFieldName(colName)) {
+                throw new SqlException(STMT_VALIDATION_ERR, "Failed to 
validate query. "
+                        + "Column '" + colName + "' is reserved name.");
+            }
+
             columns.add(convertColumnDeclaration(col, ctx.planner(), nullable 
!= null ? nullable : true));
         }
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptor.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptor.java
index ff6b2e3fc7..18dbaafac1 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptor.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptor.java
@@ -33,6 +33,11 @@ public interface ColumnDescriptor {
     /** Returns {@code true} if this column should not be expanded in query 
until user explicitly specify it as part of the statement. */
     boolean hidden();
 
+    /** Returns {@code true} if this column should not be stored. */
+    default boolean virtual() {
+        return false;
+    }
+
     /** Returns the strategy to follow when generating value for column not 
specified in the INSERT statement. */
     DefaultValueStrategy defaultStrategy();
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptorImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptorImpl.java
index 5e5c36e5a4..26325f9072 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptorImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/ColumnDescriptorImpl.java
@@ -32,6 +32,7 @@ public class ColumnDescriptorImpl implements ColumnDescriptor 
{
 
     private final boolean key;
     private final boolean hidden;
+    private final boolean virtual;
 
     private final String name;
 
@@ -49,6 +50,7 @@ public class ColumnDescriptorImpl implements ColumnDescriptor 
{
      * @param name The name of the column.
      * @param key If {@code true}, this column will be considered as a part of 
PK.
      * @param hidden If {@code true}, this column will not be expanded until 
explicitly mentioned.
+     * @param virtual If {@code true}, this column will not be stored.
      * @param nullable If {@code true}, this column will be considered as a 
nullable.
      * @param logicalIndex A 0-based index in a schema defined by a user.
      * @param type Type of the value in the underlying storage.
@@ -61,6 +63,7 @@ public class ColumnDescriptorImpl implements ColumnDescriptor 
{
             String name,
             boolean key,
             boolean hidden,
+            boolean virtual,
             boolean nullable,
             int logicalIndex,
             NativeType type,
@@ -69,6 +72,7 @@ public class ColumnDescriptorImpl implements ColumnDescriptor 
{
     ) {
         this.key = key;
         this.hidden = hidden;
+        this.virtual = virtual;
         this.nullable = nullable;
         this.name = name;
         this.defaultStrategy = defaultStrategy;
@@ -86,6 +90,12 @@ public class ColumnDescriptorImpl implements 
ColumnDescriptor {
         return hidden;
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public boolean virtual() {
+        return virtual;
+    }
+
     /** {@inheritDoc} */
     @Override
     public boolean nullable() {
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
index a7974f3c29..cff2596f36 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
@@ -43,6 +43,14 @@ public interface IgniteTable extends IgniteDataSource {
      */
     RelDataType rowTypeForInsert(IgniteTypeFactory factory);
 
+    /**
+     * Returns row type excluding effectively virtual fields.
+     *
+     * @param factory Type factory.
+     * @return Row type for UPDATE operation.
+     */
+    RelDataType rowTypeForUpdate(IgniteTypeFactory factory);
+
     /**
      * Returns row type containing only key fields.
      *
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
index 170f23861d..ce089145e9 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.function.Supplier;
+import java.util.stream.StreamSupport;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
@@ -43,6 +44,7 @@ import org.jetbrains.annotations.Nullable;
 public class IgniteTableImpl extends AbstractIgniteDataSource implements 
IgniteTable {
     private final ImmutableIntList keyColumns;
     private final @Nullable ImmutableBitSet columnsToInsert;
+    private final @Nullable ImmutableBitSet columnsToUpdate;
 
     private final Map<String, IgniteIndex> indexMap;
 
@@ -68,6 +70,12 @@ public class IgniteTableImpl extends 
AbstractIgniteDataSource implements IgniteT
         this.partitions = partitions;
         this.columnsToInsert = deriveColumnsToInsert(desc);
 
+        int virtualColumnsCount = (int) 
StreamSupport.stream(desc.spliterator(), false)
+                .filter(ColumnDescriptor::virtual)
+                .count();
+
+        this.columnsToUpdate = ImmutableBitSet.range(desc.columnsCount() - 
virtualColumnsCount);
+
         colocationColumnTypes = new Lazy<>(this::evaluateTypes);
     }
 
@@ -169,7 +177,8 @@ public class IgniteTableImpl extends 
AbstractIgniteDataSource implements IgniteT
     /** {@inheritDoc} */
     @Override
     public boolean isUpdateAllowed(int colIdx) {
-        return !descriptor().columnDescriptor(colIdx).key();
+        ColumnDescriptor columnDescriptor = 
descriptor().columnDescriptor(colIdx);
+        return !columnDescriptor.key() && !columnDescriptor.virtual();
     }
 
     /** {@inheritDoc} */
@@ -178,6 +187,12 @@ public class IgniteTableImpl extends 
AbstractIgniteDataSource implements IgniteT
         return descriptor().rowType(factory, columnsToInsert);
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public RelDataType rowTypeForUpdate(IgniteTypeFactory factory) {
+        return descriptor().rowType(factory, columnsToUpdate);
+    }
+
     /** {@inheritDoc} */
     @Override
     public RelDataType rowTypeForDelete(IgniteTypeFactory factory) {
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
index f2357c911f..4b4dbe60b5 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
@@ -56,6 +56,7 @@ import 
org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.apache.ignite.internal.sql.engine.util.cache.Cache;
 import org.apache.ignite.internal.sql.engine.util.cache.CacheFactory;
+import org.apache.ignite.internal.type.NativeTypes;
 import org.apache.ignite.lang.ErrorGroups.Common;
 
 /**
@@ -213,43 +214,60 @@ public class SqlSchemaManagerImpl implements 
SqlSchemaManager {
     }
 
     private static TableDescriptor 
createTableDescriptorForTable(CatalogTableDescriptor descriptor) {
-        List<ColumnDescriptor> colDescriptors = new ArrayList<>();
-
         List<CatalogTableColumnDescriptor> columns = descriptor.columns();
-        Object2IntMap<String> columnToIndex = new Object2IntOpenHashMap<>();
+
+        List<ColumnDescriptor> colDescriptors = new ArrayList<>(columns.size() 
+ 1);
+        Object2IntMap<String> columnToIndex = new 
Object2IntOpenHashMap<>(columns.size() + 1);
+
         for (int i = 0; i < columns.size(); i++) {
             CatalogTableColumnDescriptor col = columns.get(i);
             boolean key = descriptor.isPrimaryKeyColumn(col.name());
-            CatalogColumnDescriptor columnDescriptor = 
createColumnDescriptor(col, key, i);
+            ColumnDescriptor columnDescriptor = createColumnDescriptor(col, 
key, i);
 
             columnToIndex.put(col.name(), i);
-
             colDescriptors.add(columnDescriptor);
         }
 
+        if (Commons.implicitPkEnabled()) {
+            int implicitPkColIdx = 
columnToIndex.getOrDefault(Commons.IMPLICIT_PK_COL_NAME, -1);
+
+            if (implicitPkColIdx != -1) {
+                colDescriptors.set(implicitPkColIdx, 
injectDefault(colDescriptors.get(implicitPkColIdx)));
+            }
+        }
+
         List<Integer> colocationColumns = 
descriptor.colocationColumns().stream()
                 .map(columnToIndex::getInt)
                 .collect(Collectors.toList());
 
+        // Add virtual column.
+        ColumnDescriptorImpl partVirtualColumn = 
createPartitionVirtualColumn(columns.size());
+        colDescriptors.add(partVirtualColumn);
+        columnToIndex.put(partVirtualColumn.name(), 
partVirtualColumn.logicalIndex());
+
         // TODO Use the actual zone ID after implementing 
https://issues.apache.org/jira/browse/IGNITE-18426.
         int tableId = descriptor.id();
         IgniteDistribution distribution = 
IgniteDistributions.affinity(colocationColumns, tableId, tableId);
 
-        if (Commons.implicitPkEnabled()) {
-            colDescriptors = colDescriptors.stream()
-                    .map(column -> {
-                        if 
(Commons.IMPLICIT_PK_COL_NAME.equals(column.name())) {
-                            return injectDefault(column);
-                        }
-
-                        return column;
-                    })
-                    .collect(Collectors.toList());
-        }
-
         return new TableDescriptorImpl(colDescriptors, distribution);
     }
 
+    private static ColumnDescriptorImpl createPartitionVirtualColumn(int 
logicalIndex) {
+        return new ColumnDescriptorImpl(
+                Commons.PART_COL_NAME,
+                false,
+                true,
+                true,
+                true,
+                logicalIndex,
+                NativeTypes.INT32,
+                DefaultValueStrategy.DEFAULT_COMPUTED,
+                () -> {
+                    throw new AssertionError("Partition virtual column is 
generated by a function");
+                }
+        );
+    }
+
     private static ColumnDescriptor injectDefault(ColumnDescriptor desc) {
         assert Commons.implicitPkEnabled() && 
Commons.IMPLICIT_PK_COL_NAME.equals(desc.name()) : desc;
 
@@ -257,6 +275,7 @@ public class SqlSchemaManagerImpl implements 
SqlSchemaManager {
                 desc.name(),
                 desc.key(),
                 true,
+                false,
                 desc.nullable(),
                 desc.logicalIndex(),
                 desc.physicalType(),
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
index c9062bffea..4a1a784c3a 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
@@ -24,10 +24,11 @@ import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeFactory.Builder;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.ColumnStrategy;
 import org.apache.calcite.sql2rel.InitializerContext;
@@ -66,8 +67,9 @@ public class TableDescriptorImpl extends 
NullInitializerExpressionFactory implem
 
         Map<String, ColumnDescriptor> descriptorsMap = 
newHashMap(columnDescriptors.size());
 
-        RelDataTypeFactory factory = Commons.typeFactory();
+        IgniteTypeFactory factory = Commons.typeFactory();
         RelDataTypeFactory.Builder typeBuilder = new 
RelDataTypeFactory.Builder(factory);
+
         for (ColumnDescriptor descriptor : columnDescriptors) {
             typeBuilder.add(descriptor.name(), deriveLogicalType(factory, 
descriptor));
             descriptorsMap.put(descriptor.name(), descriptor);
@@ -75,7 +77,6 @@ public class TableDescriptorImpl extends 
NullInitializerExpressionFactory implem
 
         this.descriptors = columnDescriptors.toArray(DUMMY);
         this.descriptorsMap = descriptorsMap;
-
         this.rowType = typeBuilder.build();
     }
 
@@ -93,6 +94,9 @@ public class TableDescriptorImpl extends 
NullInitializerExpressionFactory implem
     /** {@inheritDoc} */
     @Override
     public ColumnStrategy generationStrategy(RelOptTable tbl, int colIdx) {
+        if (descriptors[colIdx].virtual()) {
+            return ColumnStrategy.VIRTUAL;
+        }
         if (descriptors[colIdx].defaultStrategy() != 
DefaultValueStrategy.DEFAULT_NULL) {
             return ColumnStrategy.DEFAULT;
         }
@@ -121,6 +125,10 @@ public class TableDescriptorImpl extends 
NullInitializerExpressionFactory implem
                 return rexBuilder.makeLiteral(internalValue, relDataType, 
false);
             }
             case DEFAULT_COMPUTED: {
+                if (descriptor.virtual()) {
+                    return 
rexBuilder.makeInputRef(tbl.getRowType().getFieldList().get(colIdx).getType(), 
colIdx);
+                }
+
                 assert descriptor.key() : "DEFAULT_COMPUTED is only supported 
for primary key columns. Column: " + descriptor.name();
 
                 return rexBuilder.makeCall(IgniteSqlOperatorTable.RAND_UUID);
@@ -136,9 +144,14 @@ public class TableDescriptorImpl extends 
NullInitializerExpressionFactory implem
         if (usedColumns == null || usedColumns.cardinality() == 
descriptors.length) {
             return rowType;
         } else {
-            return new 
RelDataTypeFactory.Builder(factory).addAll(rowType.getFieldList().stream()
-                    .filter(field -> usedColumns.get(field.getIndex()))
-                    .collect(Collectors.toList())).build();
+            Builder builder = new Builder(factory);
+
+            List<RelDataTypeField> fieldList = rowType.getFieldList();
+            for (int i : usedColumns) {
+                builder.add(fieldList.get(i));
+            }
+
+            return builder.build();
         }
     }
 
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 2ca099ad12..565000d977 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
@@ -47,8 +47,8 @@ import org.apache.ignite.internal.lang.InternalTuple;
  * </pre>
  */
 abstract class AbstractProjectedTuple implements InternalTuple {
-    InternalTuple delegate;
-    int[] projection;
+    protected InternalTuple delegate;
+    protected int[] projection;
 
     private boolean normalized = false;
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
index 98441e09d5..b53d2a719d 100644
--- 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/Commons.java
@@ -125,6 +125,7 @@ import org.jetbrains.annotations.Nullable;
  */
 public final class Commons {
     public static final String IMPLICIT_PK_COL_NAME = "__p_key";
+    public static final String PART_COL_NAME = "__part";
 
     public static final int IN_BUFFER_SIZE = 512;
 
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/ExtendedFieldDeserializingProjectedTuple.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/ExtendedFieldDeserializingProjectedTuple.java
new file mode 100644
index 0000000000..1411c32d23
--- /dev/null
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/ExtendedFieldDeserializingProjectedTuple.java
@@ -0,0 +1,329 @@
+/*
+ * 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.util;
+
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+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.List;
+import java.util.UUID;
+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.sql.engine.exec.VirtualColumn;
+
+/**
+ * A projected tuple that enriches {@link FieldDeserializingProjectedTuple} 
with extra columns.
+ *
+ * <p>Not thread safe!
+ *
+ * @see FieldDeserializingProjectedTuple
+ */
+public class ExtendedFieldDeserializingProjectedTuple extends 
FieldDeserializingProjectedTuple {
+
+    private final Int2ObjectMap<VirtualColumn> extraColumns;
+
+    /**
+     * Constructor.
+     *
+     * @param schema A schema of the original tuple (represented by delegate). 
Used to read content of the delegate to build a
+     *         proper byte buffer which content satisfying the schema with 
regard to given projection.
+     * @param delegate An original tuple to create projection from.
+     * @param projection A projection. That is, desired order of fields in 
original tuple. In that projection, index of the array is
+     *         an index of field in resulting projection, and an element of 
the array at that index is an index of column in original
+     *         tuple.
+     * @param extraColumns Extra columns.
+     */
+    public ExtendedFieldDeserializingProjectedTuple(BinaryTupleSchema schema, 
InternalTuple delegate, int[] projection,
+            List<VirtualColumn> extraColumns) {
+        super(schema, delegate, projection);
+
+        this.extraColumns = new Int2ObjectOpenHashMap<>(extraColumns.size());
+
+        extraColumns.forEach(c -> this.extraColumns.put(c.columnIndex(), c));
+    }
+
+    @Override
+    protected void normalize() {
+        var builder = new BinaryTupleBuilder(projection.length);
+        var newProjection = new int[projection.length];
+
+        for (int i = 0; i < projection.length; i++) {
+            int col = projection[i];
+
+            newProjection[i] = i;
+
+            if (extraColumns.containsKey(col)) {
+                VirtualColumn column = extraColumns.get(col);
+
+                BinaryRowConverter.appendValue(builder, new 
Element(column.type(), true), column.value());
+
+                continue;
+            }
+
+            Element element = schema.element(col);
+
+            BinaryRowConverter.appendValue(builder, element, 
schema.value(delegate, col));
+        }
+
+        delegate = new BinaryTuple(projection.length, builder.build());
+        projection = newProjection;
+        extraColumns.clear();
+    }
+
+    private boolean isExtraColumn(int col) {
+        return extraColumns.containsKey(projection[col]);
+    }
+
+    private VirtualColumn extraColumn(int col) {
+        return extraColumns.get(projection[col]);
+    }
+
+    @Override
+    public boolean hasNullValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value() == null;
+        }
+
+        return super.hasNullValue(col);
+    }
+
+    @Override
+    public boolean booleanValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.booleanValue(col);
+    }
+
+    @Override
+    public Boolean booleanValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.booleanValueBoxed(col);
+    }
+
+    @Override
+    public byte byteValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.byteValue(col);
+    }
+
+    @Override
+    public Byte byteValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.byteValueBoxed(col);
+    }
+
+    @Override
+    public short shortValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.shortValue(col);
+    }
+
+    @Override
+    public Short shortValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.shortValueBoxed(col);
+    }
+
+    @Override
+    public int intValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.intValue(col);
+    }
+
+    @Override
+    public Integer intValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.intValueBoxed(col);
+    }
+
+    @Override
+    public long longValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.longValue(col);
+    }
+
+    @Override
+    public Long longValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.longValueBoxed(col);
+    }
+
+    @Override
+    public float floatValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.floatValue(col);
+    }
+
+    @Override
+    public Float floatValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.floatValueBoxed(col);
+    }
+
+    @Override
+    public double doubleValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.doubleValue(col);
+    }
+
+    @Override
+    public Double doubleValueBoxed(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.doubleValueBoxed(col);
+    }
+
+    @Override
+    public BigDecimal decimalValue(int col, int decimalScale) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.decimalValue(col, decimalScale);
+    }
+
+    @Override
+    public BigInteger numberValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.numberValue(col);
+    }
+
+    @Override
+    public String stringValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.stringValue(col);
+    }
+
+    @Override
+    public byte[] bytesValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.bytesValue(col);
+    }
+
+    @Override
+    public UUID uuidValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.uuidValue(col);
+    }
+
+    @Override
+    public BitSet bitmaskValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.bitmaskValue(col);
+    }
+
+    @Override
+    public LocalDate dateValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.dateValue(col);
+    }
+
+    @Override
+    public LocalTime timeValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.timeValue(col);
+    }
+
+    @Override
+    public LocalDateTime dateTimeValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+
+        return super.dateTimeValue(col);
+    }
+
+    @Override
+    public Instant timestampValue(int col) {
+        if (isExtraColumn(col)) {
+            return extraColumn(col).value();
+        }
+        return super.timestampValue(col);
+    }
+}
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 22b0406851..1902fc6d68 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
@@ -36,7 +36,7 @@ import 
org.apache.ignite.internal.schema.BinaryTupleSchema.Element;
  * @see AbstractProjectedTuple
  */
 public class FieldDeserializingProjectedTuple extends AbstractProjectedTuple {
-    private final BinaryTupleSchema schema;
+    protected final BinaryTupleSchema schema;
 
     /**
      * Constructor.
diff --git 
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/RowTypeUtils.java
 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/RowTypeUtils.java
new file mode 100644
index 0000000000..b0bc589de5
--- /dev/null
+++ 
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/RowTypeUtils.java
@@ -0,0 +1,69 @@
+/*
+ * 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.util;
+
+import java.util.BitSet;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.ImmutableBitSet;
+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.type.IgniteTypeFactory;
+
+/**
+ * Row type utils class.
+ */
+public final class RowTypeUtils {
+    /**
+     * Computes row type for provided table with virtual columns filtering.
+     *
+     * @param tableDescriptor Table descriptor.
+     * @param factory Type factory.
+     * @return Row type.
+     */
+    public static RelDataType rowType(TableDescriptor tableDescriptor, 
IgniteTypeFactory factory) {
+        return tableDescriptor.rowType(factory, 
storedColumns(tableDescriptor));
+    }
+
+    /**
+     * Compute stored rows count for provided table.
+     *
+     * @param tableDescriptor Table descriptor.
+     * @return Stored rows count.
+     */
+    public static int storedRowsCount(TableDescriptor tableDescriptor) {
+        return storedColumns(tableDescriptor).cardinality();
+    }
+
+    private static ImmutableBitSet storedColumns(TableDescriptor 
tableDescriptor) {
+        BitSet virtualColumns = new BitSet();
+        for (ColumnDescriptor descriptor : tableDescriptor) {
+            if (descriptor.virtual()) {
+                virtualColumns.set(descriptor.logicalIndex());
+            }
+        }
+        ImmutableBitSet storedColumns;
+        if (virtualColumns.isEmpty()) {
+            storedColumns = 
ImmutableBitSet.range(tableDescriptor.columnsCount());
+        } else {
+            virtualColumns.flip(0, tableDescriptor.columnsCount());
+            storedColumns = ImmutableBitSet.fromBitSet(virtualColumns);
+        }
+
+        return storedColumns;
+    }
+}
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java
index b2bc6ca5de..1f9519384f 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutableTableRegistrySelfTest.java
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.when;
 
 import java.util.Collections;
 import java.util.Map;
+import java.util.Spliterators;
 import java.util.concurrent.CompletableFuture;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.ignite.internal.TestHybridClock;
@@ -156,6 +157,7 @@ public class ExecutableTableRegistrySelfTest extends 
BaseIgniteAbstractTest {
             
when(schemaManager.schemaRegistry(tableId)).thenReturn(schemaRegistry);
             
when(schemaRegistry.schema(tableVersion)).thenReturn(schemaDescriptor);
             
when(descriptor.iterator()).thenReturn(Collections.emptyIterator());
+            
when(descriptor.spliterator()).thenReturn(Spliterators.emptySpliterator());
 
             IgniteTable sqlTable = new IgniteTableImpl(
                     table.name(), tableId, tableVersion, descriptor, 
ImmutableIntList.of(0), new TestStatistic(1_000.0), Map.of(), 1
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
index 796ac9248a..067450001e 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ProjectedTableRowConverterSelfTest.java
@@ -91,7 +91,8 @@ public class ProjectedTableRowConverterSelfTest extends 
BaseIgniteAbstractTest {
                 schemaRegistry,
                 BinaryTupleSchema.createRowSchema(schema),
                 schema,
-                BitSets.of(1, 3)
+                BitSets.of(1, 3),
+                List.of()
         );
 
         RowWrapper row = converter.toRow(executionContext, binaryRow, 
rowFactory);
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterSelfTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterSelfTest.java
index f6168174fe..019867afb6 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterSelfTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TableRowConverterSelfTest.java
@@ -26,7 +26,6 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
-import org.apache.calcite.util.ImmutableIntList;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
 import org.apache.ignite.internal.schema.BinaryRow;
@@ -39,6 +38,7 @@ import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.sql.engine.exec.RowHandler.RowFactory;
 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.schema.TableDescriptor;
 import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.internal.type.NativeType;
 import org.apache.ignite.internal.type.NativeTypes;
@@ -50,6 +50,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.CsvSource;
 import org.mockito.Mock;
+import org.mockito.Mockito;
 import org.mockito.junit.jupiter.MockitoExtension;
 
 /**
@@ -178,7 +179,7 @@ public class TableRowConverterSelfTest extends 
BaseIgniteAbstractTest {
         RowWrapper wrapper = rowFactory.create(keyColumnValues[key1], 
keyColumnValues[key2]);
 
         TableRowConverter converter = new TableRowConverterFactoryImpl(
-                ImmutableIntList.of(key1, key2),
+                Mockito.mock(TableDescriptor.class),
                 schemaRegistry,
                 schema
         ).create(null);
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNodeExecutionTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNodeExecutionTest.java
index 57419b91c0..149f5be0fa 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNodeExecutionTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/ModifyNodeExecutionTest.java
@@ -33,6 +33,7 @@ import static org.mockito.Mockito.when;
 import it.unimi.dsi.fastutil.ints.Int2ObjectMaps;
 import it.unimi.dsi.fastutil.longs.Long2ObjectMaps;
 import java.util.List;
+import java.util.Spliterators;
 import java.util.concurrent.CompletableFuture;
 import org.apache.calcite.rel.core.TableModify.Operation;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
@@ -80,7 +81,7 @@ public class ModifyNodeExecutionTest extends 
AbstractExecutionTest<RowWrapper> {
 
     @BeforeEach
     void setUpMock() {
-        when(descriptors.columnsCount()).thenReturn(2);
+        
when(descriptors.spliterator()).thenReturn(Spliterators.emptySpliterator());
         when(updatableTable.descriptor()).thenReturn(descriptors);
     }
 
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestBuilders.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestBuilders.java
index a45f8fe906..eb5abd0660 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestBuilders.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestBuilders.java
@@ -128,9 +128,11 @@ import org.apache.ignite.internal.tx.InternalTransaction;
 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.TemporalNativeType;
 import org.apache.ignite.internal.type.VarlenNativeType;
 import org.apache.ignite.internal.util.ArrayUtils;
+import org.apache.ignite.internal.util.CollectionUtils;
 import org.apache.ignite.internal.util.SubscriptionUtils;
 import org.apache.ignite.internal.util.TransformingIterator;
 import org.apache.ignite.internal.util.subscription.TransformingPublisher;
@@ -851,7 +853,7 @@ public class TestBuilders {
         @Override
         public TableBuilder addColumn(String name, NativeType type, boolean 
nullable) {
             columns.add(new ColumnDescriptorImpl(
-                    name, false, false, nullable, columns.size(), type, 
DefaultValueStrategy.DEFAULT_NULL, null
+                    name, false, false, false, nullable, columns.size(), type, 
DefaultValueStrategy.DEFAULT_NULL, null
             ));
 
             return this;
@@ -870,7 +872,7 @@ public class TestBuilders {
                 return addColumn(name, type);
             } else {
                 ColumnDescriptorImpl desc = new ColumnDescriptorImpl(
-                        name, false, false, true, columns.size(), type, 
DefaultValueStrategy.DEFAULT_CONSTANT, () -> defaultValue
+                        name, false, false, false, true, columns.size(), type, 
DefaultValueStrategy.DEFAULT_CONSTANT, () -> defaultValue
                 );
                 columns.add(desc);
             }
@@ -882,7 +884,7 @@ public class TestBuilders {
         @Override
         public TableBuilder addKeyColumn(String name, NativeType type) {
             columns.add(new ColumnDescriptorImpl(
-                    name, true, false, false, columns.size(), type, 
DefaultValueStrategy.DEFAULT_NULL, null
+                    name, true, false, false, false, columns.size(), type, 
DefaultValueStrategy.DEFAULT_NULL, null
             ));
 
             return this;
@@ -926,7 +928,18 @@ public class TestBuilders {
                 throw new IllegalArgumentException("Table must contain at 
least one column");
             }
 
-            TableDescriptorImpl tableDescriptor = new 
TableDescriptorImpl(columns, distribution);
+            TableDescriptorImpl tableDescriptor = new 
TableDescriptorImpl(CollectionUtils.concat(columns,
+                    List.of(new ColumnDescriptorImpl(
+                            Commons.PART_COL_NAME,
+                            false,
+                            true,
+                            true,
+                            false,
+                            columns.size(),
+                            NativeTypes.INT32,
+                            DefaultValueStrategy.DEFAULT_NULL,
+                            null
+                    ))), distribution);
 
             Map<String, IgniteIndex> indexes = indexBuilders.stream()
                     .map(idx -> idx.build(tableDescriptor))
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CorrelatedSubqueryPlannerTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CorrelatedSubqueryPlannerTest.java
index 446d2c1f14..3b689654a5 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CorrelatedSubqueryPlannerTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/CorrelatedSubqueryPlannerTest.java
@@ -45,6 +45,7 @@ import 
org.apache.ignite.internal.sql.engine.schema.IgniteTable;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.util.RexUtils;
 import org.apache.ignite.internal.type.NativeTypes;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /** Tests to verify correlated subquery planning. */
@@ -58,6 +59,7 @@ public class CorrelatedSubqueryPlannerTest extends 
AbstractPlannerTest {
      * @throws Exception In case of any unexpected error.
      */
     @Test
+    @Disabled("IGNITE-22772")
     public void test() throws Exception {
         IgniteSchema schema = createSchema(createTestTable("A", "B", "C", "D", 
"E"));
 
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
index ce5fbf507f..d7d917ecea 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
@@ -34,6 +34,7 @@ import java.util.function.UnaryOperator;
 import java.util.stream.Stream;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.core.TableScan;
@@ -41,6 +42,7 @@ import org.apache.calcite.rel.hint.HintStrategyTable;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 import org.apache.ignite.internal.sql.engine.exec.NodeWithConsistencyToken;
 import org.apache.ignite.internal.sql.engine.framework.TestBuilders;
@@ -51,7 +53,10 @@ import 
org.apache.ignite.internal.sql.engine.prepare.PlannerPhase;
 import org.apache.ignite.internal.sql.engine.prepare.PlanningContext;
 import org.apache.ignite.internal.sql.engine.rel.IgniteConvention;
 import org.apache.ignite.internal.sql.engine.rel.IgniteFilter;
+import org.apache.ignite.internal.sql.engine.rel.IgniteMergeJoin;
+import org.apache.ignite.internal.sql.engine.rel.IgniteNestedLoopJoin;
 import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.rel.IgniteSort;
 import org.apache.ignite.internal.sql.engine.rel.IgniteTableScan;
 import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
@@ -199,15 +204,15 @@ public class PlannerTest extends AbstractPlannerTest {
         String sql = "select d.deptno, d.name, e.id, e.name from dept d join 
emp e "
                 + "on d.deptno = e.deptno and e.name >= d.name order by 
e.name, d.deptno";
 
-        RelNode phys = physicalPlan(sql, publicSchema, 
"CorrelatedNestedLoopJoin");
-
-        assertNotNull(phys);
-        assertEquals("Sort(sort0=[$3], sort1=[$0], dir0=[ASC], dir1=[ASC])" + 
System.lineSeparator()
-                        + "  Project(DEPTNO=[$3], NAME=[$4], ID=[$0], 
NAME0=[$1])" + System.lineSeparator()
-                        + "    NestedLoopJoin(condition=[AND(=($3, $2), >=($1, 
$4))], joinType=[inner])" + System.lineSeparator()
-                        + "      TableScan(table=[[PUBLIC, EMP]])" + 
System.lineSeparator()
-                        + "      TableScan(table=[[PUBLIC, DEPT]])" + 
System.lineSeparator(),
-                RelOptUtil.toString(phys));
+        assertPlan(sql, publicSchema,
+                nodeOrAnyChild(isInstanceOf(IgniteSort.class)
+                        
.and(hasCollation(RelCollations.of(ImmutableIntList.of(3, 0))))
+                        
.and(nodeOrAnyChild(isInstanceOf(IgniteNestedLoopJoin.class)
+                                .and(hasChildThat(isTableScan("EMP")))
+                                .and(hasChildThat(isTableScan("DEPT")))
+                        ))
+                
).and(Predicate.not(nodeOrAnyChild(isInstanceOf(IgniteMergeJoin.class)))),
+                "CorrelatedNestedLoopJoin");
     }
 
     @Test
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/TypeCoercionTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/TypeCoercionTest.java
index 83420d0041..06bcc6721e 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/TypeCoercionTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/TypeCoercionTest.java
@@ -723,6 +723,11 @@ public class TypeCoercionTest extends AbstractPlannerTest {
             return protoType.apply(factory);
         }
 
+        @Override
+        public RelDataType rowTypeForUpdate(IgniteTypeFactory factory) {
+            return protoType.apply(factory);
+        }
+
         @Override
         public RelDataType rowTypeForDelete(IgniteTypeFactory factory) {
             throw new AssertionError();
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImplTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImplTest.java
index 3b947f7b22..19cd84524e 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImplTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImplTest.java
@@ -79,6 +79,7 @@ import 
org.apache.ignite.internal.schema.DefaultValueGenerator;
 import org.apache.ignite.internal.sql.engine.schema.IgniteIndex.Type;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
+import org.apache.ignite.internal.sql.engine.util.RowTypeUtils;
 import org.apache.ignite.internal.sql.engine.util.cache.CaffeineCacheFactory;
 import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
 import org.apache.ignite.sql.ColumnType;
@@ -276,9 +277,9 @@ public class SqlSchemaManagerImplTest extends 
BaseIgniteAbstractTest {
         assertThat(tableDescriptor, notNullValue());
 
         TableDescriptor descriptor = table.descriptor();
-        assertEquals(tableDescriptor.columns().size(), 
descriptor.columnsCount(), "column count");
+        assertEquals(tableDescriptor.columns().size(), 
RowTypeUtils.storedRowsCount(descriptor), "column count");
 
-        for (int i = 0; i < descriptor.columnsCount(); i++) {
+        for (int i = 0; i < tableDescriptor.columns().size(); i++) {
             CatalogTableColumnDescriptor expectedColumnDescriptor = 
tableDescriptor.columns().get(i);
             ColumnDescriptor actualColumnDescriptor = 
descriptor.columnDescriptor(i);
 
diff --git 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/QueryCheckerTest.java
 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/QueryCheckerTest.java
index bba725f46d..7df6f8d86e 100644
--- 
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/QueryCheckerTest.java
+++ 
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/QueryCheckerTest.java
@@ -70,7 +70,7 @@ public class QueryCheckerTest extends BaseIgniteAbstractTest {
                     .addColumn("VAL", NativeTypes.INT32)
                     .end()
             .dataProvider(NODE_NAME, "T1", 
TestBuilders.tableScan(DataProvider.fromCollection(List.of(
-                    new Object[] {1, 1}, new Object[] {2, 2}
+                    new Object[] {1, 1, 1}, new Object[] {2, 2, 1}
             ))))
             .build();
     // @formatter:on
diff --git 
a/modules/sql-engine/src/test/resources/mapping/test_partition_pruning.test 
b/modules/sql-engine/src/test/resources/mapping/test_partition_pruning.test
index 371bfcaaae..36603ca48f 100644
--- a/modules/sql-engine/src/test/resources/mapping/test_partition_pruning.test
+++ b/modules/sql-engine/src/test/resources/mapping/test_partition_pruning.test
@@ -84,30 +84,6 @@ Fragment#2
       Sort
         TableScan(name=PUBLIC.T1_N1N2N3, source=3, partitions=3, 
distribution=affinity[table: T1_N1N2N3, columns: [ID]])
 ---
-# Self join, different predicates that produce disjoint set of partitions
-N1
-SELECT /*+ DISABLE_RULE('NestedLoopJoinConverter', 'HashJoinConverter', 
'CorrelatedNestedLoopJoin') */ * FROM t1_n1n2n3 as t1, t1_n1n2n3 as t2 WHERE 
t1.id = t2.id and t1.id = 1 and t2.id = 42
----
-Fragment#0 root
-  executionNodes: [N1]
-  remoteFragments: [1]
-  exchangeSourceNodes: {1=[N2, N3]}
-  tree:
-    Receiver(sourceFragment=1, exchange=1, distribution=single)
-
-Fragment#1
-  targetNodes: [N1]
-  executionNodes: [N2, N3]
-  tables: [T1_N1N2N3, T1_N1N2N3]
-  partitions: {N2=[1:3], N3=[2:3]}
-  tree:
-    Sender(targetFragment=0, exchange=1, distribution=single)
-      MergeJoin
-        Sort
-          TableScan(name=PUBLIC.T1_N1N2N3, source=2, partitions=3, 
distribution=affinity[table: T1_N1N2N3, columns: [ID]])
-        Sort
-          TableScan(name=PUBLIC.T1_N1N2N3, source=3, partitions=3, 
distribution=affinity[table: T1_N1N2N3, columns: [ID]])
----
 # Correlated
 # Prune partitions from left arm statically, and pass meta to the right arm.
 # Same set of nodes.

Reply via email to