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

ptupitsyn 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 f6ce1ac450 IGNITE-21525 Java Client: Clean up IEP-54 leftovers (#3307)
f6ce1ac450 is described below

commit f6ce1ac450f5597582697adbcd967a7f702252ea
Author: Pavel Tupitsyn <[email protected]>
AuthorDate: Mon Mar 4 14:50:08 2024 +0200

    IGNITE-21525 Java Client: Clean up IEP-54 leftovers (#3307)
    
    * Pass key column indices from `SchemaDescriptor` to the client
    * Refactor the logic that relies on "key columns come first" design, 
support any key column order
    * Fix colocation hash calculation to use `keyIndex`
---
 .../main/java/org/apache/ignite/table/Tuple.java   |   5 +-
 .../internal/binarytuple/BinaryTupleBuilder.java   |   2 +-
 .../table/MutableTupleBinaryTupleAdapter.java      | 133 +++++------
 .../{ClientTuple.java => ClientHandlerTuple.java}  |  62 ++++--
 .../handler/requests/table/ClientTableCommon.java  |   4 +-
 .../requests/table/ClientHandlerTupleTests.java    | 197 +++++++++++++++++
 .../ignite/internal/client/TcpClientChannel.java   |   2 +-
 .../internal/client/sql/ClientAsyncResultSet.java  |   7 +-
 .../ignite/internal/client/sql/ClientSqlRow.java   |  16 +-
 .../internal/client/table/AbstractClientView.java  |  12 +-
 .../ignite/internal/client/table/ClientColumn.java |  50 ++++-
 .../client/table/ClientKeyValueBinaryView.java     |   4 +-
 .../internal/client/table/ClientKeyValueView.java  |  10 +-
 .../client/table/ClientRecordSerializer.java       |  30 +--
 .../internal/client/table/ClientRecordView.java    |   2 +-
 .../ignite/internal/client/table/ClientSchema.java | 146 ++++++++----
 .../ignite/internal/client/table/ClientTable.java  |  25 +--
 .../ignite/internal/client/table/ClientTuple.java  |  94 ++++++--
 .../client/table/ClientTupleSerializer.java        |  44 ++--
 .../marshaller/ClientMarshallerReader.java         |  72 +++---
 .../org/apache/ignite/client/ClientTupleTest.java  | 246 ++++++++++++++++-----
 .../internal/client/table/ColocationHashTests.java |  81 +++++++
 .../ignite/internal/util/HashCalculator.java       |   3 +-
 .../cpp/ignite/client/detail/table/schema.h        |   2 +-
 .../dotnet/Apache.Ignite.Tests/FakeServer.cs       |  10 +-
 .../dotnet/Apache.Ignite/Internal/Table/Table.cs   |   3 +-
 .../internal/table/ItThinClientColocationTest.java |   5 +-
 .../ignite/internal/schema/SchemaDescriptor.java   |  20 ++
 28 files changed, 942 insertions(+), 345 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/table/Tuple.java 
b/modules/api/src/main/java/org/apache/ignite/table/Tuple.java
index c51dac39c0..4526f9335b 100644
--- a/modules/api/src/main/java/org/apache/ignite/table/Tuple.java
+++ b/modules/api/src/main/java/org/apache/ignite/table/Tuple.java
@@ -161,7 +161,10 @@ public interface Tuple extends Iterable<Object> {
                 return false;
             }
 
-            if (!Objects.deepEquals(firstTuple.value(idx), 
secondTuple.value(idx2))) {
+            Object firstVal = firstTuple.value(idx);
+            Object secondVal = secondTuple.value(idx2);
+
+            if (!Objects.deepEquals(firstVal, secondVal)) {
                 return false;
             }
         }
diff --git 
a/modules/binary-tuple/src/main/java/org/apache/ignite/internal/binarytuple/BinaryTupleBuilder.java
 
b/modules/binary-tuple/src/main/java/org/apache/ignite/internal/binarytuple/BinaryTupleBuilder.java
index ae681d0d6a..c241894220 100644
--- 
a/modules/binary-tuple/src/main/java/org/apache/ignite/internal/binarytuple/BinaryTupleBuilder.java
+++ 
b/modules/binary-tuple/src/main/java/org/apache/ignite/internal/binarytuple/BinaryTupleBuilder.java
@@ -794,7 +794,7 @@ public class BinaryTupleBuilder {
 
     /** Proceed to the next tuple element. */
     private BinaryTupleBuilder proceed() {
-        assert elementIndex < numElements;
+        assert elementIndex < numElements : "Element index overflow: " + 
elementIndex + " >= " + numElements;
 
         int offset = buffer.position() - valueBase;
         switch (entrySize) {
diff --git 
a/modules/client-common/src/main/java/org/apache/ignite/internal/client/table/MutableTupleBinaryTupleAdapter.java
 
b/modules/client-common/src/main/java/org/apache/ignite/internal/client/table/MutableTupleBinaryTupleAdapter.java
index 60bdd17ef6..c8e70993b8 100644
--- 
a/modules/client-common/src/main/java/org/apache/ignite/internal/client/table/MutableTupleBinaryTupleAdapter.java
+++ 
b/modules/client-common/src/main/java/org/apache/ignite/internal/client/table/MutableTupleBinaryTupleAdapter.java
@@ -38,15 +38,12 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
     /** Underlying BinaryTuple. */
     private BinaryTupleReader binaryTuple;
 
+    /** Column count. Can be less than binaryTuple.elementCount in case of a 
slice. */
+    private final int columnCount;
+
     /** Tuple with overwritten data. */
     private @Nullable Tuple tuple;
 
-    /** Schema offset: value tuples skip the key part. */
-    private final int schemaOffset;
-
-    /** Schema size: key tuples skip the value part. */
-    private final int schemaSize;
-
     /** No-value set. */
     private final @Nullable BitSet noValueSet;
 
@@ -54,15 +51,15 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
      * Constructor.
      *
      * @param binaryTuple Binary tuple.
+     * @param columnCount Column count.
+     * @param noValueSet No-value set.
      */
-    public MutableTupleBinaryTupleAdapter(BinaryTupleReader binaryTuple, int 
schemaOffset, int schemaSize, @Nullable BitSet noValueSet) {
+    public MutableTupleBinaryTupleAdapter(BinaryTupleReader binaryTuple, int 
columnCount, @Nullable BitSet noValueSet) {
         assert binaryTuple != null : "binaryTuple != null";
-        assert schemaOffset >= 0 : "schemaOffset >= 0";
-        assert schemaSize > 0 : "schemaSize > 0";
+        assert columnCount > 0 : "columnCount > 0";
 
         this.binaryTuple = binaryTuple;
-        this.schemaOffset = schemaOffset;
-        this.schemaSize = schemaSize;
+        this.columnCount = columnCount;
         this.noValueSet = noValueSet;
     }
 
@@ -73,7 +70,7 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
             return tuple.columnCount();
         }
 
-        int cnt = schemaSize - schemaOffset;
+        int cnt = columnCount;
 
         if (noValueSet != null) {
             cnt -= noValueSet.cardinality();
@@ -97,9 +94,9 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
             return tuple.columnIndex(columnName);
         }
 
-        int internalIndex = schemaColumnIndex(columnName, null);
+        int binaryTupleIndex = binaryTupleIndex(columnName, null);
 
-        return internalIndex < 0 || internalIndex >= schemaSize ? -1 : 
internalIndex - schemaOffset;
+        return binaryTupleIndex < 0 ? -1 : publicIndex(binaryTupleIndex);
     }
 
     /** {@inheritDoc} */
@@ -109,13 +106,13 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
             return tuple.valueOrDefault(columnName, defaultValue);
         }
 
-        int internalIndex = schemaColumnIndex(columnName, null);
+        int binaryTupleIndex = binaryTupleIndex(columnName, null);
 
-        return internalIndex < 0
-                || internalIndex >= schemaSize
-                || (noValueSet != null && noValueSet.get(internalIndex))
+        return binaryTupleIndex < 0
+                || publicIndex(binaryTupleIndex) < 0
+                || (noValueSet != null && noValueSet.get(binaryTupleIndex))
                 ? defaultValue
-                : value(internalIndex);
+                : (T) object(binaryTupleIndex);
     }
 
     /** {@inheritDoc} */
@@ -125,13 +122,13 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
             return tuple.value(columnName);
         }
 
-        int internalIndex = schemaColumnIndex(columnName, null);
+        int binaryTupleIndex = binaryTupleIndex(columnName, null);
 
-        if (internalIndex < 0 || internalIndex >= schemaSize) {
+        if (binaryTupleIndex < 0 || publicIndex(binaryTupleIndex) < 0) {
             throw new IllegalArgumentException("Column doesn't exist [name=" + 
columnName + ']');
         }
 
-        return (T) object(internalIndex);
+        return (T) object(binaryTupleIndex);
     }
 
     /** {@inheritDoc} */
@@ -141,10 +138,10 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
             return tuple.value(columnIndex);
         }
 
-        Objects.checkIndex(columnIndex, schemaSize - schemaOffset);
+        Objects.checkIndex(columnIndex, columnCount);
 
-        int internalIndex = columnIndex + schemaOffset;
-        return (T) object(internalIndex);
+        int binaryTupleIndex = binaryTupleIndex(columnIndex);
+        return (T) object(binaryTupleIndex);
     }
 
     /** {@inheritDoc} */
@@ -423,23 +420,31 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
         return binaryTuple;
     }
 
-    protected abstract String schemaColumnName(int internalIndex);
+    protected abstract String schemaColumnName(int binaryTupleIndex);
 
-    protected abstract ColumnType schemaColumnType(int columnIndex);
+    protected abstract ColumnType schemaColumnType(int binaryTupleIndex);
 
-    protected abstract int schemaDecimalScale(int columnIndex);
+    protected abstract int schemaDecimalScale(int binaryTupleIndex);
 
-    protected abstract int schemaColumnIndex(String columnName);
+    protected int publicIndex(int binaryTupleIndex) {
+        return binaryTupleIndex;
+    }
+
+    protected abstract int binaryTupleIndex(String columnName);
+
+    protected int binaryTupleIndex(int publicIndex) {
+        return publicIndex;
+    }
 
-    private int schemaColumnIndex(String columnName, @Nullable ColumnType 
type) {
-        var internalIndex = schemaColumnIndex(columnName);
+    private int binaryTupleIndex(String columnName, @Nullable ColumnType type) 
{
+        var binaryTupleIndex = binaryTupleIndex(columnName);
 
-        if (internalIndex < 0) {
-            return internalIndex;
+        if (binaryTupleIndex < 0) {
+            return binaryTupleIndex;
         }
 
         if (type != null) {
-            ColumnType actualType = schemaColumnType(internalIndex);
+            ColumnType actualType = schemaColumnType(binaryTupleIndex);
 
             if (type != actualType) {
                 throw new ClassCastException("Column with name '" + columnName 
+ "' has type " + actualType
@@ -447,11 +452,11 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
             }
         }
 
-        return internalIndex;
+        return binaryTupleIndex;
     }
 
     private int validateSchemaColumnType(String columnName, ColumnType type) {
-        var index = schemaColumnIndex(columnName, type);
+        var index = binaryTupleIndex(columnName, type);
 
         if (index < 0) {
             throw new IllegalArgumentException("Column doesn't exist [name=" + 
columnName + ']');
@@ -461,89 +466,89 @@ public abstract class MutableTupleBinaryTupleAdapter 
implements Tuple, BinaryTup
     }
 
     private int validateSchemaColumnType(int publicIndex, ColumnType type) {
-        Objects.checkIndex(publicIndex, schemaSize - schemaOffset);
+        Objects.checkIndex(publicIndex, columnCount);
 
-        int internalIndex = publicIndex + schemaOffset;
-        var actualType = schemaColumnType(internalIndex);
+        int binaryTupleIndex = binaryTupleIndex(publicIndex);
+        var actualType = schemaColumnType(binaryTupleIndex);
 
         if (type != actualType) {
             throw new ClassCastException("Column with index " + publicIndex + 
" has type " + actualType
                     + " but " + type + " was requested");
         }
 
-        return internalIndex;
+        return binaryTupleIndex;
     }
 
     private String schemaColumnName0(int publicIndex) {
-        Objects.checkIndex(publicIndex, schemaSize - schemaOffset);
+        Objects.checkIndex(publicIndex, columnCount);
 
-        return schemaColumnName(publicIndex + schemaOffset);
+        return schemaColumnName(binaryTupleIndex(publicIndex));
     }
 
-    private @Nullable Object object(int internalIndex) {
-        if (binaryTuple.hasNullValue(internalIndex)) {
+    private @Nullable Object object(int binaryTupleIndex) {
+        if (binaryTuple.hasNullValue(binaryTupleIndex)) {
             return null;
         }
 
-        var type = schemaColumnType(internalIndex);
+        var type = schemaColumnType(binaryTupleIndex);
 
         switch (type) {
             case BOOLEAN:
-                return binaryTuple.booleanValue(internalIndex);
+                return binaryTuple.booleanValue(binaryTupleIndex);
 
             case INT8:
-                return binaryTuple.byteValue(internalIndex);
+                return binaryTuple.byteValue(binaryTupleIndex);
 
             case INT16:
-                return binaryTuple.shortValue(internalIndex);
+                return binaryTuple.shortValue(binaryTupleIndex);
 
             case INT32:
-                return binaryTuple.intValue(internalIndex);
+                return binaryTuple.intValue(binaryTupleIndex);
 
             case INT64:
-                return binaryTuple.longValue(internalIndex);
+                return binaryTuple.longValue(binaryTupleIndex);
 
             case FLOAT:
-                return binaryTuple.floatValue(internalIndex);
+                return binaryTuple.floatValue(binaryTupleIndex);
 
             case DOUBLE:
-                return binaryTuple.doubleValue(internalIndex);
+                return binaryTuple.doubleValue(binaryTupleIndex);
 
             case DECIMAL:
-                return binaryTuple.decimalValue(internalIndex, 
schemaDecimalScale(internalIndex));
+                return binaryTuple.decimalValue(binaryTupleIndex, 
schemaDecimalScale(binaryTupleIndex));
 
             case DATE:
-                return binaryTuple.dateValue(internalIndex);
+                return binaryTuple.dateValue(binaryTupleIndex);
 
             case TIME:
-                return binaryTuple.timeValue(internalIndex);
+                return binaryTuple.timeValue(binaryTupleIndex);
 
             case DATETIME:
-                return binaryTuple.dateTimeValue(internalIndex);
+                return binaryTuple.dateTimeValue(binaryTupleIndex);
 
             case TIMESTAMP:
-                return binaryTuple.timestampValue(internalIndex);
+                return binaryTuple.timestampValue(binaryTupleIndex);
 
             case UUID:
-                return binaryTuple.uuidValue(internalIndex);
+                return binaryTuple.uuidValue(binaryTupleIndex);
 
             case BITMASK:
-                return binaryTuple.bitmaskValue(internalIndex);
+                return binaryTuple.bitmaskValue(binaryTupleIndex);
 
             case STRING:
-                return binaryTuple.stringValue(internalIndex);
+                return binaryTuple.stringValue(binaryTupleIndex);
 
             case BYTE_ARRAY:
-                return binaryTuple.bytesValue(internalIndex);
+                return binaryTuple.bytesValue(binaryTupleIndex);
 
             case PERIOD:
-                return binaryTuple.periodValue(internalIndex);
+                return binaryTuple.periodValue(binaryTupleIndex);
 
             case DURATION:
-                return binaryTuple.durationValue(internalIndex);
+                return binaryTuple.durationValue(binaryTupleIndex);
 
             case NUMBER:
-                return binaryTuple.numberValue(internalIndex);
+                return binaryTuple.numberValue(binaryTupleIndex);
 
             default:
                 throw new IllegalStateException("Unsupported type: " + type);
diff --git 
a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTuple.java
 
b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientHandlerTuple.java
similarity index 53%
rename from 
modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTuple.java
rename to 
modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientHandlerTuple.java
index e7efa82cf7..4cb8cf3a16 100644
--- 
a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTuple.java
+++ 
b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientHandlerTuple.java
@@ -29,25 +29,29 @@ import org.apache.ignite.table.Tuple;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Server-side client Tuple.
+ * Server-side Tuple implementation, wraps binary data coming from the client 
to pass to the table internals. This tuple implementation
+ * does not end up in the user's hands, so mutability is not supported.
  */
-class ClientTuple extends MutableTupleBinaryTupleAdapter implements 
SchemaAware {
-    /** Schema. */
+class ClientHandlerTuple extends MutableTupleBinaryTupleAdapter implements 
SchemaAware {
     private final SchemaDescriptor schema;
 
+    private final boolean keyOnly;
+
     /**
      * Constructor.
      *
      * @param schema Schema.
      * @param noValueSet No-value set.
      * @param tuple Tuple.
-     * @param schemaOffset Schema offset.
-     * @param schemaSize Schema size.
+     * @param keyOnly Key only.
      */
-    ClientTuple(SchemaDescriptor schema, BitSet noValueSet, BinaryTupleReader 
tuple, int schemaOffset, int schemaSize) {
-        super(tuple, schemaOffset, schemaSize, noValueSet);
+    ClientHandlerTuple(SchemaDescriptor schema, @Nullable BitSet noValueSet, 
BinaryTupleReader tuple, boolean keyOnly) {
+        super(tuple, tuple.elementCount(), noValueSet);
+
+        assert tuple.elementCount() == (keyOnly ? schema.keyColumns().length() 
: schema.length()) : "Tuple element count mismatch";
 
         this.schema = schema;
+        this.keyOnly = keyOnly;
     }
 
     /** {@inheritDoc} */
@@ -64,28 +68,56 @@ class ClientTuple extends MutableTupleBinaryTupleAdapter 
implements SchemaAware
 
     /** {@inheritDoc} */
     @Override
-    protected String schemaColumnName(int internalIndex) {
-        return schema.column(internalIndex).name();
+    protected String schemaColumnName(int binaryTupleIndex) {
+        return schema.column(binaryTupleIndex).name();
     }
 
     /** {@inheritDoc} */
     @Override
-    protected int schemaColumnIndex(String columnName) {
+    protected int binaryTupleIndex(String columnName) {
         Column column = schema.column(columnName);
-        return column == null ? -1 : column.schemaIndex();
+
+        if (column == null) {
+            return -1;
+        }
+
+        if (keyOnly) {
+            return schema.keyIndex(column);
+        }
+
+        return column.schemaIndex();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected int binaryTupleIndex(int publicIndex) {
+        return keyOnly
+                ? schema.keyColumns().column(publicIndex).schemaIndex()
+                : super.binaryTupleIndex(publicIndex);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected int publicIndex(int binaryTupleIndex) {
+        if (keyOnly) {
+            var col = schema.keyColumns().column(binaryTupleIndex);
+            return schema.keyIndex(col);
+        }
+
+        return super.publicIndex(binaryTupleIndex);
     }
 
     /** {@inheritDoc} */
     @Override
-    protected ColumnType schemaColumnType(int columnIndex) {
-        NativeTypeSpec spec = schema.column(columnIndex).type().spec();
+    protected ColumnType schemaColumnType(int binaryTupleIndex) {
+        NativeTypeSpec spec = schema.column(binaryTupleIndex).type().spec();
 
         return ClientTableCommon.getColumnType(spec);
     }
 
     /** {@inheritDoc} */
     @Override
-    protected int schemaDecimalScale(int columnIndex) {
-        return 
ClientTableCommon.getDecimalScale(schema.column(columnIndex).type());
+    protected int schemaDecimalScale(int binaryTupleIndex) {
+        return 
ClientTableCommon.getDecimalScale(schema.column(binaryTupleIndex).type());
     }
 }
diff --git 
a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
 
b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
index f99bc99b69..181126dab2 100644
--- 
a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
+++ 
b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
@@ -83,7 +83,7 @@ public class ClientTableCommon {
             packer.packInt(7);
             packer.packString(col.name());
             packer.packInt(getColumnType(col.type().spec()).ordinal());
-            packer.packBoolean(schema.isKeyColumn(colIdx));
+            packer.packInt(schema.keyIndex(col));
             packer.packBoolean(col.nullable());
             packer.packInt(schema.colocationIndex(col));
             packer.packInt(getDecimalScale(col.type()));
@@ -297,7 +297,7 @@ public class ClientTableCommon {
         var noValueSet = unpacker.unpackBitSet();
         var binaryTupleReader = new BinaryTupleReader(cnt, 
unpacker.readBinary());
 
-        return new ClientTuple(schema, noValueSet, binaryTupleReader, 0, cnt);
+        return new ClientHandlerTuple(schema, noValueSet, binaryTupleReader, 
keyOnly);
     }
 
     /**
diff --git 
a/modules/client-handler/src/test/java/org/apache/ignite/client/handler/requests/table/ClientHandlerTupleTests.java
 
b/modules/client-handler/src/test/java/org/apache/ignite/client/handler/requests/table/ClientHandlerTupleTests.java
new file mode 100644
index 0000000000..93c94e5c52
--- /dev/null
+++ 
b/modules/client-handler/src/test/java/org/apache/ignite/client/handler/requests/table/ClientHandlerTupleTests.java
@@ -0,0 +1,197 @@
+/*
+ * 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.client.handler.requests.table;
+
+import static org.apache.ignite.internal.type.NativeTypes.BYTES;
+import static org.apache.ignite.internal.type.NativeTypes.DOUBLE;
+import static org.apache.ignite.internal.type.NativeTypes.FLOAT;
+import static org.apache.ignite.internal.type.NativeTypes.INT16;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.INT64;
+import static org.apache.ignite.internal.type.NativeTypes.INT8;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.datetime;
+import static org.apache.ignite.internal.type.NativeTypes.time;
+import static org.apache.ignite.internal.type.NativeTypes.timestamp;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+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.time.Month;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException;
+import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
+import org.apache.ignite.internal.type.NativeTypes;
+import org.apache.ignite.table.Tuple;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for {@link ClientHandlerTuple}.
+ */
+@SuppressWarnings("DataFlowIssue")
+public class ClientHandlerTupleTests {
+    private static final UUID GUID = UUID.randomUUID();
+
+    private static final LocalDate DATE = LocalDate.of(1995, Month.MAY, 23);
+
+    private static final LocalTime TIME = LocalTime.of(17, 0, 1, 222_333_444);
+
+    private static final LocalDateTime DATE_TIME = LocalDateTime.of(1995, 
Month.MAY, 23, 17, 0, 1, 222_333_444);
+
+    private static final Instant TIMESTAMP = Instant.now();
+
+    /** Schema descriptor. */
+    private final SchemaDescriptor fullSchema = new SchemaDescriptor(42,
+            new Column[]{new Column("keyUuidCol".toUpperCase(), 
NativeTypes.UUID, false)},
+            new Column[]{
+                    new Column("valByteCol".toUpperCase(), INT8, true),
+                    new Column("valShortCol".toUpperCase(), INT16, true),
+                    new Column("valIntCol".toUpperCase(), INT32, true),
+                    new Column("valLongCol".toUpperCase(), INT64, true),
+                    new Column("valFloatCol".toUpperCase(), FLOAT, true),
+                    new Column("valDoubleCol".toUpperCase(), DOUBLE, true),
+                    new Column("valDateCol".toUpperCase(), NativeTypes.DATE, 
true),
+                    new Column("valTimeCol".toUpperCase(), 
time(NativeTypes.MAX_TIME_PRECISION), true),
+                    new Column("valDateTimeCol".toUpperCase(), 
datetime(NativeTypes.MAX_TIME_PRECISION), true),
+                    new Column("valTimeStampCol".toUpperCase(), 
timestamp(NativeTypes.MAX_TIME_PRECISION), true),
+                    new Column("valBitmask1Col".toUpperCase(), 
NativeTypes.bitmaskOf(22), true),
+                    new Column("valBytesCol".toUpperCase(), BYTES, false),
+                    new Column("valStringCol".toUpperCase(), STRING, false),
+                    new Column("valNumberCol".toUpperCase(), 
NativeTypes.numberOf(20), false),
+                    new Column("valDecimalCol".toUpperCase(), 
NativeTypes.decimalOf(25, 5), false),
+            }
+    );
+
+    @Test
+    public void testTupleEquality() throws TupleMarshallerException {
+        Tuple tuple = createTuple();
+
+        BinaryTuple binaryTuple = new 
TupleMarshallerImpl(fullSchema).marshal(tuple).binaryTuple();
+        Tuple clientHandlerTuple = new ClientHandlerTuple(fullSchema, null, 
binaryTuple, false);
+
+        assertEquals(tuple, clientHandlerTuple);
+    }
+
+    @Test
+    public void testTupleEqualityKeyOnly() throws TupleMarshallerException {
+        Tuple tuple = createKeyTuple();
+
+        BinaryTuple binaryTuple = new 
TupleMarshallerImpl(fullSchema).marshalKey(tuple).binaryTuple();
+        Tuple clientHandlerTuple = new ClientHandlerTuple(fullSchema, null, 
binaryTuple, true);
+
+        assertEquals(tuple, clientHandlerTuple);
+    }
+
+    @Test
+    public void testValueReturnsValueByName() {
+        Tuple tuple = createTuple();
+
+        assertEquals(1, (byte) tuple.value("valByteCol"));
+        assertEquals(DATE, tuple.value("valDateCol"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
tuple.value("bad-name"));
+    }
+
+    @Test
+    public void testValueReturnsValueByNameKeyOnly() {
+        Tuple tuple = createKeyTuple();
+
+        assertEquals(GUID, tuple.value("keyUuidCol"));
+
+        assertThrows(IllegalArgumentException.class, () -> 
tuple.value("valDateCol"));
+    }
+
+    @Test
+    public void testValueReturnsValueByIndex() {
+        Tuple tuple = createTuple();
+
+        assertEquals(1, (byte) tuple.value(0));
+        assertEquals(4L, tuple.longValue(3));
+
+        assertThrows(IndexOutOfBoundsException.class, () -> tuple.value(123));
+    }
+
+    @Test
+    public void testValueReturnsValueByIndexKeyOnly() {
+        Tuple tuple = createKeyTuple();
+
+        assertEquals(GUID, tuple.value(0));
+
+        assertThrows(IndexOutOfBoundsException.class, () -> tuple.value(1));
+    }
+
+    @Test
+    public void testColumnCount() {
+        assertEquals(16, createTuple().columnCount());
+        assertEquals(1, createKeyTuple().columnCount());
+    }
+
+    @Test
+    public void testColumnIndex() {
+        Tuple tuple = createTuple();
+
+        assertEquals(0, tuple.columnIndex("valByteCol"));
+        assertEquals(3, tuple.columnIndex("valLongCol"));
+        assertEquals(-1, tuple.columnIndex("bad-name"));
+    }
+
+    @Test
+    public void testColumnIndexKeyOnly() {
+        Tuple tuple = createKeyTuple();
+
+        assertEquals(0, tuple.columnIndex("keyUuidCol"));
+        assertEquals(-1, tuple.columnIndex("valLongCol"));
+    }
+
+    private static Tuple createKeyTuple() {
+        return Tuple.create()
+                .set("keyUuidCol", GUID);
+    }
+
+    private static Tuple createTuple() {
+        Random rnd = new Random();
+
+        return Tuple.create()
+                .set("valByteCol", (byte) 1)
+                .set("valShortCol", (short) 2)
+                .set("valIntCol", 3)
+                .set("valLongCol", 4L)
+                .set("valFloatCol", 0.055f)
+                .set("valDoubleCol", 0.066d)
+                .set("keyUuidCol", GUID)
+                .set("valDateCol", DATE)
+                .set("valDateTimeCol", DATE_TIME)
+                .set("valTimeCol", TIME)
+                .set("valTimeStampCol", TIMESTAMP)
+                .set("valBitmask1Col", IgniteTestUtils.randomBitSet(rnd, 12))
+                .set("valBytesCol", IgniteTestUtils.randomBytes(rnd, 13))
+                .set("valStringCol", IgniteTestUtils.randomString(rnd, 14))
+                .set("valNumberCol", BigInteger.valueOf(rnd.nextLong()))
+                .set("valDecimalCol", BigDecimal.valueOf(rnd.nextLong(), 5));
+    }
+}
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/TcpClientChannel.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/TcpClientChannel.java
index 84c323f29b..e7382523f0 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/TcpClientChannel.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/TcpClientChannel.java
@@ -369,7 +369,7 @@ class TcpClientChannel implements ClientChannel, 
ClientMessageHandler, ClientCon
             try {
                 T res = pendingReq.payloadReader.apply(new 
PayloadInputChannel(this, unpacker, pendingReq.notificationFut));
                 pendingReq.complete(res);
-            } catch (Exception e) {
+            } catch (Throwable e) {
                 log.error("Failed to deserialize server response 
[remoteAddress=" + cfg.getAddress() + "]: " + e.getMessage(), e);
 
                 pendingReq.completeExceptionally(
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientAsyncResultSet.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientAsyncResultSet.java
index 18265be920..f9a7401f4b 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientAsyncResultSet.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientAsyncResultSet.java
@@ -216,7 +216,7 @@ class ClientAsyncResultSet<T> implements AsyncResultSet<T> {
             try {
                 for (int i = 0; i < size; i++) {
                     var tupleReader = new BinaryTupleReader(rowSize, 
in.readBinaryUnsafe());
-                    var reader = new ClientMarshallerReader(tupleReader);
+                    var reader = new ClientMarshallerReader(tupleReader, null);
 
                     res.add((T) marshaller.readObject(reader, null));
                 }
@@ -311,7 +311,8 @@ class ClientAsyncResultSet<T> implements AsyncResultSet<T> {
                     metaColumn.name(),
                     metaColumn.type(),
                     metaColumn.nullable(),
-                    true,
+                    i,
+                    -1,
                     -1,
                     i,
                     metaColumn.scale(),
@@ -320,7 +321,7 @@ class ClientAsyncResultSet<T> implements AsyncResultSet<T> {
             schemaColumns[i] = schemaColumn;
         }
 
-        var schema = new ClientSchema(0, schemaColumns, null, marshallers);
+        var schema = new ClientSchema(0, schemaColumns, marshallers);
         return schema.getMarshaller(mapper);
     }
 }
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientSqlRow.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientSqlRow.java
index 6bbaa4e6c5..bb7bf84210 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientSqlRow.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/sql/ClientSqlRow.java
@@ -38,7 +38,7 @@ public class ClientSqlRow extends 
MutableTupleBinaryTupleAdapter implements SqlR
      * @param meta Meta.
      */
     ClientSqlRow(BinaryTupleReader row, ResultSetMetadata meta) {
-        super(row, 0, meta.columns().size(), null);
+        super(row, meta.columns().size(), null);
 
         assert row != null;
         assert meta != null;
@@ -72,26 +72,26 @@ public class ClientSqlRow extends 
MutableTupleBinaryTupleAdapter implements SqlR
 
     /** {@inheritDoc} */
     @Override
-    protected String schemaColumnName(int internalIndex) {
-        return columnName(internalIndex);
+    protected String schemaColumnName(int binaryTupleIndex) {
+        return columnName(binaryTupleIndex);
     }
 
     /** {@inheritDoc} */
     @Override
-    protected int schemaColumnIndex(String columnName) {
+    protected int binaryTupleIndex(String columnName) {
         return columnIndex(columnName);
     }
 
     /** {@inheritDoc} */
     @Override
-    protected ColumnType schemaColumnType(int columnIndex) {
-        return metadata.columns().get(columnIndex).type();
+    protected ColumnType schemaColumnType(int binaryTupleIndex) {
+        return metadata.columns().get(binaryTupleIndex).type();
     }
 
     /** {@inheritDoc} */
     @Override
-    protected int schemaDecimalScale(int columnIndex) {
-        return metadata.columns().get(columnIndex).scale();
+    protected int schemaDecimalScale(int binaryTupleIndex) {
+        return metadata.columns().get(binaryTupleIndex).scale();
     }
 
     /** {@inheritDoc} */
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/AbstractClientView.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/AbstractClientView.java
index 33ad3f074c..c420955bdf 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/AbstractClientView.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/AbstractClientView.java
@@ -67,17 +67,13 @@ abstract class AbstractClientView<T> implements 
CriteriaQuerySource<T> {
      * Map columns to it's names.
      *
      * @param columns Target columns.
-     * @param startInclusive The first index to cover.
-     * @param endExclusive Index immediately past the last index to cover.
      * @return Column names.
      */
-    protected static String[] columnNames(ClientColumn[] columns, int 
startInclusive, int endExclusive) {
-        int sz = endExclusive - startInclusive;
+    protected static String[] columnNames(ClientColumn[] columns) {
+        String[] columnNames = new String[columns.length];
 
-        String[] columnNames = new String[sz];
-
-        for (int i = 0; i < sz; i++) {
-            columnNames[i] = columns[startInclusive + i].name();
+        for (int i = 0; i < columns.length; i++) {
+            columnNames[i] = columns[i].name();
         }
 
         return columnNames;
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientColumn.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientColumn.java
index 8484bf69c8..d2de625235 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientColumn.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientColumn.java
@@ -32,10 +32,13 @@ public class ClientColumn {
     /** Nullable flag. */
     private final boolean nullable;
 
-    /** Key column flag. */
-    private final boolean isKey;
+    /** Key index (-1 for non-key). */
+    private final int keyIndex;
 
-    /** Colocation index. */
+    /** Value index (-1 for non-value). */
+    private final int valIndex;
+
+    /** Colocation index (-1 when not part of colocation index). */
     private final int colocationIndex;
 
     /** Index of the column in the schema. */
@@ -53,12 +56,13 @@ public class ClientColumn {
      * @param name Column name.
      * @param type Column type.
      * @param nullable Nullable flag.
-     * @param isKey Key column flag.
+     * @param keyIndex Key index, -1 if not part of the key.
+     * @param valIndex Val index, -1 if not part of the value.
      * @param colocationIndex Colocation index.
      * @param schemaIndex Index of the column in the schema.
      */
-    public ClientColumn(String name, ColumnType type, boolean nullable, 
boolean isKey, int colocationIndex, int schemaIndex) {
-        this(name, type, nullable, isKey, colocationIndex, schemaIndex, 0, 0);
+    public ClientColumn(String name, ColumnType type, boolean nullable, int 
keyIndex, int valIndex, int colocationIndex, int schemaIndex) {
+        this(name, type, nullable, keyIndex, valIndex, colocationIndex, 
schemaIndex, 0, 0);
     }
 
     /**
@@ -67,8 +71,9 @@ public class ClientColumn {
      * @param name Column name.
      * @param type Column type code.
      * @param nullable Nullable flag.
-     * @param isKey Key column flag.
-     * @param colocationIndex Colocation index.
+     * @param keyIndex Key index, -1 if not part of the key.
+     * @param valIndex Val index, -1 if not part of the value.
+     * @param colocationIndex Colocation index, -1 if not part of the 
colocation key.
      * @param schemaIndex Index of the column in the schema.
      * @param scale Scale of the column, if applicable.
      */
@@ -76,7 +81,8 @@ public class ClientColumn {
             String name,
             ColumnType type,
             boolean nullable,
-            boolean isKey,
+            int keyIndex,
+            int valIndex,
             int colocationIndex,
             int schemaIndex,
             int scale,
@@ -87,11 +93,15 @@ public class ClientColumn {
         this.name = name;
         this.type = type;
         this.nullable = nullable;
-        this.isKey = isKey;
+        this.keyIndex = keyIndex;
+        this.valIndex = valIndex;
         this.colocationIndex = colocationIndex;
         this.schemaIndex = schemaIndex;
         this.scale = scale;
         this.precision = precision;
+
+        assert keyIndex < 0 || valIndex < 0
+                : "Column cannot be part of both key and value: " + name + " 
(keyIndex=" + keyIndex + ", valIndex=" + valIndex + ")";
     }
 
     public String name() {
@@ -122,7 +132,7 @@ public class ClientColumn {
      * @return Value indicating whether this column is a part of key.a part of 
key
      */
     public boolean key() {
-        return isKey;
+        return keyIndex >= 0;
     }
 
     /**
@@ -134,6 +144,24 @@ public class ClientColumn {
         return colocationIndex;
     }
 
+    /**
+     * Gets the key index, or -1 when not part of the key.
+     *
+     * @return Index within the key, or -1 when not part of the key.
+     */
+    public int keyIndex() {
+        return keyIndex;
+    }
+
+    /**
+     * Gets the value index, or -1 when part of the key.
+     *
+     * @return Index within the value part, or -1 when part of the key.
+     */
+    public int valIndex() {
+        return valIndex;
+    }
+
     /**
      * Gets the index of the column in the schema.
      *
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueBinaryView.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueBinaryView.java
index d4eac97012..df02b114dd 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueBinaryView.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueBinaryView.java
@@ -469,8 +469,8 @@ public class ClientKeyValueBinaryView extends 
AbstractClientView<Entry<Tuple, Tu
     /** {@inheritDoc} */
     @Override
     protected Function<SqlRow, Entry<Tuple, Tuple>> 
queryMapper(ResultSetMetadata meta, ClientSchema schema) {
-        String[] keyCols = columnNames(schema.columns(), 0, 
schema.keyColumnCount());
-        String[] valCols = columnNames(schema.columns(), 
schema.keyColumnCount(), schema.columns().length);
+        String[] keyCols = columnNames(schema.keyColumns());
+        String[] valCols = columnNames(schema.valColumns());
 
         return (row) -> new IgniteBiTuple<>(new SqlRowProjection(row, meta, 
keyCols), new SqlRowProjection(row, meta, valCols));
     }
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
index 1416fe04bd..a5392b721a 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java
@@ -79,7 +79,7 @@ public class ClientKeyValueView<K, V> extends 
AbstractClientView<Entry<K, V>> im
      * @param keyMapper Key mapper.
      * @param valMapper value mapper.
      */
-    public ClientKeyValueView(ClientTable tbl, Mapper<K> keyMapper, Mapper<V> 
valMapper) {
+    ClientKeyValueView(ClientTable tbl, Mapper<K> keyMapper, Mapper<V> 
valMapper) {
         super(tbl);
 
         assert keyMapper != null;
@@ -509,7 +509,7 @@ public class ClientKeyValueView<K, V> extends 
AbstractClientView<Entry<K, V>> im
                 // TODO: Optimize (IGNITE-16022).
                 if (in.in().unpackBoolean()) {
                     var tupleReader = new 
BinaryTupleReader(schema.columns().length, in.in().readBinaryUnsafe());
-                    var reader = new ClientMarshallerReader(tupleReader);
+                    var reader = new ClientMarshallerReader(tupleReader, 
schema.columns());
                     res.put((K) keyMarsh.readObject(reader, null), (V) 
valMarsh.readObject(reader, null));
                 }
             }
@@ -547,7 +547,7 @@ public class ClientKeyValueView<K, V> extends 
AbstractClientView<Entry<K, V>> im
 
                     for (Entry<K, V> e : items) {
                         boolean del = deleted != null && deleted.get(i++);
-                        int colCount = del ? s.keyColumnCount() : 
s.columns().length;
+                        int colCount = del ? s.keyColumns().length : 
s.columns().length;
 
                         noValueSet.clear();
                         var builder = new BinaryTupleBuilder(colCount);
@@ -576,8 +576,8 @@ public class ClientKeyValueView<K, V> extends 
AbstractClientView<Entry<K, V>> im
     /** {@inheritDoc} */
     @Override
     protected Function<SqlRow, Entry<K, V>> queryMapper(ResultSetMetadata 
meta, ClientSchema schema) {
-        String[] keyCols = columnNames(schema.columns(), 0, 
schema.keyColumnCount());
-        String[] valCols = columnNames(schema.columns(), 
schema.keyColumnCount(), schema.columns().length);
+        String[] keyCols = columnNames(schema.keyColumns());
+        String[] valCols = columnNames(schema.valColumns());
 
         Marshaller keyMarsh = schema.getMarshaller(keySer.mapper(), 
TuplePart.KEY, true);
         Marshaller valMarsh = schema.getMarshaller(valSer.mapper(), 
TuplePart.VAL, true);
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordSerializer.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordSerializer.java
index 6443b20152..d184cc9c01 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordSerializer.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordSerializer.java
@@ -187,7 +187,7 @@ public class ClientRecordSerializer<R> {
                 : schema.getMarshaller(mapper, TuplePart.KEY);
 
         int columnCount = schema.columns().length;
-        int keyColumnCount = schema.keyColumnCount();
+        int keyColumnCount = schema.keyColumns().length;
 
         int i = 0;
 
@@ -217,8 +217,9 @@ public class ClientRecordSerializer<R> {
                 if (nullable && !in.unpackBoolean()) {
                     res.add(null);
                 } else {
-                    var tupleReader = new 
BinaryTupleReader(columnCount(schema, part), in.readBinaryUnsafe());
-                    var reader = new ClientMarshallerReader(tupleReader);
+                    ClientColumn[] columns = schema.columns(part);
+                    var tupleReader = new BinaryTupleReader(columns.length, 
in.readBinaryUnsafe());
+                    var reader = new ClientMarshallerReader(tupleReader, 
columns);
                     res.add((R) marshaller.readObject(reader, null));
                 }
             }
@@ -232,11 +233,8 @@ public class ClientRecordSerializer<R> {
     R readRec(ClientSchema schema, ClientMessageUnpacker in, TuplePart part) {
         Marshaller marshaller = schema.getMarshaller(mapper, part);
 
-        int columnCount = part == TuplePart.KEY ? schema.keyColumnCount() : 
schema.columns().length;
-        var tupleReader = new BinaryTupleReader(columnCount, 
in.readBinaryUnsafe());
-
-        int startIndex = part == TuplePart.VAL ? schema.keyColumnCount() : 0;
-        ClientMarshallerReader reader = new 
ClientMarshallerReader(tupleReader, startIndex);
+        var tupleReader = new BinaryTupleReader(schema.columns().length, 
in.readBinaryUnsafe());
+        ClientMarshallerReader reader = new 
ClientMarshallerReader(tupleReader, schema.columns(part));
 
         try {
             return (R) marshaller.readObject(reader, null);
@@ -253,7 +251,7 @@ public class ClientRecordSerializer<R> {
         Marshaller valMarshaller = schema.getMarshaller(mapper, 
TuplePart.KEY_AND_VAL);
 
         var tupleReader = new BinaryTupleReader(schema.columns().length, 
in.readBinaryUnsafe());
-        ClientMarshallerReader reader = new 
ClientMarshallerReader(tupleReader);
+        ClientMarshallerReader reader = new 
ClientMarshallerReader(tupleReader, schema.columns());
 
         try {
             return (R) valMarshaller.readObject(reader, null);
@@ -263,18 +261,6 @@ public class ClientRecordSerializer<R> {
     }
 
     private static int columnCount(ClientSchema schema, TuplePart part) {
-        switch (part) {
-            case KEY:
-                return schema.keyColumnCount();
-
-            case VAL:
-                return schema.columns().length - schema.keyColumnCount();
-
-            case KEY_AND_VAL:
-                return schema.columns().length;
-
-            default:
-                throw new IllegalArgumentException();
-        }
+        return schema.columns(part).length;
     }
 }
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordView.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordView.java
index 08d733e326..0c45841fb2 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordView.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientRecordView.java
@@ -403,7 +403,7 @@ public class ClientRecordView<R> extends 
AbstractClientView<R> implements Record
     /** {@inheritDoc} */
     @Override
     protected Function<SqlRow, R> queryMapper(ResultSetMetadata meta, 
ClientSchema schema) {
-        String[] cols = columnNames(schema.columns(), 0, 
schema.columns().length);
+        String[] cols = columnNames(schema.columns());
         Marshaller marsh = schema.getMarshaller(ser.mapper(), 
TuplePart.KEY_AND_VAL, true);
 
         return (row) -> {
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientSchema.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientSchema.java
index 326e820c38..53f39ce3b1 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientSchema.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientSchema.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.client.table;
 
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Objects;
 import org.apache.ignite.internal.client.proto.TuplePart;
 import org.apache.ignite.internal.marshaller.BinaryMode;
 import org.apache.ignite.internal.marshaller.Marshaller;
@@ -35,17 +37,20 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Client schema.
  */
-@SuppressWarnings({"rawtypes", "AssignmentOrReturnOfFieldWithMutableType", 
"unchecked"})
+@SuppressWarnings({"rawtypes", "AssignmentOrReturnOfFieldWithMutableType"})
 public class ClientSchema {
+    private static final ClientColumn[] EMPTY_COLUMNS = new ClientColumn[0];
+
     /** Schema version. Incremented on each schema modification. */
     private final int ver;
 
-    /** Key columns count. */
-    private final int keyColumnCount;
-
     /** Columns. */
     private final ClientColumn[] columns;
 
+    private final ClientColumn[] keyColumns;
+
+    private final ClientColumn[] valColumns;
+
     /** Colocation columns. */
     private final ClientColumn[] colocationColumns;
 
@@ -63,35 +68,66 @@ public class ClientSchema {
      *
      * @param ver Schema version.
      * @param columns Columns.
-     * @param colocationColumns Colocation columns. When null, all key columns 
are used.
      * @param marshallers Marshallers provider.
      */
-    public ClientSchema(int ver, ClientColumn[] columns, ClientColumn 
@Nullable [] colocationColumns, MarshallersProvider marshallers) {
+    public ClientSchema(
+            int ver,
+            ClientColumn[] columns,
+            MarshallersProvider marshallers) {
         assert ver >= 0;
         assert columns != null;
 
         this.ver = ver;
         this.columns = columns;
         this.marshallers = marshallers;
-        var keyCnt = 0;
+
+        int keyColumnCount = 0;
+        int colocationColumnCount = 0;
 
         for (var col : columns) {
+            ClientColumn existing = map.put(col.name(), col);
+            assert existing == null : "Duplicate column name: " + col.name();
+
             if (col.key()) {
-                keyCnt++;
+                keyColumnCount++;
             }
 
-            map.put(col.name(), col);
+            if (col.colocationIndex() >= 0) {
+                colocationColumnCount++;
+            }
         }
 
-        keyColumnCount = keyCnt;
+        int valColumnCount = columns.length - keyColumnCount;
+
+        this.keyColumns = keyColumnCount == 0 ? EMPTY_COLUMNS : new 
ClientColumn[keyColumnCount];
+        this.colocationColumns = colocationColumnCount == 0 ? keyColumns : new 
ClientColumn[colocationColumnCount];
+        this.valColumns = valColumnCount == 0 ? EMPTY_COLUMNS : new 
ClientColumn[valColumnCount];
+
+        for (var col : columns) {
+            if (col.key()) {
+                assert this.keyColumns[col.keyIndex()] == null : "Duplicate 
key index: name=" + col.name() + ", keyIndex=" + col.keyIndex()
+                        + ", other.name=" + 
this.keyColumns[col.keyIndex()].name();
+
+                this.keyColumns[col.keyIndex()] = col;
+            } else {
+                assert this.valColumns[col.valIndex()] == null : "Duplicate 
val index: name=" + col.name() + ", valIndex=" + col.valIndex()
+                        + ", other.name=" + 
this.valColumns[col.valIndex()].name();
 
-        if (colocationColumns == null) {
-            this.colocationColumns = new ClientColumn[keyCnt];
+                this.valColumns[col.valIndex()] = col;
+            }
+
+            if (col.colocationIndex() >= 0) {
+                assert this.colocationColumns[col.colocationIndex()] == null
+                        : "Duplicate colocation index: name=" + col.name() + 
", colocationIndex=" + col.colocationIndex()
+                        + ", other.name=" + 
this.colocationColumns[col.colocationIndex()].name();
 
-            System.arraycopy(columns, 0, this.colocationColumns, 0, keyCnt);
-        } else {
-            this.colocationColumns = colocationColumns;
+                this.colocationColumns[col.colocationIndex()] = col;
+            }
         }
+
+        assert Arrays.stream(keyColumns).allMatch(Objects::nonNull) : "Some 
key columns are missing";
+        assert Arrays.stream(valColumns).allMatch(Objects::nonNull) : "Some 
val columns are missing";
+        assert Arrays.stream(colocationColumns).allMatch(Objects::nonNull) : 
"Some colocation columns are missing";
     }
 
     /**
@@ -112,6 +148,41 @@ public class ClientSchema {
         return columns;
     }
 
+    /**
+     * Returns columns for the specified tuple part.
+     *
+     * @return Partial columns.
+     */
+    public ClientColumn[] columns(TuplePart part) {
+        if (part == TuplePart.KEY) {
+            return keyColumns;
+        }
+
+        if (part == TuplePart.VAL) {
+            return valColumns;
+        }
+
+        return columns;
+    }
+
+    /**
+     * Returns key columns.
+     *
+     * @return Key columns.
+     */
+    ClientColumn[] keyColumns() {
+        return keyColumns;
+    }
+
+    /**
+     * Returns key columns.
+     *
+     * @return Key columns.
+     */
+    ClientColumn[] valColumns() {
+        return valColumns;
+    }
+
     /**
      * Returns colocation columns.
      *
@@ -144,20 +215,11 @@ public class ClientSchema {
      * @param name Column name.
      * @return Column by name.
      */
-    public @Nullable ClientColumn columnSafe(String name) {
+    @Nullable ClientColumn columnSafe(String name) {
         return map.get(name);
     }
 
-    /**
-     * Returns key column count.
-     *
-     * @return Key column count.
-     */
-    public int keyColumnCount() {
-        return keyColumnCount;
-    }
-
-    public <T> Marshaller getMarshaller(Mapper mapper, TuplePart part) {
+    <T> Marshaller getMarshaller(Mapper mapper, TuplePart part) {
         return getMarshaller(mapper, part, part == TuplePart.KEY);
     }
 
@@ -182,25 +244,31 @@ public class ClientSchema {
     }
 
     private MarshallerColumn[] toMarshallerColumns(TuplePart part) {
-        int colCount = columns.length;
-        int firstColIdx = 0;
+        if (part == TuplePart.VAL) {
+            var res = new MarshallerColumn[columns.length - keyColumns.length];
+            int idx = 0;
+
+            for (var col : columns) {
+                if (!col.key()) {
+                    res[idx++] = marshallerColumn(col);
+                }
+            }
 
-        if (part == TuplePart.KEY) {
-            colCount = keyColumnCount;
-        } else if (part == TuplePart.VAL) {
-            colCount = columns.length - keyColumnCount;
-            firstColIdx = keyColumnCount;
+            return res;
         }
 
-        MarshallerColumn[] cols = new MarshallerColumn[colCount];
-
-        for (int i = 0; i < colCount; i++) {
-            var col = columns[i  + firstColIdx];
+        ClientColumn[] cols = part == TuplePart.KEY_AND_VAL ? columns : 
keyColumns;
+        var res = new MarshallerColumn[cols.length];
 
-            cols[i] = new MarshallerColumn(col.name(), mode(col.type()), null, 
col.scale());
+        for (int i = 0; i < cols.length; i++) {
+            res[i] = marshallerColumn(cols[i]);
         }
 
-        return cols;
+        return res;
+    }
+
+    private static MarshallerColumn marshallerColumn(ClientColumn col) {
+        return new MarshallerColumn(col.name(), mode(col.type()), null, 
col.scale());
     }
 
     private static BinaryMode mode(ColumnType dataType) {
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
index 390e8dad2e..f205bfa1df 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTable.java
@@ -220,7 +220,7 @@ public class ClientTable implements Table {
         var schemaVer = in.unpackInt();
         var colCnt = in.unpackInt();
         var columns = new ClientColumn[colCnt];
-        int colocationColumnCnt = 0;
+        int valCnt = 0;
 
         for (int i = 0; i < colCnt; i++) {
             var propCnt = in.unpackInt();
@@ -229,35 +229,22 @@ public class ClientTable implements Table {
 
             var name = in.unpackString();
             var type = ColumnTypeConverter.fromIdOrThrow(in.unpackInt());
-            var isKey = in.unpackBoolean();
+            var keyIndex = in.unpackInt();
             var isNullable = in.unpackBoolean();
             var colocationIndex = in.unpackInt();
             var scale = in.unpackInt();
             var precision = in.unpackInt();
 
+            var valIndex = keyIndex < 0 ? valCnt++ : -1;
+
             // Skip unknown extra properties, if any.
             in.skipValues(propCnt - 7);
 
-            var column = new ClientColumn(name, type, isNullable, isKey, 
colocationIndex, i, scale, precision);
+            var column = new ClientColumn(name, type, isNullable, keyIndex, 
valIndex, colocationIndex, i, scale, precision);
             columns[i] = column;
-
-            if (colocationIndex >= 0) {
-                colocationColumnCnt++;
-            }
-        }
-
-        var colocationColumns = colocationColumnCnt > 0 ? new 
ClientColumn[colocationColumnCnt] : null;
-        if (colocationColumns != null) {
-            for (ClientColumn col : columns) {
-                int idx = col.colocationIndex();
-                if (idx >= 0) {
-                    colocationColumns[idx] = col;
-                }
-            }
         }
 
-        var schema = new ClientSchema(schemaVer, columns, colocationColumns, 
marshallers);
-
+        var schema = new ClientSchema(schemaVer, columns, marshallers);
         schemas.put(schemaVer, CompletableFuture.completedFuture(schema));
 
         synchronized (latestSchemaLock) {
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
index 00bca9b4e6..0cd8773b7a 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
@@ -17,58 +17,122 @@
 
 package org.apache.ignite.internal.client.table;
 
+import java.util.Collection;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
+import org.apache.ignite.internal.client.proto.TuplePart;
 import org.apache.ignite.lang.util.IgniteNameUtils;
 import org.apache.ignite.sql.ColumnType;
+import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Client tuple. Wraps {@link BinaryTupleReader} and allows mutability.
+ *
+ * <p>The following use cases are supported:
+ * <ul>
+ *     <li>Full binary tuple, TuplePart.KEY_AND_VAL - obvious one.</li>
+ *     <li>Key only binary tuple, TuplePart.KEY - key only part,
+ *     returned from methods like {@link 
org.apache.ignite.table.RecordView#deleteAll(Transaction, Collection)}.</li>
+ *     <li>Full binary tuple, TuplePart.KEY - key part of full tuple, returned 
from KV APIs -
+ *     single binary tuple is presented as a pair of {@link ClientTuple} 
instances.</li>
+ *     <li>Full binary tuple, TuplePart.VAL - same as above, value part.</li>
+ *     <li>Value only binary tuple, TuplePart.VAL - not used currently,
+ *     but we might optimize the protocol to return value-only data for 
key-based operations.</li>
+ * </ul>
  */
 public class ClientTuple extends MutableTupleBinaryTupleAdapter {
-    /** Schema. */
     private final ClientSchema schema;
 
+    private final TuplePart part;
+
+    private final boolean fullBinaryTuple;
+
     /**
      * Constructor.
      *
      * @param schema Schema.
-     * @param tuple Tuple.
-     * @param schemaOffset Schema offset.
-     * @param schemaSize Schema size.
+     * @param part Schema part.
+     * @param binaryTuple Tuple.
      */
-    public ClientTuple(ClientSchema schema, BinaryTupleReader tuple, int 
schemaOffset, int schemaSize) {
-        super(tuple, schemaOffset, schemaSize, null);
+    public ClientTuple(ClientSchema schema, TuplePart part, BinaryTupleReader 
binaryTuple) {
+        super(binaryTuple, schema.columns(part).length, null);
+
+        assert binaryTuple.elementCount() <= schema.columns().length
+                : "Binary tuple element count is greater than schema column 
count: [binaryTuple="
+                + binaryTuple.elementCount() + ", schema=" + 
schema.columns().length + ']';
 
         this.schema = schema;
+        this.part = part;
+        this.fullBinaryTuple = binaryTuple.elementCount() == 
schema.columns().length;
+    }
+
+    @Override
+    protected String schemaColumnName(int binaryTupleIndex) {
+        return column(binaryTupleIndex).name();
+    }
+
+    @Override
+    protected int binaryTupleIndex(String columnName) {
+        return binaryTupleIndex(column(columnName));
     }
 
     @Override
-    protected String schemaColumnName(int index) {
-        return schema.columns()[index].name();
+    protected int binaryTupleIndex(int publicIndex) {
+        if (part == TuplePart.KEY_AND_VAL) {
+            return publicIndex;
+        }
+
+        return binaryTupleIndex(schema.columns(part)[publicIndex]);
+    }
+
+    private int binaryTupleIndex(@Nullable ClientColumn column) {
+        if (column == null) {
+            return -1;
+        }
+
+        if (fullBinaryTuple) {
+            return column.schemaIndex();
+        }
+
+        return part == TuplePart.KEY
+                ? column.keyIndex()
+                : column.valIndex();
     }
 
+
     @Override
-    protected int schemaColumnIndex(String columnName) {
-        ClientColumn column = column(columnName);
+    protected int publicIndex(int binaryTupleIndex) {
+        if (part == TuplePart.KEY_AND_VAL) {
+            return binaryTupleIndex;
+        }
+
+        var col = column(binaryTupleIndex);
 
-        return column == null ? -1 : column.schemaIndex();
+        return part == TuplePart.KEY ? col.keyIndex() : col.valIndex();
     }
 
     @Override
-    protected ColumnType schemaColumnType(int columnIndex) {
-        ClientColumn column = schema.columns()[columnIndex];
+    protected ColumnType schemaColumnType(int binaryTupleIndex) {
+        ClientColumn column = column(binaryTupleIndex);
 
         return column.type();
     }
 
     @Override
-    protected int schemaDecimalScale(int columnIndex) {
-        return schema.columns()[columnIndex].scale();
+    protected int schemaDecimalScale(int binaryTupleIndex) {
+        return column(binaryTupleIndex).scale();
     }
 
     @Nullable
     private ClientColumn column(String columnName) {
         return schema.columnSafe(IgniteNameUtils.parseSimpleName(columnName));
     }
+
+    private ClientColumn column(int binaryTupleIndex) {
+        if (fullBinaryTuple) {
+            return schema.columns()[binaryTupleIndex];
+        }
+
+        return schema.columns(part)[binaryTupleIndex];
+    }
 }
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTupleSerializer.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTupleSerializer.java
index a31d336423..e77758993e 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTupleSerializer.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTupleSerializer.java
@@ -130,16 +130,14 @@ public class ClientTupleSerializer {
      * @param keyOnly Key only.
      */
     public static void writeTupleRaw(Tuple tuple, ClientSchema schema, 
PayloadOutputChannel out, boolean keyOnly) {
-        var columns = schema.columns();
-        var count = keyOnly ? schema.keyColumnCount() : columns.length;
+        var columns = keyOnly ? schema.keyColumns() : schema.columns();
 
-        var builder = new BinaryTupleBuilder(count);
-        var noValueSet = new BitSet(count);
+        var builder = new BinaryTupleBuilder(columns.length);
+        var noValueSet = new BitSet(columns.length);
 
         int usedCols = 0;
 
-        for (var i = 0; i < count; i++) {
-            var col = columns[i];
+        for (ClientColumn col : columns) {
             Object v = tuple.valueOrDefault(col.name(), NO_VALUE);
 
             if (v != NO_VALUE) {
@@ -330,30 +328,24 @@ public class ClientTupleSerializer {
     }
 
     static Tuple readTuple(ClientSchema schema, ClientMessageUnpacker in, 
boolean keyOnly) {
-        var colCnt = keyOnly ? schema.keyColumnCount() : 
schema.columns().length;
-        var binTuple = new BinaryTupleReader(colCnt, in.readBinary());
+        var columns = keyOnly ? schema.keyColumns() : schema.columns();
+        var binTuple = new BinaryTupleReader(columns.length, in.readBinary());
 
-        return new ClientTuple(schema, binTuple, 0, colCnt);
+        return new ClientTuple(schema, keyOnly ? TuplePart.KEY : 
TuplePart.KEY_AND_VAL, binTuple);
     }
 
     static Tuple readValueTuple(ClientSchema schema, ClientMessageUnpacker in) 
{
-        var keyColCnt = schema.keyColumnCount();
-        var colCnt = schema.columns().length;
+        var binTuple = new BinaryTupleReader(schema.columns().length, 
in.readBinary());
 
-        var binTuple = new BinaryTupleReader(colCnt, in.readBinary());
-
-        return new ClientTuple(schema, binTuple, keyColCnt, colCnt);
+        return new ClientTuple(schema, TuplePart.VAL, binTuple);
     }
 
     private static IgniteBiTuple<Tuple, Tuple> readKvTuple(ClientSchema 
schema, ClientMessageUnpacker in) {
-        var keyColCnt = schema.keyColumnCount();
-        var colCnt = schema.columns().length;
-
-        var binTuple = new BinaryTupleReader(colCnt, in.readBinary());
-        var keyTuple2 = new ClientTuple(schema, binTuple, 0, keyColCnt);
-        var valTuple2 = new ClientTuple(schema, binTuple, keyColCnt, colCnt);
+        var binTuple = new BinaryTupleReader(schema.columns().length, 
in.readBinary());
+        var keyTuple = new ClientTuple(schema, TuplePart.KEY, binTuple);
+        var valTuple = new ClientTuple(schema, TuplePart.VAL, binTuple);
 
-        return new IgniteBiTuple<>(keyTuple2, valTuple2);
+        return new IgniteBiTuple<>(keyTuple, valTuple);
     }
 
     /**
@@ -478,7 +470,7 @@ public class ClientTupleSerializer {
         var marsh = schema.getMarshaller(mapper, TuplePart.KEY, true);
 
         for (ClientColumn col : schema.colocationColumns()) {
-            Object value = marsh.value(rec, col.schemaIndex());
+            Object value = marsh.value(rec, col.keyIndex());
             hashCalc.append(value, col.scale(), col.precision());
         }
 
@@ -486,18 +478,14 @@ public class ClientTupleSerializer {
     }
 
     private static void throwSchemaMismatchException(Tuple tuple, ClientSchema 
schema, TuplePart part) {
-        ClientColumn[] columns = schema.columns();
         Set<String> extraColumns = new HashSet<>();
-        int start = part == TuplePart.VAL ? schema.keyColumnCount() : 0;
-        int end = part == TuplePart.KEY ? schema.keyColumnCount() : 
columns.length;
 
         for (int i = 0; i < tuple.columnCount(); i++) {
             extraColumns.add(tuple.columnName(i));
         }
 
-
-        for (int i = start; i < end; i++) {
-            extraColumns.remove(columns[i].name());
+        for (var col : schema.columns(part)) {
+            extraColumns.remove(col.name());
         }
 
         String prefix = "Tuple";
diff --git 
a/modules/client/src/main/java/org/apache/ignite/internal/marshaller/ClientMarshallerReader.java
 
b/modules/client/src/main/java/org/apache/ignite/internal/marshaller/ClientMarshallerReader.java
index 062ccea1fb..37f731806e 100644
--- 
a/modules/client/src/main/java/org/apache/ignite/internal/marshaller/ClientMarshallerReader.java
+++ 
b/modules/client/src/main/java/org/apache/ignite/internal/marshaller/ClientMarshallerReader.java
@@ -27,6 +27,8 @@ import java.util.BitSet;
 import java.util.UUID;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
 import org.apache.ignite.internal.client.proto.ClientMessageUnpacker;
+import org.apache.ignite.internal.client.table.ClientColumn;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Binary reader over {@link ClientMessageUnpacker}.
@@ -35,6 +37,8 @@ public class ClientMarshallerReader implements 
MarshallerReader {
     /** Unpacker. */
     private final BinaryTupleReader unpacker;
 
+    private final ClientColumn @Nullable [] columns;
+
     /** Index. */
     private int index;
 
@@ -42,20 +46,11 @@ public class ClientMarshallerReader implements 
MarshallerReader {
      * Constructor.
      *
      * @param unpacker Unpacker.
+     * @param columns Columns.
      */
-    public ClientMarshallerReader(BinaryTupleReader unpacker) {
+    public ClientMarshallerReader(BinaryTupleReader unpacker, ClientColumn 
@Nullable [] columns) {
         this.unpacker = unpacker;
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param unpacker Unpacker.
-     * @param index Start index.
-     */
-    public ClientMarshallerReader(BinaryTupleReader unpacker, int index) {
-        this.unpacker = unpacker;
-        this.index = index;
+        this.columns = columns;
     }
 
     /** {@inheritDoc} */
@@ -67,161 +62,166 @@ public class ClientMarshallerReader implements 
MarshallerReader {
     /** {@inheritDoc} */
     @Override
     public boolean readBoolean() {
-        return unpacker.booleanValue(index++);
+        return unpacker.booleanValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Boolean readBooleanBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.booleanValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public byte readByte() {
-        return unpacker.byteValue(index++);
+        return unpacker.byteValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Byte readByteBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.byteValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public short readShort() {
-        return unpacker.shortValue(index++);
+        return unpacker.shortValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Short readShortBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.shortValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public int readInt() {
-        return unpacker.intValue(index++);
+        return unpacker.intValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Integer readIntBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.intValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public long readLong() {
-        return unpacker.longValue(index++);
+        return unpacker.longValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Long readLongBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.longValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public float readFloat() {
-        return unpacker.floatValue(index++);
+        return unpacker.floatValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Float readFloatBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.floatValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public double readDouble() {
-        return unpacker.doubleValue(index++);
+        return unpacker.doubleValue(nextSchemaIndex());
     }
 
     /** {@inheritDoc} */
     @Override
     public Double readDoubleBoxed() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.doubleValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public String readString() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.stringValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public UUID readUuid() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.uuidValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public byte[] readBytes() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.bytesValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public BitSet readBitSet() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.bitmaskValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public BigInteger readBigInt() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.numberValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public BigDecimal readBigDecimal(int scale) {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.decimalValue(idx, 
scale);
     }
 
     /** {@inheritDoc} */
     @Override
     public LocalDate readDate() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.dateValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public LocalTime readTime() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.timeValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public Instant readTimestamp() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : 
unpacker.timestampValue(idx);
     }
 
     /** {@inheritDoc} */
     @Override
     public LocalDateTime readDateTime() {
-        var idx = index++;
+        var idx = nextSchemaIndex();
         return unpacker.hasNullValue(idx) ? null : unpacker.dateTimeValue(idx);
     }
+
+    private int nextSchemaIndex() {
+        int i = index++;
+        return columns == null ? i : columns[i].schemaIndex();
+    }
 }
diff --git 
a/modules/client/src/test/java/org/apache/ignite/client/ClientTupleTest.java 
b/modules/client/src/test/java/org/apache/ignite/client/ClientTupleTest.java
index f36b23f6b5..790d009416 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/ClientTupleTest.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/ClientTupleTest.java
@@ -44,6 +44,7 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
+import org.apache.ignite.internal.client.proto.TuplePart;
 import org.apache.ignite.internal.client.table.ClientColumn;
 import org.apache.ignite.internal.client.table.ClientSchema;
 import org.apache.ignite.internal.client.table.ClientTuple;
@@ -51,6 +52,8 @@ import 
org.apache.ignite.internal.marshaller.ReflectionMarshallersProvider;
 import org.apache.ignite.sql.ColumnType;
 import org.apache.ignite.table.Tuple;
 import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
 
 /**
  * Tests client tuple builder implementation.
@@ -61,31 +64,31 @@ public class ClientTupleTest {
     private static final ReflectionMarshallersProvider marshallers = new 
ReflectionMarshallersProvider();
 
     private static final ClientSchema SCHEMA = new ClientSchema(1, new 
ClientColumn[]{
-            new ClientColumn("ID", ColumnType.INT64, false, true, 0, 0),
-            new ClientColumn("NAME", ColumnType.STRING, false, false, -1, 1)
-    }, null, marshallers);
+            new ClientColumn("NAME", ColumnType.STRING, false, -1, 0, -1, 0),
+            new ClientColumn("ID", ColumnType.INT64, false, 0, -1, 0, 1)
+    }, marshallers);
 
     private static final ClientSchema FULL_SCHEMA = new ClientSchema(100, new 
ClientColumn[]{
-            new ClientColumn("I8", ColumnType.INT8, false, false, -1, 0),
-            new ClientColumn("I16", ColumnType.INT16, false, false, -1, 1),
-            new ClientColumn("I32", ColumnType.INT32, false, false, -1, 2),
-            new ClientColumn("I64", ColumnType.INT64, false, false, -1, 3),
-            new ClientColumn("FLOAT", ColumnType.FLOAT, false, false, -1, 4),
-            new ClientColumn("DOUBLE", ColumnType.DOUBLE, false, false, -1, 5),
-            new ClientColumn("UUID", ColumnType.UUID, false, false, -1, 6),
-            new ClientColumn("STR", ColumnType.STRING, false, false, -1, 7),
-            new ClientColumn("BITS", ColumnType.BITMASK, false, false, -1, 8),
-            new ClientColumn("DATE", ColumnType.DATE, false, false, -1, 9),
-            new ClientColumn("TIME", ColumnType.TIME, false, false, -1, 10),
-            new ClientColumn("DATETIME", ColumnType.DATETIME, false, false, 
-1, 11),
-            new ClientColumn("TIMESTAMP", ColumnType.TIMESTAMP, false, false, 
-1, 12),
-            new ClientColumn("BOOL", ColumnType.BOOLEAN, false, false, -1, 13),
-            new ClientColumn("DECIMAL", ColumnType.DECIMAL, false, false, -1, 
14),
-            new ClientColumn("BYTES", ColumnType.BYTE_ARRAY, false, false, -1, 
15),
-            new ClientColumn("PERIOD", ColumnType.PERIOD, false, false, -1, 
16),
-            new ClientColumn("DURATION", ColumnType.DURATION, false, false, 
-1, 17),
-            new ClientColumn("NUMBER", ColumnType.NUMBER, false, false, -1, 18)
-    }, null, marshallers);
+            new ClientColumn("I8", ColumnType.INT8, false, -1, 0, -1, 0),
+            new ClientColumn("I16", ColumnType.INT16, false, -1, 1, -1, 1),
+            new ClientColumn("I32", ColumnType.INT32, false, 0, -1, -1, 2),
+            new ClientColumn("I64", ColumnType.INT64, false, 1, -1, -1, 3),
+            new ClientColumn("FLOAT", ColumnType.FLOAT, false, -1, 2, -1, 4),
+            new ClientColumn("DOUBLE", ColumnType.DOUBLE, false, -1, 3, -1, 5),
+            new ClientColumn("UUID", ColumnType.UUID, false, -1, 4, -1, 6),
+            new ClientColumn("STR", ColumnType.STRING, false, 2, -1, -1, 7),
+            new ClientColumn("BITS", ColumnType.BITMASK, false, -1, 5, -1, 8),
+            new ClientColumn("DATE", ColumnType.DATE, false, -1, 6, -1, 9),
+            new ClientColumn("TIME", ColumnType.TIME, false, -1, 7, -1, 10),
+            new ClientColumn("DATETIME", ColumnType.DATETIME, false, -1, 8, 
-1, 11),
+            new ClientColumn("TIMESTAMP", ColumnType.TIMESTAMP, false, -1, 9, 
-1, 12),
+            new ClientColumn("BOOL", ColumnType.BOOLEAN, false, -1, 10, -1, 
13),
+            new ClientColumn("DECIMAL", ColumnType.DECIMAL, false, -1, 11, -1, 
14),
+            new ClientColumn("BYTES", ColumnType.BYTE_ARRAY, false, -1, 12, 
-1, 15),
+            new ClientColumn("PERIOD", ColumnType.PERIOD, false, -1, 13, -1, 
16),
+            new ClientColumn("DURATION", ColumnType.DURATION, false, -1, 14, 
-1, 17),
+            new ClientColumn("NUMBER", ColumnType.NUMBER, false, -1, 15, -1, 
18)
+    }, marshallers);
 
     private static final UUID GUID = UUID.randomUUID();
 
@@ -105,8 +108,8 @@ public class ClientTupleTest {
 
     @Test
     public void testValueReturnsValueByIndex() {
-        assertEquals(3L, (Long) createTuple().value(0));
-        assertEquals("Shirt", createTuple().value(1));
+        assertEquals(3L, (Long) createTuple().value(1));
+        assertEquals("Shirt", createTuple().value(0));
     }
 
     @Test
@@ -122,7 +125,7 @@ public class ClientTupleTest {
 
     @Test
     public void testValueOrDefaultReturnsNullWhenColumnIsSetToNull() {
-        var tuple = createTuple().set("name", null);
+        Tuple tuple = createTuple().set("name", null);
 
         assertNull(tuple.valueOrDefault("name", "foo"));
     }
@@ -141,10 +144,34 @@ public class ClientTupleTest {
         assertEquals(SCHEMA.columns().length, createTuple().columnCount());
     }
 
+    @ParameterizedTest
+    @ValueSource(booleans = {false, true})
+    public void testColumnCountKeyOnlyReturnsKeySize(boolean partialData) {
+        assertEquals(FULL_SCHEMA.columns(TuplePart.KEY).length, 
createFullSchemaTuple(TuplePart.KEY, partialData).columnCount());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {false, true})
+    public void testColumnCountValOnlyReturnsValSize(boolean partialData) {
+        assertEquals(FULL_SCHEMA.columns(TuplePart.VAL).length, 
createFullSchemaTuple(TuplePart.VAL, partialData).columnCount());
+    }
+
     @Test
     public void testColumnNameReturnsNameByIndex() {
-        assertEquals("ID", createTuple().columnName(0));
-        assertEquals("NAME", createTuple().columnName(1));
+        assertEquals("ID", createTuple().columnName(1));
+        assertEquals("NAME", createTuple().columnName(0));
+    }
+
+    @Test
+    public void testColumnNameReturnsNameByIndexKeyOnly() {
+        assertEquals("I32", createFullSchemaTuple(TuplePart.KEY, 
false).columnName(0));
+        assertEquals("I64", createFullSchemaTuple(TuplePart.KEY, 
false).columnName(1));
+    }
+
+    @Test
+    public void testColumnNameReturnsNameByIndexValOnly() {
+        assertEquals("I8", createFullSchemaTuple(TuplePart.VAL, 
false).columnName(0));
+        assertEquals("I16", createFullSchemaTuple(TuplePart.VAL, 
false).columnName(1));
     }
 
     @Test
@@ -155,13 +182,46 @@ public class ClientTupleTest {
 
     @Test
     public void testColumnIndexReturnsIndexByName() {
-        assertEquals(0, createTuple().columnIndex("id"));
-        assertEquals(1, createTuple().columnIndex("name"));
+        assertEquals(1, createTuple().columnIndex("id"));
+        assertEquals(0, createTuple().columnIndex("name"));
+
+        assertEquals(2, createFullSchemaTuple().columnIndex("I32"));
+        assertEquals(3, createFullSchemaTuple().columnIndex("I64"));
+        assertEquals(7, createFullSchemaTuple().columnIndex("STR"));
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {false, true})
+    public void testColumnIndexReturnsIndexByNameKeyOnly(boolean partialData) {
+        assertEquals(0, createFullSchemaTuple(TuplePart.KEY, 
partialData).columnIndex("I32"));
+        assertEquals(1, createFullSchemaTuple(TuplePart.KEY, 
partialData).columnIndex("I64"));
+        assertEquals(2, createFullSchemaTuple(TuplePart.KEY, 
partialData).columnIndex("STR"));
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {false, true})
+    public void testColumnIndexReturnsIndexByNameValOnly(boolean partialData) {
+        assertEquals(0, createFullSchemaTuple(TuplePart.VAL, 
partialData).columnIndex("I8"));
+        assertEquals(1, createFullSchemaTuple(TuplePart.VAL, 
partialData).columnIndex("I16"));
+        assertEquals(4, createFullSchemaTuple(TuplePart.VAL, 
partialData).columnIndex("UUID"));
     }
 
     @Test
     public void testColumnIndexForMissingColumns() {
         assertEquals(-1, createTuple().columnIndex("foo"));
+        assertEquals(-1, createFullSchemaTuple().columnIndex("UUID1"));
+    }
+
+    @Test
+    public void testColumnIndexForMissingColumnsKeyOnly() {
+        assertEquals(-1, createFullSchemaTuple(TuplePart.KEY, 
true).columnIndex("foo"));
+        assertEquals(-1, createFullSchemaTuple(TuplePart.KEY, 
true).columnIndex("UUID"));
+    }
+
+    @Test
+    public void testColumnIndexForMissingColumnsValOnly() {
+        assertEquals(-1, createFullSchemaTuple(TuplePart.VAL, 
true).columnIndex("foo"));
+        assertEquals(-1, createFullSchemaTuple(TuplePart.VAL, 
true).columnIndex("I32"));
     }
 
     @Test
@@ -301,42 +361,120 @@ public class ClientTupleTest {
         }
     }
 
+    @Test
+    public void testKeyOnlyTupleEquality() {
+        var keyTupleFullData = createFullSchemaTuple(TuplePart.KEY, false);
+        var keyTuplePartialData = createFullSchemaTuple(TuplePart.KEY, true);
+        var keyTupleUser = Tuple.create().set("I32", 3).set("I64", 
4L).set("STR", "8");
+
+        assertEquals(keyTupleFullData, keyTuplePartialData);
+        assertEquals(keyTupleUser, keyTupleFullData);
+        assertEquals(keyTupleUser, keyTuplePartialData);
+    }
+
+    @Test
+    public void testValOnlyTupleEquality() {
+        var valTupleFullData = createFullSchemaTuple(TuplePart.VAL, false);
+        var valTuplePartialData = createFullSchemaTuple(TuplePart.VAL, true);
+
+        var valTupleUser = Tuple.create()
+                .set("I8", (byte) 1)
+                .set("I16", (short) 2)
+                .set("FLOAT", 5.5f)
+                .set("DOUBLE", 6.6)
+                .set("UUID", GUID)
+                .set("BITS", new BitSet(3))
+                .set("DATE", DATE)
+                .set("TIME", TIME)
+                .set("DATETIME", DATE_TIME)
+                .set("TIMESTAMP", TIMESTAMP)
+                .set("BOOL", true)
+                .set("DECIMAL", BigDecimal.valueOf(1234))
+                .set("BYTES", new byte[]{1, 2, 3})
+                .set("PERIOD", Period.ofDays(16))
+                .set("DURATION", Duration.ofDays(17))
+                .set("NUMBER", BigInteger.valueOf(18));
+
+        assertEquals(valTupleFullData, valTuplePartialData);
+        assertEquals(valTupleUser, valTupleFullData);
+        assertEquals(valTupleUser, valTuplePartialData);
+    }
+
     private static Tuple createTuple() {
         var binTupleBuf = new BinaryTupleBuilder(SCHEMA.columns().length)
-                .appendLong(3L)
                 .appendString("Shirt")
+                .appendLong(3L)
                 .build();
 
         var binTuple = new BinaryTupleReader(SCHEMA.columns().length, 
binTupleBuf);
 
-        return new ClientTuple(SCHEMA, binTuple, 0, SCHEMA.columns().length);
+        return new ClientTuple(SCHEMA, TuplePart.KEY_AND_VAL, binTuple);
     }
 
     private static ClientTuple createFullSchemaTuple() {
+        return createFullSchemaTuple(TuplePart.KEY_AND_VAL, false);
+    }
+
+    private static ClientTuple createFullSchemaTuple(TuplePart part, boolean 
partialData) {
         var binTupleBuf = new BinaryTupleBuilder(FULL_SCHEMA.columns().length)
-                .appendByte((byte) 1)
-                .appendShort((short) 2)
-                .appendInt(3)
-                .appendLong(4)
-                .appendFloat(5.5f)
-                .appendDouble(6.6)
-                .appendUuid(GUID)
-                .appendString("8")
-                .appendBitmask(new BitSet(3))
-                .appendDate(DATE)
-                .appendTime(TIME)
-                .appendDateTime(DATE_TIME)
-                .appendTimestamp(TIMESTAMP)
-                .appendByte((byte) 1)
-                .appendDecimal(BigDecimal.valueOf(1.234), 3)
-                .appendBytes(new byte[] {1, 2, 3})
-                .appendPeriod(Period.ofDays(16))
-                .appendDuration(Duration.ofDays(17))
-                .appendNumber(BigInteger.valueOf(18))
-                .build();
+                        .appendByte((byte) 1)
+                        .appendShort((short) 2)
+                        .appendInt(3)
+                        .appendLong(4)
+                        .appendFloat(5.5f)
+                        .appendDouble(6.6)
+                        .appendUuid(GUID)
+                        .appendString("8")
+                        .appendBitmask(new BitSet(3))
+                        .appendDate(DATE)
+                        .appendTime(TIME)
+                        .appendDateTime(DATE_TIME)
+                        .appendTimestamp(TIMESTAMP)
+                        .appendByte((byte) 1)
+                        .appendDecimal(BigDecimal.valueOf(1.234), 3)
+                        .appendBytes(new byte[] {1, 2, 3})
+                        .appendPeriod(Period.ofDays(16))
+                        .appendDuration(Duration.ofDays(17))
+                        .appendNumber(BigInteger.valueOf(18))
+                        .build();
+
+        var binTupleColumnCount = FULL_SCHEMA.columns().length;
+
+        if (part == TuplePart.KEY && partialData) {
+            binTupleBuf = new BinaryTupleBuilder(3)
+                    .appendInt(3)
+                    .appendLong(4)
+                    .appendString("8")
+                    .build();
+
+            binTupleColumnCount = 3;
+        }
+
+        if (part == TuplePart.VAL && partialData) {
+            binTupleBuf = new BinaryTupleBuilder(16)
+                    .appendByte((byte) 1)
+                    .appendShort((short) 2)
+                    .appendFloat(5.5f)
+                    .appendDouble(6.6)
+                    .appendUuid(GUID)
+                    .appendBitmask(new BitSet(3))
+                    .appendDate(DATE)
+                    .appendTime(TIME)
+                    .appendDateTime(DATE_TIME)
+                    .appendTimestamp(TIMESTAMP)
+                    .appendByte((byte) 1)
+                    .appendDecimal(BigDecimal.valueOf(1.234), 3)
+                    .appendBytes(new byte[] {1, 2, 3})
+                    .appendPeriod(Period.ofDays(16))
+                    .appendDuration(Duration.ofDays(17))
+                    .appendNumber(BigInteger.valueOf(18))
+                    .build();
+
+            binTupleColumnCount = 16;
+        }
 
-        var binTuple = new BinaryTupleReader(FULL_SCHEMA.columns().length, 
binTupleBuf);
+        var binTuple = new BinaryTupleReader(binTupleColumnCount, binTupleBuf);
 
-        return new ClientTuple(FULL_SCHEMA, binTuple, 0, 
FULL_SCHEMA.columns().length);
+        return new ClientTuple(FULL_SCHEMA, part, binTuple);
     }
 }
diff --git 
a/modules/client/src/test/java/org/apache/ignite/internal/client/table/ColocationHashTests.java
 
b/modules/client/src/test/java/org/apache/ignite/internal/client/table/ColocationHashTests.java
new file mode 100644
index 0000000000..3422437f44
--- /dev/null
+++ 
b/modules/client/src/test/java/org/apache/ignite/internal/client/table/ColocationHashTests.java
@@ -0,0 +1,81 @@
+/*
+ * 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.client.table;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import org.apache.ignite.internal.marshaller.ReflectionMarshallersProvider;
+import org.apache.ignite.sql.ColumnType;
+import org.apache.ignite.table.Tuple;
+import org.apache.ignite.table.mapper.Mapper;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for colocation hash functionality in {@link ClientTupleSerializer}.
+ */
+public class ColocationHashTests {
+    private static final ReflectionMarshallersProvider marshallers = new 
ReflectionMarshallersProvider();
+
+    // Key:        key1, key2
+    // Colocation: key2, key1
+    private static final ClientSchema SCHEMA = new ClientSchema(1, new 
ClientColumn[]{
+            new ClientColumn("VAL1", ColumnType.INT32, false, -1, 0, -1, 0),
+            new ClientColumn("KEY1", ColumnType.INT32, false, 0, -1, 1, 1),
+            new ClientColumn("VAL2", ColumnType.INT32, false, -1, 1, -1, 2),
+            new ClientColumn("KEY2", ColumnType.INT32, false, 1, -1, 0, 3),
+    }, marshallers);
+
+    @Test
+    public void testPojoInterleavedKeyAndValueColumns() {
+        var pojo = new Pojo();
+        pojo.val1 = 1;
+        pojo.key1 = 2;
+        pojo.val2 = 3;
+        pojo.key2 = 4;
+
+        var keyPojo = new Pojo();
+        keyPojo.key1 = pojo.key1;
+        keyPojo.key2 = pojo.key2;
+
+        var tuple = Tuple.create()
+                .set("VAL1", pojo.val1)
+                .set("KEY1", pojo.key1)
+                .set("VAL2", pojo.val2)
+                .set("KEY2", pojo.key2);
+
+        var keyTuple = Tuple.create()
+                .set("KEY1", pojo.key1)
+                .set("KEY2", pojo.key2);
+
+        Integer tupleHash = ClientTupleSerializer.getColocationHash(SCHEMA, 
tuple);
+        Integer keyTupleHash = ClientTupleSerializer.getColocationHash(SCHEMA, 
keyTuple);
+        Integer pojoHash = ClientTupleSerializer.getColocationHash(SCHEMA, 
Mapper.of(Pojo.class), pojo);
+        Integer keyPojoHash = ClientTupleSerializer.getColocationHash(SCHEMA, 
Mapper.of(Pojo.class), keyPojo);
+
+        assertEquals(tupleHash, keyTupleHash);
+        assertEquals(tupleHash, pojoHash);
+        assertEquals(tupleHash, keyPojoHash);
+    }
+
+    static class Pojo {
+        int val1;
+        int key1;
+        int val2;
+        int key2;
+    }
+}
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/util/HashCalculator.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/util/HashCalculator.java
index 84834f3ca7..addffc938d 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/util/HashCalculator.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/util/HashCalculator.java
@@ -30,6 +30,7 @@ import java.time.LocalTime;
 import java.util.BitSet;
 import java.util.UUID;
 import org.apache.ignite.internal.lang.IgniteInternalException;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Implement hash calculator.
@@ -44,7 +45,7 @@ public class HashCalculator {
      * @param scale Decimal scale.
      * @param precision Temporal precision.
      */
-    public void append(Object v, int scale, int precision) {
+    public void append(@Nullable Object v, int scale, int precision) {
         if (v == null) {
             appendNull();
             return;
diff --git a/modules/platforms/cpp/ignite/client/detail/table/schema.h 
b/modules/platforms/cpp/ignite/client/detail/table/schema.h
index d521ad5a0f..9d35828ce7 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/schema.h
+++ b/modules/platforms/cpp/ignite/client/detail/table/schema.h
@@ -53,7 +53,7 @@ struct column {
         column res{};
         res.name = reader.read_string();
         res.type = static_cast<ignite_type>(reader.read_int32());
-        res.is_key = reader.read_bool();
+        res.is_key = reader.read_int32() >= 0;
         res.nullable = reader.read_bool();
         reader.skip(); // Colocation index.
         res.scale = reader.read_int32();
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs 
b/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
index 404b78bf96..52b2ec9cb4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
@@ -571,7 +571,7 @@ namespace Apache.Ignite.Tests
                 writer.Write(7); // Column props.
                 writer.Write("ID");
                 writer.Write((int)ColumnType.Int32);
-                writer.Write(true); // Key.
+                writer.Write(0); // Key index.
                 writer.Write(false); // Nullable.
                 writer.Write(0); // Colocation index.
                 writer.Write(0); // Scale.
@@ -584,7 +584,7 @@ namespace Apache.Ignite.Tests
                 writer.Write(7); // Column props.
                 writer.Write("IdStr");
                 writer.Write((int)ColumnType.String);
-                writer.Write(true); // Key.
+                writer.Write(0); // Key index.
                 writer.Write(false); // Nullable.
                 writer.Write(0); // Colocation index.
                 writer.Write(0); // Scale.
@@ -593,7 +593,7 @@ namespace Apache.Ignite.Tests
                 writer.Write(7); // Column props.
                 writer.Write("IdGuid");
                 writer.Write((int)ColumnType.Uuid);
-                writer.Write(true); // Key.
+                writer.Write(1); // Key index.
                 writer.Write(false); // Nullable.
                 writer.Write(1); // Colocation index.
                 writer.Write(0); // Scale.
@@ -606,7 +606,7 @@ namespace Apache.Ignite.Tests
                 writer.Write(7); // Column props.
                 writer.Write("IdStr");
                 writer.Write((int)ColumnType.String);
-                writer.Write(true); // Key.
+                writer.Write(0); // Key index.
                 writer.Write(false); // Nullable.
                 writer.Write(0); // Colocation index.
                 writer.Write(0); // Scale.
@@ -615,7 +615,7 @@ namespace Apache.Ignite.Tests
                 writer.Write(7); // Column props.
                 writer.Write("IdGuid");
                 writer.Write((int)ColumnType.Uuid);
-                writer.Write(true); // Key.
+                writer.Write(1); // Key index.
                 writer.Write(false); // Nullable.
                 writer.Write(-1); // Colocation index.
                 writer.Write(0); // Scale.
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs 
b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs
index 754d57e900..4f39ad1ea3 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs
@@ -346,7 +346,8 @@ namespace Apache.Ignite.Internal.Table
 
                 var name = r.ReadString();
                 var type = r.ReadInt32();
-                var isKey = r.ReadBoolean();
+                var keyIndex = r.ReadInt32();
+                var isKey = keyIndex >= 0;
                 var isNullable = r.ReadBoolean();
                 var colocationIndex = r.ReadInt32();
                 var scale = r.ReadInt32();
diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java
index 074d763f17..6ec9c7417b 100644
--- 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItThinClientColocationTest.java
@@ -111,13 +111,14 @@ public class ItThinClientColocationTest extends 
ClusterPerClassIntegrationTest {
                 columnName,
                 ClientTableCommon.getColumnType(type.spec()),
                 false,
-                true,
+                0,
+                -1,
                 -1,
                 0,
                 ClientTableCommon.getDecimalScale(type),
                 ClientTableCommon.getPrecision(type));
 
-        return new ClientSchema(0, new ClientColumn[]{clientColumn}, null, 
marshallers);
+        return new ClientSchema(0, new ClientColumn[]{clientColumn}, 
marshallers);
     }
 
     private static TupleMarshallerImpl tupleMarshaller(NativeType type, String 
columnName) {
diff --git 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
index 4de80f6c8d..cea37c4b2c 100644
--- 
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
+++ 
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaDescriptor.java
@@ -60,6 +60,9 @@ public class SchemaDescriptor {
     /** Colocation columns. */
     private final @Nullable Map<Column, Integer> colocationColIndexes;
 
+    /** Key columns indexes. */
+    private final Map<Column, Integer> keyColIndexes;
+
     /** Mapping 'Column name' -&gt; Column. */
     private final Map<String, Column> colMap;
 
@@ -139,6 +142,13 @@ public class SchemaDescriptor {
                 this.colocationColIndexes.put(col, i);
             }
         }
+
+        // TODO: Move keyIndex and colocationIndex to Column class for faster 
and simpler access?
+        keyColIndexes = new HashMap<>(keyCols.length);
+
+        for (int i = 0; i < keyCols.length; i++) {
+            keyColIndexes.put(keyCols[i], i);
+        }
     }
 
     /**
@@ -242,6 +252,16 @@ public class SchemaDescriptor {
                 : colocationColIndexes.getOrDefault(col, -1);
     }
 
+    /**
+     * Get key index of the specified column.
+     *
+     * @param col Column.
+     * @return Index in the keyColumns array, or -1 when not a part of the key.
+     */
+    public int keyIndex(Column col) {
+        return keyColIndexes.getOrDefault(col, -1);
+    }
+
     /**
      * Get value columns.
      *

Reply via email to