You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tk...@apache.org on 2023/05/26 11:45:30 UTC

[ignite-3] branch main updated: IGNITE-19531 Switch table IDs from UUIDs to integers (#2093)

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

tkalkirill 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 317aca963e IGNITE-19531 Switch table IDs from UUIDs to integers (#2093)
317aca963e is described below

commit 317aca963ed9c87e14dd6eebd3284408fb550c8c
Author: Roman Puchkovskiy <ro...@gmail.com>
AuthorDate: Fri May 26 15:45:24 2023 +0400

    IGNITE-19531 Switch table IDs from UUIDs to integers (#2093)
---
 .../handler/requests/table/ClientTableCommon.java  |   3 +-
 .../requests/table/ClientTableGetRequest.java      |   2 +-
 .../ClientTablePartitionAssignmentGetRequest.java  |   3 +-
 .../requests/table/ClientTablesGetRequest.java     |   2 +-
 .../internal/client/compute/ClientCompute.java     |   4 +-
 .../internal/client/table/ClientKeyValueView.java  |   2 +-
 .../client/table/ClientRecordSerializer.java       |  12 ++-
 .../ignite/internal/client/table/ClientTable.java  |  12 ++-
 .../ignite/internal/client/table/ClientTables.java |   4 +-
 .../client/table/ClientTupleSerializer.java        |  12 +--
 .../ignite/client/PartitionAwarenessTest.java      |   8 +-
 .../ignite/client/fakes/FakeIgniteTables.java      |  19 ++--
 .../ignite/client/fakes/FakeInternalTable.java     |   7 +-
 .../internal/replicator/TablePartitionId.java      |  15 ++--
 .../rebalance/DistributionZoneRebalanceEngine.java |   5 +-
 .../rebalance/TablePartitionId.java                |  14 ++-
 .../DistributionZoneRebalanceEngineTest.java       |  29 ++++--
 .../RebalanceUtilUpdateAssignmentsTest.java        |   3 +-
 .../ignite/internal/index/BuildIndexTaskId.java    |  10 +--
 .../apache/ignite/internal/index/HashIndex.java    |   2 +-
 .../org/apache/ignite/internal/index/Index.java    |   2 +-
 .../apache/ignite/internal/index/IndexManager.java |  33 ++++---
 .../ignite/internal/index/SortedIndexImpl.java     |   2 +-
 .../internal/index/event/IndexEventParameters.java |   8 +-
 .../ignite/internal/index/IndexManagerTest.java    |  14 +--
 .../serialization/BaseMethodNameResolver.java      |   2 +
 .../network/serialization/MessageReader.java       |   9 ++
 .../network/serialization/MessageWriter.java       |  10 +++
 .../network/direct/DirectMessageReader.java        |  11 +++
 .../network/direct/DirectMessageWriter.java        |   9 ++
 .../direct/stream/DirectByteBufferStream.java      |  15 ++++
 .../stream/DirectByteBufferStreamImplV1.java       |  32 +++++++
 .../network/processor/AllTypesMessage.java         |   2 +
 .../MultiActorPlacementDriverTest.java             |  13 +--
 .../PlacementDriverManagerTest.java                |  13 +--
 .../placementdriver/AssignmentsTracker.java        |   3 +-
 .../placementdriver/PlacementDriverTest.java       |   3 +-
 .../cpp/ignite/client/detail/table/table_impl.cpp  |   2 +-
 .../cpp/ignite/client/detail/table/table_impl.h    |   6 +-
 .../cpp/ignite/client/detail/table/tables_impl.cpp |   6 +-
 .../dotnet/Apache.Ignite.Tests/FakeServer.cs       |   8 +-
 .../dotnet/Apache.Ignite/Internal/Table/Table.cs   |   4 +-
 .../dotnet/Apache.Ignite/Internal/Table/Tables.cs  |  10 +--
 .../storage/ItRebalanceDistributedTest.java        |   3 +-
 .../inmemory/ItRaftStorageVolatilityTest.java      |   3 +-
 .../app/ItIgniteInMemoryNodeRestartTest.java       |  10 +--
 .../internal/runner/app/ItTablesApiTest.java       |   3 +-
 .../internal/sql/engine/ItSecondaryIndexTest.java  |   2 +-
 .../ignite/internal/schema/SchemaManager.java      |  52 ++++++-----
 .../ExtendedTableConfigurationSchema.java          |   6 --
 .../configuration/TableConfigurationSchema.java    |  12 ++-
 .../configuration/index/IndexValidatorImpl.java    |  16 +++-
 .../index/TableIndexConfigurationSchema.java       |   4 +-
 .../schema/event/SchemaEventParameters.java        |   7 +-
 .../index/IndexValidatorImplTest.java              |  19 ++--
 .../testutils/SchemaConfigurationConverter.java    |   2 +-
 .../sql/engine/exec/ExecutionServiceImpl.java      |   6 +-
 .../internal/sql/engine/externalize/RelJson.java   |   5 +-
 .../internal/sql/engine/rel/IgniteTableModify.java |   2 +-
 .../engine/rel/ProjectableFilterableTableScan.java |   2 +-
 .../internal/sql/engine/schema/IgniteTable.java    |   3 +-
 .../sql/engine/schema/IgniteTableImpl.java         |  15 +---
 .../sql/engine/schema/SqlSchemaManager.java        |   3 +-
 .../sql/engine/schema/SqlSchemaManagerImpl.java    |  32 ++++---
 .../sql/engine/trait/DistributionFunction.java     |   9 +-
 .../sql/engine/trait/IgniteDistributions.java      |   5 +-
 .../sql/engine/util/HashFunctionFactory.java       |   4 +-
 .../sql/engine/util/HashFunctionFactoryImpl.java   |   3 +-
 .../internal/sql/engine/StopCalciteModuleTest.java |   5 +-
 .../sql/engine/exec/ExecutionServiceImplTest.java  |   3 +-
 .../exec/rel/TableScanNodeExecutionTest.java       |   2 +-
 .../engine/exec/schema/SqlSchemaManagerTest.java   |  10 +--
 .../sql/engine/framework/NoOpTransaction.java      |   2 +-
 .../engine/framework/PredefinedSchemaManager.java  |   5 +-
 .../internal/sql/engine/framework/TestTable.java   |   5 +-
 .../planner/AbstractAggregatePlannerTest.java      |   4 +-
 .../sql/engine/planner/AbstractPlannerTest.java    |  26 ++++--
 .../sql/engine/planner/DmlPlannerTest.java         |   5 +-
 .../sql/engine/planner/HashIndexPlannerTest.java   |   5 +-
 .../engine/planner/HashIndexSpoolPlannerTest.java  |  13 ++-
 .../engine/planner/JoinColocationPlannerTest.java  |  13 ++-
 .../sql/engine/planner/JoinCommutePlannerTest.java |   5 +-
 .../internal/sql/engine/planner/PlannerTest.java   |  21 +++--
 .../sql/engine/planner/SetOpPlannerTest.java       |  13 ++-
 .../planner/SortedIndexSpoolPlannerTest.java       |  14 ++-
 .../sql/engine/planner/TableSpoolPlannerTest.java  |   6 +-
 .../sql/engine/planner/UnionPlannerTest.java       |   8 +-
 .../internal/storage/MvPartitionStorage.java       |   4 +-
 .../apache/ignite/internal/storage/ReadResult.java |   8 +-
 .../storage/index/HashIndexDescriptor.java         |  14 ++-
 .../storage/index/SortedIndexDescriptor.java       |  13 ++-
 .../storage/AbstractMvPartitionStorageTest.java    |   2 +-
 .../storage/AbstractMvTableStorageTest.java        |  13 +--
 .../storage/BaseMvPartitionStorageTest.java        |   2 +-
 .../storage/impl/TestMvPartitionStorage.java       |  17 ++--
 .../index/AbstractHashIndexStorageTest.java        |   4 +-
 .../index/AbstractSortedIndexStorageTest.java      |   4 +-
 .../PersistentPageMemoryTableStorage.java          |  24 ++---
 .../pagememory/VolatilePageMemoryTableStorage.java |  10 ++-
 .../mv/AbstractPageMemoryMvPartitionStorage.java   |   6 +-
 .../pagememory/mv/AddWriteInvokeClosure.java       |   8 +-
 .../mv/PersistentPageMemoryMvPartitionStorage.java |   4 +-
 .../storage/pagememory/mv/VersionChain.java        |   8 +-
 .../storage/pagememory/mv/io/VersionChainIo.java   |  21 ++---
 .../storage/rocksdb/PartitionDataHelper.java       |   2 +-
 .../storage/rocksdb/RocksDbMvPartitionStorage.java |  19 ++--
 .../storage/rocksdb/RocksDbStorageEngine.java      |   2 +-
 .../storage/rocksdb/RocksDbMvTableStorageTest.java |   6 +-
 .../ignite/distributed/ItTablePersistenceTest.java |   7 +-
 .../distributed/ItTxDistributedTestSingleNode.java |   6 +-
 .../ignite/distributed/ReplicaUnavailableTest.java |   5 +-
 .../ignite/internal/table/ItColocationTest.java    |   4 +-
 .../internal/table/IgniteTablesInternal.java       |   6 +-
 .../ignite/internal/table/InternalTable.java       |   2 +-
 .../apache/ignite/internal/table/TableImpl.java    |   2 +-
 .../table/distributed/StorageUpdateHandler.java    |   4 +-
 .../internal/table/distributed/TableManager.java   | 100 +++++++++++----------
 .../command/TablePartitionIdMessage.java           |   3 +-
 .../table/distributed/index/IndexBuildTaskId.java  |  10 +--
 .../table/distributed/index/IndexBuilder.java      |   8 +-
 .../table/distributed/message/HasDataRequest.java  |   3 +-
 .../distributed/raft/PartitionDataStorage.java     |   4 +-
 .../distributed/raft/snapshot/PartitionAccess.java |   4 +-
 .../raft/snapshot/PartitionAccessImpl.java         |   2 +-
 .../distributed/raft/snapshot/PartitionKey.java    |  11 +--
 .../snapshot/message/SnapshotMvDataResponse.java   |   2 +-
 .../raft/snapshot/outgoing/OutgoingSnapshot.java   |   2 +-
 .../SnapshotAwarePartitionDataStorage.java         |   2 +-
 .../replicator/CompatValidationResult.java         |  19 ++--
 .../replicator/PartitionReplicaListener.java       |  15 ++--
 .../replicator/SchemaCompatValidator.java          |  12 +--
 .../distributed/schema/NonHistoricSchemas.java     |   9 +-
 .../internal/table/distributed/schema/Schemas.java |   7 +-
 .../distributed/storage/InternalTableImpl.java     |   6 +-
 .../internal/table/event/TableEventParameters.java |   7 +-
 .../ignite/internal/utils/RebalanceUtil.java       |   7 +-
 .../RepeatedFinishReadWriteTransactionTest.java    |  12 ++-
 .../AbstractMvStorageUpdateHandlerTest.java        |   2 +-
 .../internal/table/distributed/IndexBaseTest.java  |   4 +-
 .../distributed/StorageUpdateHandlerTest.java      |   4 +-
 .../table/distributed/TableManagerTest.java        |   2 +-
 .../PartitionRaftCommandsSerializationTest.java    |  10 +--
 .../internal/table/distributed/gc/MvGcTest.java    |   6 +-
 .../raft/PartitionCommandListenerTest.java         |  14 +--
 .../raft/snapshot/PartitionAccessImplTest.java     |  14 +--
 .../incoming/IncomingSnapshotCopierTest.java       |  14 +--
 .../outgoing/OutgoingSnapshotCommonTest.java       |   2 +-
 .../OutgoingSnapshotMvDataStreamingTest.java       |   4 +-
 .../outgoing/OutgoingSnapshotReaderTest.java       |   3 +-
 .../OutgoingSnapshotTxDataStreamingTest.java       |   6 +-
 .../outgoing/OutgoingSnapshotsManagerTest.java     |   2 +-
 .../SnapshotAwarePartitionDataStorageTest.java     |   8 +-
 .../PartitionReplicaListenerIndexLockingTest.java  |   2 +-
 .../replication/PartitionReplicaListenerTest.java  |  26 +++---
 .../distributed/storage/InternalTableImplTest.java |   3 +-
 .../distributed/TestPartitionDataStorage.java      |   2 +-
 .../table/impl/DummyInternalTableImpl.java         |   9 +-
 .../ignite/internal/table/impl/DummySchemas.java   |   7 +-
 .../org/apache/ignite/internal/tx/LockKey.java     |   7 +-
 .../apache/ignite/internal/tx/TxManagerTest.java   |   2 +-
 .../storage/state/AbstractTxStateStorageTest.java  |   2 +-
 161 files changed, 762 insertions(+), 643 deletions(-)

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 ac6c45a5a0..d417a5a805 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
@@ -23,7 +23,6 @@ import static org.apache.ignite.lang.ErrorGroups.Client.TABLE_ID_NOT_FOUND_ERR;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Set;
-import java.util.UUID;
 import org.apache.ignite.client.handler.ClientResourceRegistry;
 import org.apache.ignite.internal.binarytuple.BinaryTupleContainer;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
@@ -332,7 +331,7 @@ public class ClientTableCommon {
      *                         </ul>
      */
     public static TableImpl readTable(ClientMessageUnpacker unpacker, IgniteTables tables) {
-        UUID tableId = unpacker.unpackUuid();
+        int tableId = unpacker.unpackInt();
 
         try {
             TableImpl table = ((IgniteTablesInternal) tables).table(tableId);
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableGetRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableGetRequest.java
index 03dcc4fbec..f010263c73 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableGetRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableGetRequest.java
@@ -46,7 +46,7 @@ public class ClientTableGetRequest {
             if (table == null) {
                 out.packNil();
             } else {
-                out.packUuid(((TableImpl) table).tableId());
+                out.packInt(((TableImpl) table).tableId());
             }
         });
     }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablePartitionAssignmentGetRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablePartitionAssignmentGetRequest.java
index d40a74a650..a47d59ab67 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablePartitionAssignmentGetRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablePartitionAssignmentGetRequest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.client.handler.requests.table;
 
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.client.proto.ClientMessagePacker;
 import org.apache.ignite.internal.client.proto.ClientMessageUnpacker;
@@ -43,7 +42,7 @@ public class ClientTablePartitionAssignmentGetRequest {
             ClientMessagePacker out,
             IgniteTablesInternal tables
     ) throws NodeStoppingException {
-        UUID tableId = in.unpackUuid();
+        int tableId = in.unpackInt();
         var assignment = tables.assignments(tableId);
 
         if (assignment == null) {
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablesGetRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablesGetRequest.java
index 9e0d7ddb4b..cf478a09ef 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablesGetRequest.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTablesGetRequest.java
@@ -43,7 +43,7 @@ public class ClientTablesGetRequest {
             for (var table : tables) {
                 var tableImpl = (TableImpl) table;
 
-                out.packUuid(tableImpl.tableId());
+                out.packInt(tableImpl.tableId());
                 out.packString(table.name());
             }
         });
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java
index 481e0818bf..305d209cde 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/compute/ClientCompute.java
@@ -210,7 +210,7 @@ public class ClientCompute implements IgniteCompute {
                 (schema, outputChannel) -> {
                     ClientMessagePacker w = outputChannel.out();
 
-                    w.packUuid(t.tableId());
+                    w.packInt(t.tableId());
                     w.packInt(schema.version());
 
                     ClientRecordSerializer.writeRecRaw(key, keyMapper, schema, w, TuplePart.KEY);
@@ -232,7 +232,7 @@ public class ClientCompute implements IgniteCompute {
                 (schema, outputChannel) -> {
                     ClientMessagePacker w = outputChannel.out();
 
-                    w.packUuid(t.tableId());
+                    w.packInt(t.tableId());
                     w.packInt(schema.version());
 
                     ClientTupleSerializer.writeTupleRaw(key, schema, outputChannel, true);
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 4751ea6237..27845afd4d 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
@@ -445,7 +445,7 @@ public class ClientKeyValueView<K, V> implements KeyValueView<K, V> {
     }
 
     private void writeSchemaAndTx(ClientSchema s, PayloadOutputChannel w, @Nullable Transaction tx) {
-        w.out().packUuid(tbl.tableId());
+        w.out().packInt(tbl.tableId());
         writeTx(tx, w);
         w.out().packInt(s.version());
     }
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 dd5703410d..ba0901f571 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
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.UUID;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
 import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
 import org.apache.ignite.internal.client.PayloadOutputChannel;
@@ -47,7 +46,7 @@ import org.jetbrains.annotations.Nullable;
  */
 public class ClientRecordSerializer<R> {
     /** Table ID. */
-    private final UUID tableId;
+    private final int tableId;
 
     /** Mapper. */
     private final Mapper<R> mapper;
@@ -61,8 +60,7 @@ public class ClientRecordSerializer<R> {
      * @param tableId       Table ID.
      * @param mapper        Mapper.
      */
-    ClientRecordSerializer(UUID tableId, Mapper<R> mapper) {
-        assert tableId != null;
+    ClientRecordSerializer(int tableId, Mapper<R> mapper) {
         assert mapper != null;
 
         this.tableId = tableId;
@@ -122,7 +120,7 @@ public class ClientRecordSerializer<R> {
     }
 
     void writeRec(@Nullable Transaction tx, @Nullable R rec, ClientSchema schema, PayloadOutputChannel out, TuplePart part) {
-        out.out().packUuid(tableId);
+        out.out().packInt(tableId);
         writeTx(tx, out);
         out.out().packInt(schema.version());
 
@@ -137,7 +135,7 @@ public class ClientRecordSerializer<R> {
             PayloadOutputChannel out,
             TuplePart part
     ) {
-        out.out().packUuid(tableId);
+        out.out().packInt(tableId);
         writeTx(tx, out);
         out.out().packInt(schema.version());
 
@@ -155,7 +153,7 @@ public class ClientRecordSerializer<R> {
             PayloadOutputChannel out,
             TuplePart part
     ) {
-        out.out().packUuid(tableId);
+        out.out().packInt(tableId);
         writeTx(tx, out);
         out.out().packInt(schema.version());
         out.out().packInt(recs.size());
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 d6b837b5ce..e1cc6075b5 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
@@ -23,7 +23,6 @@ import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionStage;
 import java.util.concurrent.ConcurrentHashMap;
@@ -55,7 +54,7 @@ import org.jetbrains.annotations.Nullable;
  * Client table API implementation.
  */
 public class ClientTable implements Table {
-    private final UUID id;
+    private final int id;
 
     private final String name;
 
@@ -84,9 +83,8 @@ public class ClientTable implements Table {
      * @param id   Table id.
      * @param name Table name.
      */
-    public ClientTable(ReliableChannel ch, UUID id, String name) {
+    public ClientTable(ReliableChannel ch, int id, String name) {
         assert ch != null;
-        assert id != null;
         assert name != null && !name.isEmpty();
 
         this.ch = ch;
@@ -100,7 +98,7 @@ public class ClientTable implements Table {
      *
      * @return Table id.
      */
-    public UUID tableId() {
+    public int tableId() {
         return id;
     }
 
@@ -159,7 +157,7 @@ public class ClientTable implements Table {
 
     private CompletableFuture<ClientSchema> loadSchema(int ver) {
         return ch.serviceAsync(ClientOp.SCHEMAS_GET, w -> {
-            w.out().packUuid(id);
+            w.out().packInt(id);
 
             if (ver == UNKNOWN_SCHEMA_VERSION) {
                 w.out().packNil();
@@ -394,7 +392,7 @@ public class ClientTable implements Table {
 
             // Load currentVersion or newer.
             partitionAssignment = ch.serviceAsync(ClientOp.PARTITION_ASSIGNMENT_GET,
-                    w -> w.out().packUuid(id),
+                    w -> w.out().packInt(id),
                     r -> {
                         int cnt = r.in().unpackArrayHeader();
                         List<String> res = new ArrayList<>(cnt);
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTables.java b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTables.java
index c994db5c4b..314fc6363b 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTables.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTables.java
@@ -58,7 +58,7 @@ public class ClientTables implements IgniteTables {
             var res = new ArrayList<Table>(cnt);
 
             for (int i = 0; i < cnt; i++) {
-                res.add(new ClientTable(ch, in.unpackUuid(), in.unpackString()));
+                res.add(new ClientTable(ch, in.unpackInt(), in.unpackString()));
             }
 
             return res;
@@ -77,6 +77,6 @@ public class ClientTables implements IgniteTables {
         Objects.requireNonNull(name);
 
         return ch.serviceAsync(ClientOp.TABLE_GET, w -> w.out().packString(name),
-                r -> r.in().tryUnpackNil() ? null : new ClientTable(ch, r.in().unpackUuid(), name));
+                r -> r.in().tryUnpackNil() ? null : new ClientTable(ch, r.in().unpackInt(), name));
     }
 }
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 8b11c524bb..cf9caafaff 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
@@ -53,14 +53,14 @@ import org.jetbrains.annotations.Nullable;
  */
 public class ClientTupleSerializer {
     /** Table ID. */
-    private final UUID tableId;
+    private final int tableId;
 
     /**
      * Constructor.
      *
      * @param tableId Table id.
      */
-    ClientTupleSerializer(UUID tableId) {
+    ClientTupleSerializer(int tableId) {
         this.tableId = tableId;
     }
 
@@ -116,7 +116,7 @@ public class ClientTupleSerializer {
             boolean skipHeader
     ) {
         if (!skipHeader) {
-            out.out().packUuid(tableId);
+            out.out().packInt(tableId);
             writeTx(tx, out);
             out.out().packInt(schema.version());
         }
@@ -167,7 +167,7 @@ public class ClientTupleSerializer {
             boolean skipHeader
     ) {
         if (!skipHeader) {
-            out.out().packUuid(tableId);
+            out.out().packInt(tableId);
             writeTx(tx, out);
             out.out().packInt(schema.version());
         }
@@ -197,7 +197,7 @@ public class ClientTupleSerializer {
      * @param out Out.
      */
     void writeKvTuples(@Nullable Transaction tx, Map<Tuple, Tuple> pairs, ClientSchema schema, PayloadOutputChannel out) {
-        out.out().packUuid(tableId);
+        out.out().packInt(tableId);
         writeTx(tx, out);
         out.out().packInt(schema.version());
         out.out().packInt(pairs.size());
@@ -222,7 +222,7 @@ public class ClientTupleSerializer {
             PayloadOutputChannel out,
             boolean keyOnly
     ) {
-        out.out().packUuid(tableId);
+        out.out().packInt(tableId);
         writeTx(tx, out);
         out.out().packInt(schema.version());
         out.out().packInt(tuples.size());
diff --git a/modules/client/src/test/java/org/apache/ignite/client/PartitionAwarenessTest.java b/modules/client/src/test/java/org/apache/ignite/client/PartitionAwarenessTest.java
index f2d5d1b996..49afafbffa 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/PartitionAwarenessTest.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/PartitionAwarenessTest.java
@@ -24,7 +24,7 @@ import io.netty.util.ResourceLeakDetector;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.client.fakes.FakeIgnite;
@@ -61,6 +61,8 @@ public class PartitionAwarenessTest extends AbstractClientTest {
 
     private String lastOpServerName;
 
+    private final AtomicInteger nextTableId = new AtomicInteger(101);
+
     /**
      * Before all.
      */
@@ -457,7 +459,7 @@ public class PartitionAwarenessTest extends AbstractClientTest {
 
     private Table table(String name) {
         // Create table on both servers with the same ID.
-        var tableId = UUID.randomUUID();
+        int tableId = nextTableId.getAndIncrement();
 
         createTable(server, tableId, name);
         createTable(server2, tableId, name);
@@ -469,7 +471,7 @@ public class PartitionAwarenessTest extends AbstractClientTest {
         return client2.compute();
     }
 
-    private void createTable(Ignite ignite, UUID id, String name) {
+    private void createTable(Ignite ignite, int id, String name) {
         FakeIgniteTables tables = (FakeIgniteTables) ignite.tables();
         TableImpl tableImpl = tables.createTable(name, id);
 
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeIgniteTables.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeIgniteTables.java
index 1bc79600cd..0074306e16 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeIgniteTables.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeIgniteTables.java
@@ -20,10 +20,10 @@ package org.apache.ignite.client.fakes;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import org.apache.ignite.internal.schema.BinaryRow;
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.table.IgniteTablesInternal;
 import org.apache.ignite.internal.table.TableImpl;
 import org.apache.ignite.internal.tx.impl.HeapLockManager;
 import org.apache.ignite.lang.IgniteException;
-import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.table.Table;
 import org.jetbrains.annotations.NotNull;
 
@@ -63,12 +62,14 @@ public class FakeIgniteTables implements IgniteTablesInternal {
 
     private final ConcurrentHashMap<String, TableImpl> tables = new ConcurrentHashMap<>();
 
-    private final ConcurrentHashMap<UUID, TableImpl> tablesById = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<Integer, TableImpl> tablesById = new ConcurrentHashMap<>();
 
     private final CopyOnWriteArrayList<Consumer<IgniteTablesInternal>> assignmentsChangeListeners = new CopyOnWriteArrayList<>();
 
     private volatile List<String> partitionAssignments = null;
 
+    private final AtomicInteger nextTableId = new AtomicInteger(1);
+
     /**
      * Creates a table.
      *
@@ -76,7 +77,7 @@ public class FakeIgniteTables implements IgniteTablesInternal {
      * @return Table.
      */
     public Table createTable(String name) {
-        return createTable(name, UUID.randomUUID());
+        return createTable(name, nextTableId.getAndIncrement());
     }
 
     /**
@@ -86,7 +87,7 @@ public class FakeIgniteTables implements IgniteTablesInternal {
      * @param id Table id.
      * @return Table.
      */
-    public TableImpl createTable(String name, UUID id) {
+    public TableImpl createTable(String name, int id) {
         var newTable = getNewTable(name, id);
 
         var oldTable = tables.putIfAbsent(name, newTable);
@@ -137,7 +138,7 @@ public class FakeIgniteTables implements IgniteTablesInternal {
 
     /** {@inheritDoc} */
     @Override
-    public TableImpl table(UUID id) {
+    public TableImpl table(int id) {
         return tablesById.get(id);
     }
 
@@ -149,7 +150,7 @@ public class FakeIgniteTables implements IgniteTablesInternal {
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<TableImpl> tableAsync(UUID id) {
+    public CompletableFuture<TableImpl> tableAsync(int id) {
         return CompletableFuture.completedFuture(tablesById.get(id));
     }
 
@@ -167,7 +168,7 @@ public class FakeIgniteTables implements IgniteTablesInternal {
 
     /** {@inheritDoc} */
     @Override
-    public List<String> assignments(UUID tableId) throws NodeStoppingException {
+    public List<String> assignments(int tableId) {
         return partitionAssignments;
     }
 
@@ -201,7 +202,7 @@ public class FakeIgniteTables implements IgniteTablesInternal {
     }
 
     @NotNull
-    private TableImpl getNewTable(String name, UUID id) {
+    private TableImpl getNewTable(String name, int id) {
         Function<Integer, SchemaDescriptor> history;
 
         switch (name) {
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
index de872f9454..7e800a1a96 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
@@ -54,7 +54,7 @@ public class FakeInternalTable implements InternalTable {
     private final String tableName;
 
     /** Table ID. */
-    private final UUID tableId;
+    private final int tableId;
 
     private final Function<BinaryRow, BinaryTuple> keyExtractor;
 
@@ -71,7 +71,7 @@ public class FakeInternalTable implements InternalTable {
      * @param tableId Id.
      * @param keyExtractor Function which converts given binary row to an index key.
      */
-    public FakeInternalTable(String tableName, UUID tableId, Function<BinaryRow, BinaryTuple> keyExtractor) {
+    public FakeInternalTable(String tableName, int tableId, Function<BinaryRow, BinaryTuple> keyExtractor) {
         this.tableName = tableName;
         this.tableId = tableId;
         this.keyExtractor = keyExtractor;
@@ -89,9 +89,8 @@ public class FakeInternalTable implements InternalTable {
         return 1;
     }
 
-    /** {@inheritDoc} */
     @Override
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/replicator/TablePartitionId.java b/modules/core/src/main/java/org/apache/ignite/internal/replicator/TablePartitionId.java
index 0ee68108b9..f752d238c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/replicator/TablePartitionId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/replicator/TablePartitionId.java
@@ -17,16 +17,13 @@
 
 package org.apache.ignite.internal.replicator;
 
-import java.util.UUID;
-import org.jetbrains.annotations.NotNull;
-
 /**
  * The class is used to identify a table replication group.
  */
 public class TablePartitionId implements ReplicationGroupId {
 
     /** Table id. */
-    private final UUID tableId;
+    private final int tableId;
 
     /** Partition id. */
     private final int partId;
@@ -37,7 +34,7 @@ public class TablePartitionId implements ReplicationGroupId {
      * @param tableId Table id.
      * @param partId Partition id.
      */
-    public TablePartitionId(@NotNull UUID tableId, int partId) {
+    public TablePartitionId(int tableId, int partId) {
         this.tableId = tableId;
         this.partId = partId;
     }
@@ -51,7 +48,7 @@ public class TablePartitionId implements ReplicationGroupId {
     public static TablePartitionId fromString(String str) {
         String[] parts = str.split("_part_");
 
-        return new TablePartitionId(UUID.fromString(parts[0]), Integer.valueOf(parts[1]));
+        return new TablePartitionId(Integer.parseInt(parts[0]), Integer.parseInt(parts[1]));
     }
 
     /**
@@ -68,7 +65,7 @@ public class TablePartitionId implements ReplicationGroupId {
      *
      * @return Table id.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
@@ -84,12 +81,12 @@ public class TablePartitionId implements ReplicationGroupId {
 
         TablePartitionId that = (TablePartitionId) o;
 
-        return partId == that.partId && tableId.equals(that.tableId);
+        return partId == that.partId && tableId == that.tableId;
     }
 
     @Override
     public int hashCode() {
-        return tableId.hashCode() ^ partId;
+        return tableId ^ partId;
     }
 
     @Override
diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java
index 04c06b93d5..a3315ffa41 100644
--- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java
+++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngine.java
@@ -32,7 +32,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.configuration.NamedConfigurationTree;
@@ -195,7 +194,7 @@ public class DistributionZoneRebalanceEngine {
                                     : ByteUtils.fromBytes(assignmentsBytes);
 
                             for (int part = 0; part < distributionZoneConfiguration.partitions().value(); part++) {
-                                UUID tableId = ((ExtendedTableConfiguration) tableCfg).id().value();
+                                int tableId = tableCfg.id().value();
 
                                 TablePartitionId replicaGrpId = new TablePartitionId(tableId, part);
 
@@ -285,7 +284,7 @@ public class DistributionZoneRebalanceEngine {
                     }
 
                     for (int i = 0; i < partCnt; i++) {
-                        TablePartitionId replicaGrpId = new TablePartitionId(((ExtendedTableConfiguration) tblCfg).id().value(), i);
+                        TablePartitionId replicaGrpId = new TablePartitionId(tblCfg.id().value(), i);
 
                         futs[furCur++] = updatePendingAssignmentsKeys(
                                 tblCfg.name().value(),
diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/TablePartitionId.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/TablePartitionId.java
index adff0dcd52..f0f7098117 100644
--- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/TablePartitionId.java
+++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/TablePartitionId.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.distributionzones.rebalance;
 
-import java.util.UUID;
 import org.apache.ignite.internal.replicator.ReplicationGroupId;
-import org.jetbrains.annotations.NotNull;
 
 // TODO: https://issues.apache.org/jira/browse/IGNITE-19170 Should be refactored to ZonePartitionId.
 /**
@@ -29,7 +27,7 @@ public class TablePartitionId implements ReplicationGroupId {
     private static final long serialVersionUID = -2428659904367844831L;
 
     /** Table id. */
-    private final UUID tableId;
+    private final int tableId;
 
     /** Partition id. */
     private final int partId;
@@ -40,7 +38,7 @@ public class TablePartitionId implements ReplicationGroupId {
      * @param tableId Table id.
      * @param partId Partition id.
      */
-    public TablePartitionId(@NotNull UUID tableId, int partId) {
+    public TablePartitionId(int tableId, int partId) {
         this.tableId = tableId;
         this.partId = partId;
     }
@@ -54,7 +52,7 @@ public class TablePartitionId implements ReplicationGroupId {
     public static TablePartitionId fromString(String str) {
         String[] parts = str.split("_part_");
 
-        return new TablePartitionId(UUID.fromString(parts[0]), Integer.valueOf(parts[1]));
+        return new TablePartitionId(Integer.parseInt(parts[0]), Integer.parseInt(parts[1]));
     }
 
     /**
@@ -71,7 +69,7 @@ public class TablePartitionId implements ReplicationGroupId {
      *
      * @return Table id.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
@@ -87,12 +85,12 @@ public class TablePartitionId implements ReplicationGroupId {
 
         TablePartitionId that = (TablePartitionId) o;
 
-        return partId == that.partId && tableId.equals(that.tableId);
+        return partId == that.partId && tableId == that.tableId;
     }
 
     @Override
     public int hashCode() {
-        return tableId.hashCode() ^ partId;
+        return tableId ^ partId;
     }
 
     @Override
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngineTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngineTest.java
index d51c123669..8be54db3e4 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngineTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/DistributionZoneRebalanceEngineTest.java
@@ -44,7 +44,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
@@ -74,7 +73,7 @@ import org.apache.ignite.internal.raft.Command;
 import org.apache.ignite.internal.raft.WriteCommand;
 import org.apache.ignite.internal.raft.service.CommandClosure;
 import org.apache.ignite.internal.raft.service.RaftGroupService;
-import org.apache.ignite.internal.schema.configuration.ExtendedTableView;
+import org.apache.ignite.internal.schema.configuration.TableView;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.testframework.IgniteAbstractTest;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
@@ -227,6 +226,8 @@ public class DistributionZoneRebalanceEngineTest extends IgniteAbstractTest {
                             + "table5 = { zoneId = 2 }}")
             TablesConfiguration tablesConfiguration
     ) {
+        assignTableIds(tablesConfiguration);
+
         rebalanceEngine = new DistributionZoneRebalanceEngine(
                 new AtomicBoolean(),
                 new IgniteSpinBusyLock(),
@@ -251,10 +252,24 @@ public class DistributionZoneRebalanceEngineTest extends IgniteAbstractTest {
         verify(keyValueStorage, timeout(1000).times(8)).invoke(any(), any());
     }
 
+    private static void assignTableIds(TablesConfiguration tablesConfiguration) {
+        tablesConfiguration.change(tablesChange -> {
+            tablesChange.changeTables(tablesListChange -> {
+                for (int i = 0; i < tablesListChange.size(); i++) {
+                    TableView tableView = tablesListChange.get(i);
+                    int finalI = i;
+                    tablesListChange.update(tableView.name(), tableChange -> tableChange.changeId(finalI + 1));
+                }
+            });
+        }).join();
+    }
+
     @Test
     void sequentialAssignmentsChanging(
             @InjectConfiguration ("mock.tables {table0 = { zoneId = 1 }}") TablesConfiguration tablesConfiguration
     ) {
+        assignTableIds(tablesConfiguration);
+
         rebalanceEngine = new DistributionZoneRebalanceEngine(
                 new AtomicBoolean(),
                 new IgniteSpinBusyLock(),
@@ -294,6 +309,8 @@ public class DistributionZoneRebalanceEngineTest extends IgniteAbstractTest {
     void sequentialEmptyAssignmentsChanging(
             @InjectConfiguration("mock.tables {table0 = { zoneId = 1 }}") TablesConfiguration tablesConfiguration
     ) {
+        assignTableIds(tablesConfiguration);
+
         rebalanceEngine = new DistributionZoneRebalanceEngine(
                 new AtomicBoolean(),
                 new IgniteSpinBusyLock(),
@@ -335,6 +352,8 @@ public class DistributionZoneRebalanceEngineTest extends IgniteAbstractTest {
     void staleDataNodesEvent(
             @InjectConfiguration("mock.tables {table0 = { zoneId = 1 }}") TablesConfiguration tablesConfiguration
     ) {
+        assignTableIds(tablesConfiguration);
+
         rebalanceEngine = new DistributionZoneRebalanceEngine(
                 new AtomicBoolean(),
                 new IgniteSpinBusyLock(),
@@ -364,7 +383,7 @@ public class DistributionZoneRebalanceEngineTest extends IgniteAbstractTest {
 
         checkAssignments(tablesConfiguration, zoneNodes, RebalanceUtil::pendingPartAssignmentsKey);
 
-        TablePartitionId partId = new TablePartitionId(new UUID(0, 0), 0);
+        TablePartitionId partId = new TablePartitionId(1, 0);
 
         assertNull(keyValueStorage.get(RebalanceUtil.plannedPartAssignmentsKey(partId).bytes()).value());
 
@@ -377,9 +396,7 @@ public class DistributionZoneRebalanceEngineTest extends IgniteAbstractTest {
             Function<TablePartitionId, ByteArray> assignmentFunction
     ) {
         tablesConfiguration.tables().value().forEach(tableView -> {
-            ExtendedTableView extendedTableView = (ExtendedTableView) tableView;
-
-            UUID tableId = extendedTableView.id();
+            int tableId = tableView.id();
 
             DistributionZoneConfiguration distributionZoneConfiguration =
                     getZoneById(distributionZonesConfiguration, tableView.zoneId());
diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtilUpdateAssignmentsTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtilUpdateAssignmentsTest.java
index fbc1448ab2..b960d08d0f 100644
--- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtilUpdateAssignmentsTest.java
+++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtilUpdateAssignmentsTest.java
@@ -34,7 +34,6 @@ import java.io.Serializable;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.IntStream;
@@ -487,7 +486,7 @@ public class RebalanceUtilUpdateAssignmentsTest extends IgniteAbstractTest {
             Set<Assignment> expectedPendingAssignments,
             Set<Assignment> expectedPlannedAssignments
     ) {
-        TablePartitionId tablePartitionId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId tablePartitionId = new TablePartitionId(1, 1);
 
         if (currentStableAssignments != null) {
             keyValueStorage.put(RebalanceUtil.stablePartAssignmentsKey(tablePartitionId).bytes(), toBytes(currentStableAssignments),
diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/BuildIndexTaskId.java b/modules/index/src/main/java/org/apache/ignite/internal/index/BuildIndexTaskId.java
index 65efc98710..e6c778370d 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/BuildIndexTaskId.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/BuildIndexTaskId.java
@@ -24,19 +24,19 @@ import org.apache.ignite.internal.tostring.S;
  * ID of the index build task.
  */
 class BuildIndexTaskId {
-    private final UUID tableId;
+    private final int tableId;
 
     private final UUID indexId;
 
     private final int partitionId;
 
-    BuildIndexTaskId(UUID tableId, UUID indexId, int partitionId) {
+    BuildIndexTaskId(int tableId, UUID indexId, int partitionId) {
         this.tableId = tableId;
         this.indexId = indexId;
         this.partitionId = partitionId;
     }
 
-    UUID getTableId() {
+    int getTableId() {
         return tableId;
     }
 
@@ -62,7 +62,7 @@ class BuildIndexTaskId {
         if (partitionId != that.partitionId) {
             return false;
         }
-        if (!tableId.equals(that.tableId)) {
+        if (tableId != that.tableId) {
             return false;
         }
         return indexId.equals(that.indexId);
@@ -70,7 +70,7 @@ class BuildIndexTaskId {
 
     @Override
     public int hashCode() {
-        int result = tableId.hashCode();
+        int result = tableId;
         result = 31 * result + indexId.hashCode();
         result = 31 * result + partitionId;
         return result;
diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java b/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
index 9c0c6b27ec..46c7dfabd1 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
@@ -59,7 +59,7 @@ public class HashIndex implements Index<IndexDescriptor> {
 
     /** {@inheritDoc} */
     @Override
-    public UUID tableId() {
+    public int tableId() {
         return table.tableId();
     }
 
diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java b/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
index 486d3a1743..7d2b441979 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
@@ -40,7 +40,7 @@ public interface Index<DescriptorT extends IndexDescriptor> {
     String name();
 
     /** Returns table id index belong to. */
-    UUID tableId();
+    int tableId();
 
     /** Returns index descriptor. */
     DescriptorT descriptor();
diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
index 8e5e2cb81f..777ec8a111 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
@@ -32,6 +32,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Consumer;
 import java.util.function.Function;
+import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
 import org.apache.ignite.internal.index.event.IndexEvent;
@@ -47,8 +48,8 @@ import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaManager;
 import org.apache.ignite.internal.schema.SchemaRegistry;
-import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
 import org.apache.ignite.internal.schema.configuration.TableConfiguration;
+import org.apache.ignite.internal.schema.configuration.TableView;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.schema.configuration.TablesView;
 import org.apache.ignite.internal.schema.configuration.index.HashIndexChange;
@@ -72,6 +73,7 @@ import org.apache.ignite.lang.IndexNotFoundException;
 import org.apache.ignite.lang.NodeStoppingException;
 import org.apache.ignite.lang.TableNotFoundException;
 import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * An Ignite component that is responsible for handling index-related commands like CREATE or DROP
@@ -216,9 +218,7 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
                     throw new TableNotFoundException(schemaName, tableName);
                 }
 
-                ExtendedTableConfiguration exTableCfg = ((ExtendedTableConfiguration) tableCfg);
-
-                final UUID tableId = exTableCfg.id().value();
+                int tableId = tableCfg.id().value();
 
                 Consumer<TableIndexChange> chg = indexChange.andThen(c -> c.changeTableId(tableId));
 
@@ -330,13 +330,15 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
      */
     public List<TableIndexView> indexConfigurations(String tableName) {
         List<TableIndexView> res = new ArrayList<>();
-        UUID targetTableId = null;
+        Integer targetTableId = null;
+
+        NamedListView<TableView> tablesView = tablesCfg.tables().value();
 
         for (TableIndexView cfg : tablesCfg.indexes().value()) {
             if (targetTableId == null) {
-                TableConfiguration tbl = tablesCfg.tables().get(cfg.tableId());
+                TableView tbl = findTableView(cfg.tableId(), tablesView);
 
-                if (tbl == null || !tableName.equals(tbl.name().value())) {
+                if (tbl == null || !tableName.equals(tbl.name())) {
                     continue;
                 }
 
@@ -351,6 +353,17 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
         return res;
     }
 
+    @Nullable
+    private static TableView findTableView(int tableId, NamedListView<TableView> tablesView) {
+        for (TableView tableView : tablesView) {
+            if (tableView.id() == tableId) {
+                return tableView;
+            }
+        }
+
+        return null;
+    }
+
     private void validateName(String indexName) {
         if (StringUtils.nullOrEmpty(indexName)) {
             throw new IgniteInternalException(
@@ -371,7 +384,7 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
 
         UUID idxId = tableIndexView.id();
 
-        UUID tableId = tableIndexView.tableId();
+        int tableId = tableIndexView.tableId();
 
         long causalityToken = evt.storageRevision();
 
@@ -407,7 +420,7 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
      * @return A future.
      */
     private CompletableFuture<?> onIndexCreate(ConfigurationNotificationEvent<TableIndexView> evt) {
-        UUID tableId = evt.newValue().tableId();
+        int tableId = evt.newValue().tableId();
 
         if (!busyLock.enterBusy()) {
             UUID idxId = evt.newValue().id();
@@ -429,7 +442,7 @@ public class IndexManager extends Producer<IndexEvent, IndexEventParameters> imp
 
     private CompletableFuture<?> createIndexLocally(
             long causalityToken,
-            UUID tableId,
+            int tableId,
             TableIndexView tableIndexView,
             TablesView tablesView
     ) {
diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java b/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
index eab928b3de..59a09b3050 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
@@ -60,7 +60,7 @@ public class SortedIndexImpl implements SortedIndex {
 
     /** {@inheritDoc} */
     @Override
-    public UUID tableId() {
+    public int tableId() {
         return table.tableId();
     }
 
diff --git a/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java b/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
index 0ea95e3270..4319f6c9a0 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
@@ -27,7 +27,7 @@ import org.jetbrains.annotations.Nullable;
  */
 public class IndexEventParameters extends EventParameters {
     /** Table identifier. */
-    private final UUID tableId;
+    private final int tableId;
 
     /** Index identifier. */
     private final UUID indexId;
@@ -42,7 +42,7 @@ public class IndexEventParameters extends EventParameters {
      * @param tableId Table identifier.
      * @param indexId Index identifier.
      */
-    public IndexEventParameters(long revision, UUID tableId, UUID indexId) {
+    public IndexEventParameters(long revision, int tableId, UUID indexId) {
         this(revision, tableId, indexId, null);
     }
 
@@ -54,7 +54,7 @@ public class IndexEventParameters extends EventParameters {
      * @param indexId Index identifier.
      * @param indexDescriptor Index descriptor.
      */
-    public IndexEventParameters(long revision, UUID tableId, UUID indexId, @Nullable IndexDescriptor indexDescriptor) {
+    public IndexEventParameters(long revision, int tableId, UUID indexId, @Nullable IndexDescriptor indexDescriptor) {
         super(revision);
 
         this.tableId = tableId;
@@ -67,7 +67,7 @@ public class IndexEventParameters extends EventParameters {
      *
      * @return An id of the table.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
diff --git a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
index 06564a898c..c24cac3d3e 100644
--- a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
+++ b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
@@ -26,7 +26,7 @@ import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.notNullValue;
 import static org.junit.jupiter.api.Assertions.fail;
-import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -45,7 +45,6 @@ import org.apache.ignite.internal.index.event.IndexEvent;
 import org.apache.ignite.internal.index.event.IndexEventParameters;
 import org.apache.ignite.internal.schema.SchemaManager;
 import org.apache.ignite.internal.schema.configuration.ExtendedTableChange;
-import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.schema.configuration.index.SortedIndexChange;
 import org.apache.ignite.internal.table.InternalTable;
@@ -67,6 +66,7 @@ import org.mockito.Mockito;
 public class IndexManagerTest {
     @InjectConfiguration(
             "mock.tables.tName {"
+                    + "id: 1, "
                     + "columns.c1 {type.type: STRING}, "
                     + "columns.c2 {type.type: STRING}, "
                     + "primaryKey {columns: [c1], colocationColumns: [c1]}"
@@ -80,7 +80,7 @@ public class IndexManagerTest {
     public void setUp() {
         TableManager tableManagerMock = mock(TableManager.class);
 
-        when(tableManagerMock.tableAsync(anyLong(), any(UUID.class))).thenAnswer(inv -> {
+        when(tableManagerMock.tableAsync(anyLong(), anyInt())).thenAnswer(inv -> {
             InternalTable tbl = mock(InternalTable.class);
 
             Mockito.doReturn(inv.getArgument(1)).when(tbl).tableId();
@@ -90,7 +90,7 @@ public class IndexManagerTest {
 
         SchemaManager schManager = mock(SchemaManager.class);
 
-        when(schManager.schemaRegistry(anyLong(), any())).thenReturn(completedFuture(null));
+        when(schManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(null));
 
         indexManager = new IndexManager(tablesConfig, schManager, tableManagerMock);
         indexManager.start();
@@ -139,7 +139,7 @@ public class IndexManagerTest {
                         "name", indexName,
                         "type", "SORTED",
                         "uniq", false,
-                        "tableId", tableId().toString()
+                        "tableId", tableId()
                 )
         );
 
@@ -295,7 +295,7 @@ public class IndexManagerTest {
         }
     }
 
-    private UUID tableId() {
-        return ((ExtendedTableConfiguration) tablesConfig.tables().get("tName")).id().value();
+    private int tableId() {
+        return tablesConfig.tables().get("tName").id().value();
     }
 }
diff --git a/modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/serialization/BaseMethodNameResolver.java b/modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/serialization/BaseMethodNameResolver.java
index b6e373e8f8..188aaa6537 100644
--- a/modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/serialization/BaseMethodNameResolver.java
+++ b/modules/network-annotation-processor/src/main/java/org/apache/ignite/internal/network/processor/serialization/BaseMethodNameResolver.java
@@ -117,6 +117,8 @@ class BaseMethodNameResolver {
 
         if (typeUtils.isSameType(parameterType, String.class)) {
             return "String";
+        } else if (typeUtils.isSameType(parameterType, Integer.class)) {
+            return "BoxedInt";
         } else if (typeUtils.isSameType(parameterType, UUID.class)) {
             return "Uuid";
         } else if (typeUtils.isSameType(parameterType, IgniteUuid.class)) {
diff --git a/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageReader.java b/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageReader.java
index 22a3b55cb3..6887f5ea58 100644
--- a/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageReader.java
+++ b/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageReader.java
@@ -28,6 +28,7 @@ import java.util.UUID;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.network.NetworkMessage;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Stateful message reader.
@@ -95,6 +96,14 @@ public interface MessageReader {
      */
     public int readInt(String name, int dflt);
 
+    /**
+     * Reads an {@code Integer} value.
+     *
+     * @param name Field name.
+     * @return {@code Integer} value.
+     */
+    public @Nullable Integer readBoxedInt(String name);
+
     /**
      * Reads a {@code long} value.
      *
diff --git a/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageWriter.java b/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageWriter.java
index 1c51c73029..776b369f9a 100644
--- a/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageWriter.java
+++ b/modules/network-api/src/main/java/org/apache/ignite/network/serialization/MessageWriter.java
@@ -27,6 +27,7 @@ import java.util.UUID;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.network.NetworkMessage;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Stateful message writer.
@@ -83,6 +84,15 @@ public interface MessageWriter {
      */
     public boolean writeInt(String name, int val);
 
+    /**
+     * Writes an {@code Integer} value.
+     *
+     * @param name Field name.
+     * @param val  {@code Integer} value.
+     * @return Whether a value was fully written.
+     */
+    public boolean writeBoxedInt(String name, @Nullable Integer val);
+
     /**
      * Writes a {@code long} value.
      *
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageReader.java b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageReader.java
index fbfecd1ec0..b42b8c0bc0 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageReader.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageReader.java
@@ -124,6 +124,17 @@ public class DirectMessageReader implements MessageReader {
         return readInt(name);
     }
 
+    @Override
+    public Integer readBoxedInt(String name) {
+        DirectByteBufferStream stream = state.item().stream;
+
+        Integer val = stream.readBoxedInt();
+
+        lastRead = stream.lastFinished();
+
+        return val;
+    }
+
     /** {@inheritDoc} */
     @Override
     public long readLong(String name) {
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageWriter.java b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageWriter.java
index 04fc3a773d..43bf3f17af 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageWriter.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/DirectMessageWriter.java
@@ -116,6 +116,15 @@ public class DirectMessageWriter implements MessageWriter {
         return stream.lastFinished();
     }
 
+    @Override
+    public boolean writeBoxedInt(String name, Integer val) {
+        DirectByteBufferStream stream = state.item().stream;
+
+        stream.writeBoxedInt(val);
+
+        return stream.lastFinished();
+    }
+
     /** {@inheritDoc} */
     @Override
     public boolean writeLong(String name, long val) {
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStream.java b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStream.java
index 0cec79ec82..fa5e0977cf 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStream.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStream.java
@@ -28,6 +28,7 @@ import org.apache.ignite.network.NetworkMessage;
 import org.apache.ignite.network.serialization.MessageReader;
 import org.apache.ignite.network.serialization.MessageWriter;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Direct marshalling I/O stream.
@@ -75,6 +76,13 @@ public interface DirectByteBufferStream {
      */
     void writeInt(int val);
 
+    /**
+     * Writes {@code Integer}.
+     *
+     * @param val Value.
+     */
+    void writeBoxedInt(@Nullable Integer val);
+
     /**
      * Writes {@code long}.
      *
@@ -290,6 +298,13 @@ public interface DirectByteBufferStream {
      */
     int readInt();
 
+    /**
+     * Reads {@code Integer}.
+     *
+     * @return Value.
+     */
+    @Nullable Integer readBoxedInt();
+
     /**
      * Reads {@code long}.
      *
diff --git a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStreamImplV1.java b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStreamImplV1.java
index 4d79a0580d..069450d61f 100644
--- a/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStreamImplV1.java
+++ b/modules/network/src/main/java/org/apache/ignite/internal/network/direct/stream/DirectByteBufferStreamImplV1.java
@@ -252,6 +252,21 @@ public class DirectByteBufferStreamImplV1 implements DirectByteBufferStream {
         }
     }
 
+    @Override
+    public void writeBoxedInt(Integer val) {
+        if (val != null) {
+            lastFinished = buf.remaining() >= 1 + 5;
+
+            if (lastFinished) {
+                writeBoolean(true);
+
+                writeInt(val);
+            }
+        } else {
+            writeBoolean(false);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override
     public void writeLong(long val) {
@@ -909,6 +924,23 @@ public class DirectByteBufferStreamImplV1 implements DirectByteBufferStream {
         return val;
     }
 
+    @Override
+    public Integer readBoxedInt() {
+        int pos = buf.position();
+
+        int limit = buf.limit();
+
+        if (limit - pos >= 5) {
+            if (readBoolean()) {
+                return readInt();
+            } else {
+                return null;
+            }
+        }
+
+        return null;
+    }
+
     /** {@inheritDoc} */
     @Override
     public long readLong() {
diff --git a/modules/network/src/test/resources/org/apache/ignite/internal/network/processor/AllTypesMessage.java b/modules/network/src/test/resources/org/apache/ignite/internal/network/processor/AllTypesMessage.java
index 313003888a..9dcdb1e0d5 100644
--- a/modules/network/src/test/resources/org/apache/ignite/internal/network/processor/AllTypesMessage.java
+++ b/modules/network/src/test/resources/org/apache/ignite/internal/network/processor/AllTypesMessage.java
@@ -37,6 +37,8 @@ public interface AllTypesMessage extends NetworkMessage, Serializable {
 
     int intC();
 
+    Integer integerC2();
+
     long longD();
 
     float floatE();
diff --git a/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/MultiActorPlacementDriverTest.java b/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/MultiActorPlacementDriverTest.java
index 94977e3cff..b4d4be3ede 100644
--- a/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/MultiActorPlacementDriverTest.java
+++ b/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/MultiActorPlacementDriverTest.java
@@ -36,7 +36,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
@@ -119,6 +119,8 @@ public class MultiActorPlacementDriverTest extends IgniteAbstractTest {
     /** This closure handles {@link LeaseGrantedMessage} to check the placement driver manager behavior. */
     private IgniteTriFunction<LeaseGrantedMessage, String, String, LeaseGrantedMessageResponse> leaseGrantHandler;
 
+    private final AtomicInteger nextTableId = new AtomicInteger(1);
+
     @BeforeEach
     public void beforeTest(TestInfo testInfo) throws Exception {
         this.placementDriverNodeNames = IntStream.range(BASE_PORT, BASE_PORT + 3).mapToObj(port -> testNodeName(testInfo, port))
@@ -571,7 +573,7 @@ public class MultiActorPlacementDriverTest extends IgniteAbstractTest {
      * @throws Exception If failed.
      */
     private TablePartitionId createTableAssignment() throws Exception {
-        AtomicReference<UUID> tblIdRef = new AtomicReference<>();
+        int tableId = nextTableId.incrementAndGet();
 
         List<Set<Assignment>> assignments = AffinityUtils.calculateAssignments(nodeNames, 1, nodeNames.size());
 
@@ -580,17 +582,16 @@ public class MultiActorPlacementDriverTest extends IgniteAbstractTest {
         tblsCfg.tables().change(tableViewTableChangeNamedListChange -> {
             tableViewTableChangeNamedListChange.create("test-table", tableChange -> {
                 var extConfCh = ((ExtendedTableChange) tableChange);
+                extConfCh.changeId(tableId);
                 extConfCh.changeZoneId(zoneId);
 
-                tblIdRef.set(extConfCh.id());
-
                 extConfCh.changeAssignments(ByteUtils.toBytes(assignments));
             });
         }).get();
 
-        var grpPart0 = new TablePartitionId(tblIdRef.get(), 0);
+        var grpPart0 = new TablePartitionId(tableId, 0);
 
-        log.info("Fake table created [id={}, repGrp={}]", tblIdRef.get(), grpPart0);
+        log.info("Fake table created [id={}, repGrp={}]", tableId, grpPart0);
 
         return grpPart0;
     }
diff --git a/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/PlacementDriverManagerTest.java b/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/PlacementDriverManagerTest.java
index e66165a488..06e8856121 100644
--- a/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/PlacementDriverManagerTest.java
+++ b/modules/placement-driver/src/integrationTest/java/org/apache/ignite/internal/placementdriver/PlacementDriverManagerTest.java
@@ -38,9 +38,9 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiFunction;
 import org.apache.ignite.internal.affinity.AffinityUtils;
@@ -131,6 +131,8 @@ public class PlacementDriverManagerTest extends IgniteAbstractTest {
     /** This closure handles {@link LeaseGrantedMessage} to check the placement driver manager behavior. */
     private BiFunction<LeaseGrantedMessage, String, LeaseGrantedMessageResponse> leaseGrantHandler;
 
+    private final AtomicInteger nextTableId = new AtomicInteger();
+
     @BeforeEach
     public void beforeTest(TestInfo testInfo) throws NodeStoppingException {
         this.nodeName = testNodeName(testInfo, PORT);
@@ -470,7 +472,7 @@ public class PlacementDriverManagerTest extends IgniteAbstractTest {
      * @throws Exception If failed.
      */
     private TablePartitionId createTableAssignment() throws Exception {
-        AtomicReference<UUID> tblIdRef = new AtomicReference<>();
+        int tableId = nextTableId.incrementAndGet();
 
         List<Set<Assignment>> assignments = AffinityUtils.calculateAssignments(List.of(nodeName, anotherNodeName), 1, 2);
 
@@ -479,17 +481,16 @@ public class PlacementDriverManagerTest extends IgniteAbstractTest {
         tblsCfg.tables().change(tableViewTableChangeNamedListChange -> {
             tableViewTableChangeNamedListChange.create("test-table", tableChange -> {
                 var extConfCh = ((ExtendedTableChange) tableChange);
+                extConfCh.changeId(tableId);
                 extConfCh.changeZoneId(zoneId);
 
-                tblIdRef.set(extConfCh.id());
-
                 extConfCh.changeAssignments(ByteUtils.toBytes(assignments));
             });
         }).get();
 
-        var grpPart0 = new TablePartitionId(tblIdRef.get(), 0);
+        var grpPart0 = new TablePartitionId(tableId, 0);
 
-        log.info("Fake table created [id={}, repGrp={}]", tblIdRef.get(), grpPart0);
+        log.info("Fake table created [id={}, repGrp={}]", tableId, grpPart0);
 
         return grpPart0;
     }
diff --git a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java
index a1f2a43e17..3d4013ee84 100644
--- a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java
+++ b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/AssignmentsTracker.java
@@ -25,7 +25,6 @@ import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import org.apache.ignite.configuration.notifications.ConfigurationListener;
@@ -158,7 +157,7 @@ public class AssignmentsTracker {
             DistributionZoneView distributionZoneView =
                     getZoneById(distributionZonesConfiguration, tblCfg.zoneId()).value();
 
-            UUID tblId = tblCfg.id();
+            int tblId = tblCfg.id();
 
             LOG.debug("Table assignments configuration update for placement driver [revision={}, tblId={}]",
                     assignmentsCtx.storageRevision(), tblId);
diff --git a/modules/placement-driver/src/test/java/org/apache/ignite/internal/placementdriver/PlacementDriverTest.java b/modules/placement-driver/src/test/java/org/apache/ignite/internal/placementdriver/PlacementDriverTest.java
index ecdde0a99c..50497b06d2 100644
--- a/modules/placement-driver/src/test/java/org/apache/ignite/internal/placementdriver/PlacementDriverTest.java
+++ b/modules/placement-driver/src/test/java/org/apache/ignite/internal/placementdriver/PlacementDriverTest.java
@@ -28,7 +28,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
@@ -53,7 +52,7 @@ public class PlacementDriverTest {
 
     private static final ByteArray FAKE_KEY = new ByteArray("foobar");
 
-    private static final TablePartitionId GROUP_1 = new TablePartitionId(UUID.randomUUID(), 0);
+    private static final TablePartitionId GROUP_1 = new TablePartitionId(1000, 0);
 
     private static final ByteArray MS_LEASE_KEY = ByteArray.fromString(PLACEMENTDRIVER_PREFIX + GROUP_1);
 
diff --git a/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp b/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp
index 8a27e62429..dfb69e5579 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp
@@ -36,7 +36,7 @@ namespace ignite::detail {
  * @param id Table ID.
  * @param sch Table schema.
  */
-void write_table_operation_header(protocol::writer &writer, uuid id, transaction_impl *tx, const schema &sch) {
+void write_table_operation_header(protocol::writer &writer, std::int32_t id, transaction_impl *tx, const schema &sch) {
     writer.write(id);
 
     if (!tx)
diff --git a/modules/platforms/cpp/ignite/client/detail/table/table_impl.h b/modules/platforms/cpp/ignite/client/detail/table/table_impl.h
index 1c2a700814..125e9fef92 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/table_impl.h
+++ b/modules/platforms/cpp/ignite/client/detail/table/table_impl.h
@@ -51,7 +51,7 @@ public:
      * @param id ID.
      * @param connection Connection.
      */
-    table_impl(std::string name, const uuid &id, std::shared_ptr<cluster_connection> connection)
+    table_impl(std::string name, const std::int32_t id, std::shared_ptr<cluster_connection> connection)
         : m_name(std::move(name))
         , m_id(id)
         , m_connection(std::move(connection)) {}
@@ -301,7 +301,7 @@ public:
      *
      * @return ID.
      */
-    [[nodiscard]] uuid get_id() const { return m_id; }
+    [[nodiscard]] std::int32_t get_id() const { return m_id; }
 
 private:
     /**
@@ -361,7 +361,7 @@ private:
     const std::string m_name;
 
     /** Table ID. */
-    const uuid m_id;
+    const std::int32_t m_id;
 
     /** Cluster connection. */
     std::shared_ptr<cluster_connection> m_connection;
diff --git a/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp b/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp
index 177982d3c9..386d7f0cb7 100644
--- a/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp
+++ b/modules/platforms/cpp/ignite/client/detail/table/tables_impl.cpp
@@ -29,7 +29,7 @@ void tables_impl::get_table_async(std::string_view name, ignite_callback<std::op
         if (reader.try_read_nil())
             return std::nullopt;
 
-        auto id = reader.read_uuid();
+        auto id = reader.read_int32();
         auto table0 = std::make_shared<table_impl>(std::string(name), id, std::move(conn));
 
         return std::make_optional(table(table0));
@@ -47,8 +47,8 @@ void tables_impl::get_tables_async(ignite_callback<std::vector<table>> callback)
         std::vector<table> tables;
         tables.reserve(reader.read_map_size());
 
-        reader.read_map<uuid, std::string>([conn, &tables](auto &&id, auto &&name) {
-            auto table0 = std::make_shared<table_impl>(std::forward<std::string>(name), std::forward<uuid>(id), conn);
+        reader.read_map<std::int32_t, std::string>([conn, &tables](auto &&id, auto &&name) {
+            auto table0 = std::make_shared<table_impl>(std::forward<std::string>(name), std::forward<std::int32_t>(id), conn);
             tables.push_back(table{table0});
         });
 
diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
index cca87ea907..b621f433a1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Tests/FakeServer.cs
@@ -48,11 +48,11 @@ namespace Apache.Ignite.Tests
 
         public const string CustomColocationKeyTableName = "tbl3";
 
-        private static readonly Guid ExistingTableId = Guid.NewGuid();
+        private const int ExistingTableId = 1001;
 
-        private static readonly Guid CompositeKeyTableId = Guid.NewGuid();
+        private const int CompositeKeyTableId = 1002;
 
-        private static readonly Guid CustomColocationKeyTableId = Guid.NewGuid();
+        private const int CustomColocationKeyTableId = 1003;
 
         private readonly Socket _listener;
 
@@ -350,7 +350,7 @@ namespace Apache.Ignite.Tests
 
         private void GetSchemas(MsgPackReader reader, Socket handler, long requestId)
         {
-            var tableId = reader.ReadGuid();
+            var tableId = reader.ReadInt32();
 
             using var arrayBufferWriter = new PooledArrayBuffer();
             var writer = new MsgPackWriter(arrayBufferWriter);
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs
index 6a310e7eb3..35d0eaf926 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Table.cs
@@ -79,7 +79,7 @@ namespace Apache.Ignite.Internal.Table
         /// <param name="id">Table id.</param>
         /// <param name="socket">Socket.</param>
         /// <param name="sql">SQL.</param>
-        public Table(string name, Guid id, ClientFailoverSocket socket, Sql sql)
+        public Table(string name, int id, ClientFailoverSocket socket, Sql sql)
         {
             _socket = socket;
             _sql = sql;
@@ -121,7 +121,7 @@ namespace Apache.Ignite.Internal.Table
         /// <summary>
         /// Gets the table id.
         /// </summary>
-        internal Guid Id { get; }
+        internal int Id { get; }
 
         /// <inheritdoc/>
         public IRecordView<T> GetRecordView<T>()
diff --git a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Tables.cs b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Tables.cs
index 9ab5dd90f8..aa61d0fb2e 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Tables.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Internal/Table/Tables.cs
@@ -39,7 +39,7 @@ namespace Apache.Ignite.Internal.Table
         private readonly Sql _sql;
 
         /** Cached tables. Caching here is required to retain schema and serializer caches in <see cref="Table"/>. */
-        private readonly ConcurrentDictionary<Guid, Table> _cachedTables = new();
+        private readonly ConcurrentDictionary<int, Table> _cachedTables = new();
 
         /// <summary>
         /// Initializes a new instance of the <see cref="Tables"/> class.
@@ -72,12 +72,12 @@ namespace Apache.Ignite.Internal.Table
 
                 for (var i = 0; i < len; i++)
                 {
-                    var id = r.ReadGuid();
+                    var id = r.ReadInt32();
                     var name = r.ReadString();
 
                     var table = _cachedTables.GetOrAdd(
                         id,
-                        static (Guid id0, (string Name, Tables Tables) arg) =>
+                        static (int id0, (string Name, Tables Tables) arg) =>
                             new Table(arg.Name, id0, arg.Tables._socket, arg.Tables._sql),
                         (name, this));
 
@@ -114,8 +114,8 @@ namespace Apache.Ignite.Internal.Table
                 r.TryReadNil()
                     ? null
                     : _cachedTables.GetOrAdd(
-                        r.ReadGuid(),
-                        (Guid id, (string Name, Tables Tables) arg) => new Table(arg.Name, id, arg.Tables._socket, arg.Tables._sql),
+                        r.ReadInt32(),
+                        (int id, (string Name, Tables Tables) arg) => new Table(arg.Name, id, arg.Tables._socket, arg.Tables._sql),
                         (name, this));
         }
     }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
index 9857bc59cc..0b1c87dae1 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/configuration/storage/ItRebalanceDistributedTest.java
@@ -53,7 +53,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
@@ -897,7 +896,7 @@ public class ItRebalanceDistributedTest {
             }
 
             int partitionId = extractPartitionNumber(stableAssignmentsWatchEvent.key());
-            UUID tableId = extractTableId(stableAssignmentsWatchEvent.key(), STABLE_ASSIGNMENTS_PREFIX);
+            int tableId = extractTableId(stableAssignmentsWatchEvent.key(), STABLE_ASSIGNMENTS_PREFIX);
 
             return new TablePartitionId(tableId, partitionId);
         }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java
index 29c01469a4..9c7f7d4115 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/inmemory/ItRaftStorageVolatilityTest.java
@@ -37,7 +37,6 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.UUID;
 import java.util.stream.Stream;
 import org.apache.ignite.internal.ClusterPerTestIntegrationTest;
 import org.apache.ignite.internal.app.IgniteImpl;
@@ -111,7 +110,7 @@ class ItRaftStorageVolatilityTest extends ClusterPerTestIntegrationTest {
         return testTableId(ignite) + "_part_";
     }
 
-    private UUID testTableId(IgniteImpl ignite) {
+    private int testTableId(IgniteImpl ignite) {
         TableManager tables = (TableManager) ignite.tables();
         return tables.tableImpl(TABLE_NAME).tableId();
     }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java
index 415fba0d38..68c5d4bd5e 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteInMemoryNodeRestartTest.java
@@ -29,7 +29,6 @@ import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.function.IntFunction;
@@ -197,7 +196,6 @@ public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
         createTableWithData(ignite, TABLE_NAME, 3, 1);
 
         TableImpl table = (TableImpl) ignite.tables().table(TABLE_NAME);
-        UUID tableId = table.tableId();
 
         // Find the leader of the table's partition group.
         RaftGroupService raftGroupService = table.internalTable().partitionRaftGroupService(0);
@@ -226,7 +224,7 @@ public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
                 () -> {
                     boolean raftNodeStarted = loza.localNodes().stream().anyMatch(nodeId -> {
                         if (nodeId.groupId() instanceof TablePartitionId) {
-                            return ((TablePartitionId) nodeId.groupId()).tableId().equals(tableId);
+                            return ((TablePartitionId) nodeId.groupId()).tableId() == table.tableId();
                         }
 
                         return false;
@@ -264,7 +262,6 @@ public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
         createTableWithData(ignite0, TABLE_NAME, 3, 1);
 
         TableImpl table = (TableImpl) ignite0.tables().table(TABLE_NAME);
-        UUID tableId = table.tableId();
 
         // Lose the majority.
         stopNode(1);
@@ -278,7 +275,7 @@ public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
         assertTrue(IgniteTestUtils.waitForCondition(
                 () -> loza.localNodes().stream().anyMatch(nodeId -> {
                     if (nodeId.groupId() instanceof TablePartitionId) {
-                        return ((TablePartitionId) nodeId.groupId()).tableId().equals(tableId);
+                        return ((TablePartitionId) nodeId.groupId()).tableId() == table.tableId();
                     }
 
                     return true;
@@ -304,7 +301,6 @@ public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
         createTableWithData(ignite0, TABLE_NAME, 3, 1);
 
         TableImpl table = (TableImpl) ignite0.tables().table(TABLE_NAME);
-        UUID tableId = table.tableId();
 
         stopNode(0);
         stopNode(1);
@@ -321,7 +317,7 @@ public class ItIgniteInMemoryNodeRestartTest extends IgniteAbstractTest {
             assertTrue(IgniteTestUtils.waitForCondition(
                     () -> loza.localNodes().stream().anyMatch(nodeId -> {
                         if (nodeId.groupId() instanceof TablePartitionId) {
-                            return ((TablePartitionId) nodeId.groupId()).tableId().equals(tableId);
+                            return ((TablePartitionId) nodeId.groupId()).tableId() == table.tableId();
                         }
 
                         return true;
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
index a4dd20c902..a3eac695ef 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItTablesApiTest.java
@@ -36,7 +36,6 @@ import static org.junit.jupiter.api.Assertions.fail;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
@@ -411,7 +410,7 @@ public class ItTablesApiTest extends IgniteAbstractTest {
 
         Table table = createTable(clusterNodes.get(0), TABLE_NAME);
 
-        UUID tblId = ((TableImpl) table).tableId();
+        int tblId = ((TableImpl) table).tableId();
 
         CompletableFuture<Table> tableByNameFut = supplyAsync(() -> ignite1.tables().table(TABLE_NAME));
 
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
index 24c3502153..4ddf21cad0 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
@@ -1075,7 +1075,7 @@ public class ItSecondaryIndexTest extends ClusterPerClassIntegrationTest {
         }
 
         @Override
-        public UUID tableId() {
+        public int tableId() {
             return delegate.tableId();
         }
 
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java
index 5554f229a3..1babd206a6 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java
@@ -28,7 +28,6 @@ import static org.apache.ignite.lang.ErrorGroups.Common.NODE_STOPPING_ERR;
 import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Flow.Subscriber;
@@ -47,8 +46,8 @@ import org.apache.ignite.internal.manager.Producer;
 import org.apache.ignite.internal.metastorage.Entry;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
 import org.apache.ignite.internal.schema.configuration.ColumnView;
-import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
 import org.apache.ignite.internal.schema.configuration.ExtendedTableView;
+import org.apache.ignite.internal.schema.configuration.TableConfiguration;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.schema.event.SchemaEvent;
 import org.apache.ignite.internal.schema.event.SchemaEventParameters;
@@ -84,7 +83,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
     private final TablesConfiguration tablesCfg;
 
     /** Versioned store for tables by name. */
-    private final IncrementalVersionedValue<Map<UUID, SchemaRegistryImpl>> registriesVv;
+    private final IncrementalVersionedValue<Map<Integer, SchemaRegistryImpl>> registriesVv;
 
     /** Meta storage manager. */
     private final MetaStorageManager metastorageMgr;
@@ -122,7 +121,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
 
             int newSchemaVersion = tblCfg.schemaId();
 
-            UUID tblId = tblCfg.id();
+            int tblId = tblCfg.id();
 
             if (searchSchemaByVersion(tblId, newSchemaVersion) != null) {
                 return completedFuture(null);
@@ -169,7 +168,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
         }
     }
 
-    private void setColumnMapping(SchemaDescriptor schema, UUID tableId) throws ExecutionException, InterruptedException {
+    private void setColumnMapping(SchemaDescriptor schema, int tableId) throws ExecutionException, InterruptedException {
         if (schema.version() == INITIAL_SCHEMA_VERSION) {
             return;
         }
@@ -197,9 +196,9 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @return Future that, when complete, will resolve into an updated map of schema registries
      *     (to be used in {@link IncrementalVersionedValue#update}).
      */
-    private CompletableFuture<Map<UUID, SchemaRegistryImpl>> registerSchema(
-            Map<UUID, SchemaRegistryImpl> registries,
-            UUID tableId,
+    private CompletableFuture<Map<Integer, SchemaRegistryImpl>> registerSchema(
+            Map<Integer, SchemaRegistryImpl> registries,
+            int tableId,
             String tableName,
             SchemaDescriptor schema
     ) {
@@ -212,7 +211,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
                     SchemaRegistryImpl reg = registries.get(tableId);
 
                     if (reg == null) {
-                        Map<UUID, SchemaRegistryImpl> copy = new HashMap<>(registries);
+                        Map<Integer, SchemaRegistryImpl> copy = new HashMap<>(registries);
 
                         copy.put(tableId, createSchemaRegistry(tableId, tableName, schema));
 
@@ -233,7 +232,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param initialSchema Initial schema for the registry.
      * @return Schema registry.
      */
-    private SchemaRegistryImpl createSchemaRegistry(UUID tableId, String tableName, SchemaDescriptor initialSchema) {
+    private SchemaRegistryImpl createSchemaRegistry(int tableId, String tableName, SchemaDescriptor initialSchema) {
         return new SchemaRegistryImpl(
                 ver -> inBusyLock(busyLock, () -> tableSchema(tableId, tableName, ver)),
                 () -> inBusyLock(busyLock, () -> latestSchemaVersion(tableId)),
@@ -248,8 +247,8 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param schemaVer Schema version.
      * @return Schema descriptor.
      */
-    private CompletableFuture<SchemaDescriptor> tableSchema(UUID tblId, String tableName, int schemaVer) {
-        ExtendedTableConfiguration tblCfg = ((ExtendedTableConfiguration) tablesCfg.tables().get(tableName));
+    private CompletableFuture<SchemaDescriptor> tableSchema(int tblId, String tableName, int schemaVer) {
+        TableConfiguration tblCfg = tablesCfg.tables().get(tableName);
 
         CompletableFuture<SchemaDescriptor> fut = new CompletableFuture<>();
 
@@ -259,7 +258,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
             return getSchemaDescriptor(schemaVer, tblCfg);
         }
 
-        CompletionListener<Map<UUID, SchemaRegistryImpl>> schemaListener = (token, regs, e) -> {
+        CompletionListener<Map<Integer, SchemaRegistryImpl>> schemaListener = (token, regs, e) -> {
             if (schemaVer <= regs.get(tblId).lastSchemaVersion()) {
                 SchemaRegistry registry0 = registriesVv.latest().get(tblId);
 
@@ -300,7 +299,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param schemaVer Schema version for the table.
      * @return True, if the schema version is less or equal than the latest version from the schema registry, false otherwise.
      */
-    private boolean checkSchemaVersion(UUID tblId, int schemaVer) {
+    private boolean checkSchemaVersion(int tblId, int schemaVer) {
         SchemaRegistry registry = registriesVv.latest().get(tblId);
 
         assert registry != null : IgniteStringFormatter.format("Registry for the table not found [tblId={}]", tblId);
@@ -315,7 +314,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param schemaVer Schema version.
      * @return Descriptor if required schema found, or {@code null} otherwise.
      */
-    private @Nullable SchemaDescriptor searchSchemaByVersion(UUID tblId, int schemaVer) {
+    private @Nullable SchemaDescriptor searchSchemaByVersion(int tblId, int schemaVer) {
         SchemaRegistry registry = registriesVv.latest().get(tblId);
 
         if (registry != null && schemaVer <= registry.lastSchemaVersion()) {
@@ -332,9 +331,8 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param tblCfg Table configuration.
      * @return Schema descriptor.
      */
-    private CompletableFuture<SchemaDescriptor> getSchemaDescriptor(int schemaVer, ExtendedTableConfiguration tblCfg) {
-        CompletableFuture<Entry> ent = metastorageMgr.get(
-                schemaWithVerHistKey(tblCfg.id().value(), schemaVer));
+    private CompletableFuture<SchemaDescriptor> getSchemaDescriptor(int schemaVer, TableConfiguration tblCfg) {
+        CompletableFuture<Entry> ent = metastorageMgr.get(schemaWithVerHistKey(tblCfg.id().value(), schemaVer));
 
         return ent.thenApply(e -> SchemaSerializerImpl.INSTANCE.deserialize(e.value()));
     }
@@ -348,14 +346,14 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      *     token.
      * @return A future which will be completed when schema registries for given causality token are ready.
      */
-    public CompletableFuture<SchemaRegistry> schemaRegistry(long causalityToken, @Nullable UUID tableId) {
+    public CompletableFuture<SchemaRegistry> schemaRegistry(long causalityToken, int tableId) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(NODE_STOPPING_ERR, new NodeStoppingException());
         }
 
         try {
             return registriesVv.get(causalityToken)
-                    .thenApply(regs -> inBusyLock(busyLock, () -> tableId == null ? null : regs.get(tableId)));
+                    .thenApply(regs -> inBusyLock(busyLock, () -> regs.get(tableId)));
         } finally {
             busyLock.leaveBusy();
         }
@@ -367,7 +365,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param tableId Table id.
      * @return Schema registry.
      */
-    public SchemaRegistry schemaRegistry(UUID tableId) {
+    public SchemaRegistry schemaRegistry(int tableId) {
         return registriesVv.latest().get(tableId);
     }
 
@@ -377,14 +375,14 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param causalityToken Causality token.
      * @param tableId Table id.
      */
-    public CompletableFuture<?> dropRegistry(long causalityToken, UUID tableId) {
+    public CompletableFuture<?> dropRegistry(long causalityToken, int tableId) {
         return registriesVv.update(causalityToken, (registries, e) -> inBusyLock(busyLock, () -> {
             if (e != null) {
                 return failedFuture(new IgniteInternalException(
                         IgniteStringFormatter.format("Cannot remove a schema registry for the table [tblId={}]", tableId), e));
             }
 
-            Map<UUID, SchemaRegistryImpl> regs = new HashMap<>(registries);
+            Map<Integer, SchemaRegistryImpl> regs = new HashMap<>(registries);
 
             regs.remove(tableId);
 
@@ -408,7 +406,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param tblId Table id.
      * @return The latest schema version.
      */
-    private CompletableFuture<Integer> latestSchemaVersion(UUID tblId) {
+    private CompletableFuture<Integer> latestSchemaVersion(int tblId) {
         var latestVersionFuture = new CompletableFuture<Integer>();
 
         metastorageMgr.prefix(schemaHistPrefix(tblId)).subscribe(new Subscriber<>() {
@@ -450,7 +448,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param tblId Table id.
      * @return Schema representation if schema found, {@code null} otherwise.
      */
-    private CompletableFuture<SchemaDescriptor> schemaByVersion(UUID tblId, int ver) {
+    private CompletableFuture<SchemaDescriptor> schemaByVersion(int tblId, int ver) {
         return metastorageMgr.get(schemaWithVerHistKey(tblId, ver))
                 .thenApply(entry -> {
                     byte[] value = entry.value();
@@ -476,7 +474,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param ver Schema version.
      * @return {@link ByteArray} representation.
      */
-    private static ByteArray schemaWithVerHistKey(UUID tblId, int ver) {
+    private static ByteArray schemaWithVerHistKey(int tblId, int ver) {
         return ByteArray.fromString(tblId + SCHEMA_STORE_PREFIX + ver);
     }
 
@@ -486,7 +484,7 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
      * @param tblId Table id.
      * @return {@link ByteArray} representation.
      */
-    private static ByteArray schemaHistPrefix(UUID tblId) {
+    private static ByteArray schemaHistPrefix(int tblId) {
         return ByteArray.fromString(tblId + SCHEMA_STORE_PREFIX);
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ExtendedTableConfigurationSchema.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ExtendedTableConfigurationSchema.java
index a7780ca66e..3a2f408d86 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ExtendedTableConfigurationSchema.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/ExtendedTableConfigurationSchema.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.schema.configuration;
 
-import java.util.UUID;
 import org.apache.ignite.configuration.annotation.InternalConfiguration;
-import org.apache.ignite.configuration.annotation.InternalId;
 import org.apache.ignite.configuration.annotation.Value;
 
 /**
@@ -28,10 +26,6 @@ import org.apache.ignite.configuration.annotation.Value;
 @InternalConfiguration
 // TODO: IGNITE-15480 Add id's to columns in order to properly process column renaming withing index context.
 public class ExtendedTableConfigurationSchema extends TableConfigurationSchema {
-    /** Table id. */
-    @InternalId
-    public UUID id;
-
     /**
      * Serialized version of an affinity assignments. Currently, configuration doesn't support neither collections nor array of arrays, so
      * that serialization was chosen.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/TableConfigurationSchema.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/TableConfigurationSchema.java
index d599c771ef..548fd79a60 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/TableConfigurationSchema.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/TableConfigurationSchema.java
@@ -22,7 +22,6 @@ import org.apache.ignite.configuration.annotation.ConfigValue;
 import org.apache.ignite.configuration.annotation.InjectedName;
 import org.apache.ignite.configuration.annotation.NamedConfigValue;
 import org.apache.ignite.configuration.annotation.Value;
-import org.apache.ignite.configuration.validation.Immutable;
 import org.apache.ignite.configuration.validation.Range;
 
 /**
@@ -30,16 +29,15 @@ import org.apache.ignite.configuration.validation.Range;
  */
 @Config
 public class TableConfigurationSchema {
+    /** Table id. */
+    @Value(hasDefault = true)
+    @Range(min = 1)
+    public int id = 1;
+
     /** Table name. */
     @InjectedName
     public String name;
 
-    /** Integer table id. */
-    @Immutable
-    @Range(min = 1)
-    @Value(hasDefault = true)
-    public int tableId = 1;
-
     /** Distribution zone id. */
     @Value(hasDefault = true)
     public int zoneId = 0;
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImpl.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImpl.java
index 8259cbef39..9a72f3da29 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImpl.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImpl.java
@@ -23,7 +23,6 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.validation.ValidationContext;
 import org.apache.ignite.configuration.validation.ValidationIssue;
@@ -31,6 +30,7 @@ import org.apache.ignite.configuration.validation.Validator;
 import org.apache.ignite.internal.schema.configuration.TableView;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.schema.configuration.TablesView;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Index configuration validator implementation.
@@ -58,9 +58,7 @@ public class IndexValidatorImpl implements Validator<IndexValidator, NamedListVi
 
             TableIndexView idxView = indexView.get(key);
 
-            UUID tableId = idxView.tableId();
-
-            TableView tableView = tablesView.get(tableId);
+            TableView tableView = findTableById(idxView.tableId(), tablesView);
 
             if (tableView == null) {
                 ctx.addIssue(new ValidationIssue(key, "Unable to create index [name=" + key + "]. Table not found."));
@@ -127,6 +125,16 @@ public class IndexValidatorImpl implements Validator<IndexValidator, NamedListVi
         }
     }
 
+    private static @Nullable TableView findTableById(int tableId, NamedListView<? extends TableView> tablesView) {
+        for (TableView table : tablesView) {
+            if (table.id() == tableId) {
+                return table;
+            }
+        }
+
+        return null;
+    }
+
     private List<String> newKeys(NamedListView<?> before, NamedListView<?> after) {
         List<String> result = new ArrayList<>(after.namedListKeys());
 
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
index 78142925bb..6cbd0412b6 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
@@ -47,9 +47,9 @@ public class TableIndexConfigurationSchema {
     @InjectedName
     public String name;
 
-    /** Table id the index associated with. */
+    /** ID of the table the index is associated with. */
     @Value
-    public UUID tableId;
+    public int tableId;
 
     /** Has default value flag. */
     @Value(hasDefault = true)
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/event/SchemaEventParameters.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/event/SchemaEventParameters.java
index b58f680c5b..635109479c 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/event/SchemaEventParameters.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/event/SchemaEventParameters.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.schema.event;
 
-import java.util.UUID;
 import org.apache.ignite.internal.manager.EventParameters;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 
@@ -26,7 +25,7 @@ import org.apache.ignite.internal.schema.SchemaDescriptor;
  */
 public class SchemaEventParameters extends EventParameters {
     /** Table identifier. */
-    private final UUID tableId;
+    private final int tableId;
 
     /** Schema descriptor. */
     private final SchemaDescriptor schemaDescriptor;
@@ -38,7 +37,7 @@ public class SchemaEventParameters extends EventParameters {
      * @param tableId Table id.
      * @param schemaDescriptor Schema descriptor.
      */
-    public SchemaEventParameters(long causalityToken, UUID tableId, SchemaDescriptor schemaDescriptor) {
+    public SchemaEventParameters(long causalityToken, int tableId, SchemaDescriptor schemaDescriptor) {
         super(causalityToken);
 
         this.tableId = tableId;
@@ -50,7 +49,7 @@ public class SchemaEventParameters extends EventParameters {
      *
      * @return Table id.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImplTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImplTest.java
index 5845335cb3..731b4511cd 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImplTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/index/IndexValidatorImplTest.java
@@ -19,19 +19,19 @@ package org.apache.ignite.internal.schema.configuration.index;
 
 import static org.apache.ignite.internal.configuration.validation.TestValidationUtil.validate;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
+import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import java.util.UUID;
 import java.util.function.Consumer;
 import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.validation.ValidationContext;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
-import org.apache.ignite.internal.schema.configuration.ExtendedTableConfiguration;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
@@ -40,13 +40,20 @@ import org.junit.jupiter.api.extension.ExtendWith;
  */
 @ExtendWith(ConfigurationExtension.class)
 public class IndexValidatorImplTest {
+    private static final int TABLE_ID = 1;
+
     @InjectConfiguration("mock.tables.fooTable {columns.column0 {type.type: STRING}}")
     private TablesConfiguration tablesConfig;
 
+    @BeforeEach
+    void setupTableConfig() {
+        assertThat(tablesConfig.tables().get("fooTable").id().update(TABLE_ID), willSucceedFast());
+    }
+
     @Test
     void testMissingTable() {
-        createIndex("fooIndex", indexChange -> indexChange.convert(HashIndexChange.class));
-        createIndex("barIndex", indexChange -> indexChange.convert(SortedIndexChange.class));
+        createIndex("fooIndex", indexChange -> indexChange.changeTableId(999).convert(HashIndexChange.class));
+        createIndex("barIndex", indexChange -> indexChange.changeTableId(999).convert(SortedIndexChange.class));
 
         validate0(
                 "Unable to create index [name=fooIndex]. Table not found.",
@@ -124,8 +131,8 @@ public class IndexValidatorImplTest {
         validate(IndexValidatorImpl.INSTANCE, mock(IndexValidator.class), validationContext, errorMessagePrefixes);
     }
 
-    private UUID tableId() {
-        return ((ExtendedTableConfiguration) tablesConfig.tables().get("fooTable")).id().value();
+    private int tableId() {
+        return tablesConfig.tables().get("fooTable").id().value();
     }
 
     private void createIndex(String indexName, Consumer<TableIndexChange> indexChange) {
diff --git a/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java b/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
index 22717d4f66..f7ec77867f 100644
--- a/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
+++ b/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
@@ -488,7 +488,7 @@ public class SchemaConfigurationConverter {
      * @param change Indexes change to fulfill.
      * @return TableChange to get result from.
      */
-    public static TableIndexChange addIndex(IndexDefinition idx, UUID tableId, TableIndexChange change) {
+    public static TableIndexChange addIndex(IndexDefinition idx, int tableId, TableIndexChange change) {
         return convert(idx, change).changeTableId(tableId);
     }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java
index fafa31af4f..ac013c61c3 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java
@@ -773,7 +773,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
 
                 @Override
                 public IgniteRel visit(IgniteTableModify rel) {
-                    UUID tableId = rel.getTable().unwrap(IgniteTable.class).id();
+                    int tableId = rel.getTable().unwrap(IgniteTable.class).id();
                     List<NodeWithTerm> assignments = fragment.mapping().updatingTableAssignments();
 
                     enlist(tableId, assignments);
@@ -781,7 +781,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
                     return super.visit(rel);
                 }
 
-                private void enlist(UUID tableId, List<NodeWithTerm> assignments) {
+                private void enlist(int tableId, List<NodeWithTerm> assignments) {
                     if (assignments.isEmpty()) {
                         return;
                     }
@@ -799,7 +799,7 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
                 }
 
                 private void enlist(SourceAwareIgniteRel rel) {
-                    UUID tableId = rel.getTable().unwrap(IgniteTable.class).id();
+                    int tableId = rel.getTable().unwrap(IgniteTable.class).id();
                     List<NodeWithTerm> assignments = fragment.mapping().findGroup(rel.sourceId()).assignments().stream()
                             .map(l -> l.get(0))
                             .collect(Collectors.toList());
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/externalize/RelJson.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/externalize/RelJson.java
index d02bb72eb6..2b85eb3b03 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/externalize/RelJson.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/externalize/RelJson.java
@@ -37,7 +37,6 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import org.apache.calcite.avatica.AvaticaUtils;
@@ -540,7 +539,7 @@ class RelJson {
 
                 if (function.affinity()) {
                     map.put("zoneId", ((AffinityDistribution) function).zoneId());
-                    map.put("tableId", ((AffinityDistribution) function).tableId().toString());
+                    map.put("tableId", Integer.toString(((AffinityDistribution) function).tableId()));
                 }
 
                 return map;
@@ -676,7 +675,7 @@ class RelJson {
         String tableIdStr = (String) map.get("tableId");
 
         if (tableIdStr != null) {
-            UUID tableId = UUID.fromString(tableIdStr);
+            int tableId = Integer.parseInt(tableIdStr);
             Object zoneId = map.get("zoneId");
 
             return IgniteDistributions.affinity((List<Integer>) map.get("keys"), tableId, zoneId);
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/IgniteTableModify.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/IgniteTableModify.java
index e4b3fc3bf6..1561847b7b 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/IgniteTableModify.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/IgniteTableModify.java
@@ -116,7 +116,7 @@ public class IgniteTableModify extends TableModify implements IgniteRel {
     public RelWriter explainTerms(RelWriter pw) {
         // for correct rel obtaining from ExecutionServiceImpl#physNodesCache.
         return super.explainTerms(pw)
-                .itemIf("tableId", getTable().unwrap(IgniteTable.class).id().toString(),
+                .itemIf("tableId", Integer.toString(getTable().unwrap(IgniteTable.class).id()),
                 pw.getDetailLevel() == ALL_ATTRIBUTES);
     }
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/ProjectableFilterableTableScan.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/ProjectableFilterableTableScan.java
index e264aded24..a9d71db288 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/ProjectableFilterableTableScan.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/rel/ProjectableFilterableTableScan.java
@@ -127,7 +127,7 @@ public abstract class ProjectableFilterableTableScan extends TableScan {
     public RelWriter explainTerms(RelWriter pw) {
         return explainTerms0(pw
                 .item("table", table.getQualifiedName())
-                .itemIf("tableId", table.unwrap(IgniteTable.class).id().toString(),
+                .itemIf("tableId", Integer.toString(table.unwrap(IgniteTable.class).id()),
                 pw.getDetailLevel() == ALL_ATTRIBUTES));
     }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
index 6bdcfd09d8..be50c17638 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTable.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.sql.engine.schema;
 import java.util.BitSet;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
@@ -57,7 +56,7 @@ public interface IgniteTable extends TranslatableTable, Wrapper {
      *
      * @return And id of the table.
      */
-    UUID id();
+    int id();
 
     /**
      * Returns the version of the table's schema.
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
index 2fcc10c3f7..10c7e23b35 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteTableImpl.java
@@ -29,7 +29,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
@@ -167,7 +166,7 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
 
     /** {@inheritDoc} */
     @Override
-    public UUID id() {
+    public int id() {
         return table.tableId();
     }
 
@@ -336,8 +335,6 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
 
         assert commitPartitionId != null;
 
-        UUID tableId = table.tableId();
-
         Int2ObjectOpenHashMap<List<BinaryRow>> rowsByPartition = new Int2ObjectOpenHashMap<>();
 
         for (RowT row : rows) {
@@ -351,7 +348,7 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
         int batchNum = 0;
 
         for (Int2ObjectMap.Entry<List<BinaryRow>> partToRows : rowsByPartition.int2ObjectEntrySet()) {
-            TablePartitionId partGroupId = new TablePartitionId(tableId, partToRows.getIntKey());
+            TablePartitionId partGroupId = new TablePartitionId(table.tableId(), partToRows.getIntKey());
             NodeWithTerm nodeWithTerm = ectx.description().mapping().updatingTableAssignments().get(partToRows.getIntKey());
 
             ReplicaRequest request = MESSAGES_FACTORY.readWriteMultiRowReplicaRequest()
@@ -383,8 +380,6 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
 
         RowHandler<RowT> handler = ectx.rowHandler();
 
-        UUID tableId = table.tableId();
-
         Int2ObjectOpenHashMap<List<BinaryRow>> rowsByPartition = new Int2ObjectOpenHashMap<>();
 
         for (RowT row : rows) {
@@ -398,7 +393,7 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
         int batchNum = 0;
 
         for (Int2ObjectMap.Entry<List<BinaryRow>> partToRows : rowsByPartition.int2ObjectEntrySet()) {
-            TablePartitionId partGroupId = new TablePartitionId(tableId, partToRows.getIntKey());
+            TablePartitionId partGroupId = new TablePartitionId(table.tableId(), partToRows.getIntKey());
             NodeWithTerm nodeWithTerm = ectx.description().mapping().updatingTableAssignments().get(partToRows.getIntKey());
 
             ReplicaRequest request = MESSAGES_FACTORY.readWriteMultiRowReplicaRequest()
@@ -448,8 +443,6 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
 
         assert commitPartitionId != null;
 
-        UUID tableId = table.tableId();
-
         Int2ObjectOpenHashMap<List<BinaryRow>> keyRowsByPartition = new Int2ObjectOpenHashMap<>();
 
         for (RowT row : rows) {
@@ -463,7 +456,7 @@ public class IgniteTableImpl extends AbstractTable implements IgniteTable, Updat
         int batchNum = 0;
 
         for (Int2ObjectMap.Entry<List<BinaryRow>> partToRows : keyRowsByPartition.int2ObjectEntrySet()) {
-            TablePartitionId partGroupId = new TablePartitionId(tableId, partToRows.getIntKey());
+            TablePartitionId partGroupId = new TablePartitionId(table.tableId(), partToRows.getIntKey());
             NodeWithTerm nodeWithTerm = ectx.description().mapping().updatingTableAssignments().get(partToRows.getIntKey());
 
             ReplicaRequest request = MESSAGES_FACTORY.readWriteMultiRowReplicaRequest()
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManager.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManager.java
index 75462e7a47..71592081c8 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManager.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManager.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.sql.engine.schema;
 
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.calcite.schema.SchemaPlus;
 import org.jetbrains.annotations.Nullable;
@@ -38,7 +37,7 @@ public interface SqlSchemaManager {
      *
      * @return The table.
      */
-    IgniteTable tableById(UUID id);
+    IgniteTable tableById(int id);
 
     /**
      * Wait for {@code ver} schema version, just a stub, need to be removed after IGNITE-18733.
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
index 00a916c421..00bf72aa05 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
@@ -65,7 +65,6 @@ import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.NodeStoppingException;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -74,9 +73,9 @@ import org.jetbrains.annotations.Nullable;
 public class SqlSchemaManagerImpl implements SqlSchemaManager {
     private final IncrementalVersionedValue<Map<String, IgniteSchema>> schemasVv;
 
-    private final IncrementalVersionedValue<Map<UUID, IgniteTable>> tablesVv;
+    private final IncrementalVersionedValue<Map<Integer, IgniteTable>> tablesVv;
 
-    private final Map<UUID, CompletableFuture<?>> pkIdxReady = new ConcurrentHashMap<>();
+    private final Map<Integer, CompletableFuture<?>> pkIdxReady = new ConcurrentHashMap<>();
 
     private final IncrementalVersionedValue<Map<UUID, IgniteIndex>> indicesVv;
 
@@ -186,8 +185,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
 
     /** {@inheritDoc} */
     @Override
-    @NotNull
-    public IgniteTable tableById(UUID id) {
+    public IgniteTable tableById(int id) {
         if (!busyLock.enterBusy()) {
             throw new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException());
         }
@@ -213,7 +211,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
      * OnSqlTypeCreated.
      * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
      */
-    public CompletableFuture<?> onTableCreated(String schemaName, UUID tableId, long causalityToken) {
+    public CompletableFuture<?> onTableCreated(String schemaName, int tableId, long causalityToken) {
         if (!busyLock.enterBusy()) {
             return failedFuture(new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException()));
         }
@@ -221,7 +219,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
         try {
             pkIdxReady.computeIfAbsent(tableId, k -> new CompletableFuture<>());
 
-            CompletableFuture<Map<UUID, IgniteTable>> updatedTables = tablesVv.update(causalityToken, (tables, e) ->
+            CompletableFuture<Map<Integer, IgniteTable>> updatedTables = tablesVv.update(causalityToken, (tables, e) ->
                     inBusyLock(busyLock, () -> {
                         if (e != null) {
                             return failedFuture(e);
@@ -230,7 +228,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
                         return tableManager.tableAsync(causalityToken, tableId)
                                 .thenCompose(table -> convert(causalityToken, table))
                                 .thenApply(igniteTable -> {
-                                    Map<UUID, IgniteTable> resTbls = new HashMap<>(tables);
+                                    Map<Integer, IgniteTable> resTbls = new HashMap<>(tables);
 
                                     IgniteTable oldTable = resTbls.put(igniteTable.id(), igniteTable);
 
@@ -276,7 +274,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
      * OnSqlTypeUpdated.
      * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
      */
-    public CompletableFuture<?> onTableUpdated(String schemaName, UUID tableId, long causalityToken) {
+    public CompletableFuture<?> onTableUpdated(String schemaName, int tableId, long causalityToken) {
         return onTableCreated(schemaName, tableId, causalityToken);
     }
 
@@ -284,7 +282,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
      * OnSqlTypeDropped.
      * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
      */
-    public CompletableFuture<?> onTableDropped(String schemaName, UUID tableId, long causalityToken) {
+    public CompletableFuture<?> onTableDropped(String schemaName, int tableId, long causalityToken) {
         if (!busyLock.enterBusy()) {
             return failedFuture(new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException()));
         }
@@ -299,7 +297,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
                     return failedFuture(e);
                 }
 
-                Map<UUID, IgniteTable> resTbls = new HashMap<>(tables);
+                Map<Integer, IgniteTable> resTbls = new HashMap<>(tables);
 
                 IgniteTable removedTable = resTbls.remove(tableId);
 
@@ -414,7 +412,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
      * @param causalityToken Causality token.
      * @return Schema registration future.
      */
-    public CompletableFuture<?> onIndexCreated(UUID tableId, UUID indexId, IndexDescriptor indexDescriptor, long causalityToken) {
+    public CompletableFuture<?> onIndexCreated(int tableId, UUID indexId, IndexDescriptor indexDescriptor, long causalityToken) {
         if (!busyLock.enterBusy()) {
             return failedFuture(new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException()));
         }
@@ -437,7 +435,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
                         });
                     }));
 
-            CompletableFuture<Map<UUID, IgniteTable>> updatedTables = tablesVv.update(causalityToken, (tables, e) ->
+            CompletableFuture<Map<Integer, IgniteTable>> updatedTables = tablesVv.update(causalityToken, (tables, e) ->
                     inBusyLock(busyLock, () -> {
                         if (e != null) {
                             return failedFuture(e);
@@ -446,7 +444,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
                         return updatedIndices.thenApply(indices -> {
                             IgniteIndex igniteIndex = indices.get(indexId);
 
-                            Map<UUID, IgniteTable> resTbls = new HashMap<>(tables);
+                            Map<Integer, IgniteTable> resTbls = new HashMap<>(tables);
 
                             IgniteTable igniteTable = resTbls.computeIfPresent(tableId,
                                     (k, v) -> IgniteTableImpl.copyOf((IgniteTableImpl) v));
@@ -509,7 +507,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
      * @param causalityToken Causality token.
      * @return Schema registration future.
      */
-    public CompletableFuture<?> onIndexDropped(String schemaName, UUID tableId, UUID indexId, long causalityToken) {
+    public CompletableFuture<?> onIndexDropped(String schemaName, int tableId, UUID indexId, long causalityToken) {
         if (!busyLock.enterBusy()) {
             return failedFuture(new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException()));
         }
@@ -533,13 +531,13 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
                 return completedFuture(resIdxs);
             }));
 
-            CompletableFuture<Map<UUID, IgniteTable>> updatedTables = tablesVv.update(causalityToken, (tables, e) ->
+            CompletableFuture<Map<Integer, IgniteTable>> updatedTables = tablesVv.update(causalityToken, (tables, e) ->
                     inBusyLock(busyLock, () -> {
                         if (e != null) {
                             return failedFuture(e);
                         }
 
-                        Map<UUID, IgniteTable> resTbls = new HashMap<>(tables);
+                        Map<Integer, IgniteTable> resTbls = new HashMap<>(tables);
 
                         IgniteTable table = resTbls.computeIfPresent(tableId, (k, v) -> IgniteTableImpl.copyOf((IgniteTableImpl) v));
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionFunction.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionFunction.java
index a34258caad..ed5e8584a2 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionFunction.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionFunction.java
@@ -23,7 +23,6 @@ import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
-import java.util.UUID;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.util.ImmutableIntList;
@@ -63,7 +62,7 @@ public abstract class DistributionFunction {
         return false;
     }
 
-    public static DistributionFunction affinity(UUID tableId, Object zoneId) {
+    public static DistributionFunction affinity(int tableId, Object zoneId) {
         return new AffinityDistribution(tableId, zoneId);
     }
 
@@ -245,7 +244,7 @@ public abstract class DistributionFunction {
      * Affinity distribution.
      */
     public static final class AffinityDistribution extends HashDistribution {
-        private final UUID tableId;
+        private final int tableId;
 
         private final Object zoneId;
 
@@ -255,7 +254,7 @@ public abstract class DistributionFunction {
          * @param tableId Table ID.
          * @param zoneId  Distribution zone ID.
          */
-        private AffinityDistribution(UUID tableId, Object zoneId) {
+        private AffinityDistribution(int tableId, Object zoneId) {
             this.zoneId = zoneId;
             this.tableId = tableId;
         }
@@ -266,7 +265,7 @@ public abstract class DistributionFunction {
             return true;
         }
 
-        public UUID tableId() {
+        public int tableId() {
             return tableId;
         }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistributions.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistributions.java
index 11e865f67c..9f5a0c0057 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistributions.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistributions.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.sql.engine.trait;
 
 import java.util.List;
-import java.util.UUID;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.util.ImmutableIntList;
 
@@ -72,7 +71,7 @@ public class IgniteDistributions {
      * @param zoneId  Distribution zone ID.
      * @return Affinity distribution.
      */
-    public static IgniteDistribution affinity(int key, UUID tableId, Object zoneId) {
+    public static IgniteDistribution affinity(int key, int tableId, Object zoneId) {
         return hash(ImmutableIntList.of(key), DistributionFunction.affinity(tableId, zoneId));
     }
 
@@ -85,7 +84,7 @@ public class IgniteDistributions {
      * @param zoneId  Distribution zone ID.
      * @return Affinity distribution.
      */
-    public static IgniteDistribution affinity(List<Integer> keys, UUID tableId, Object zoneId) {
+    public static IgniteDistribution affinity(List<Integer> keys, int tableId, Object zoneId) {
         return hash(keys, DistributionFunction.affinity(tableId, zoneId));
     }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactory.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactory.java
index eef2ca2a5f..98db93e282 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactory.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactory.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.sql.engine.util;
 
-import java.util.UUID;
-
 /**
  * Factory for creating a function to calculate the hash of the specified fields of the row.
  */
@@ -54,5 +52,5 @@ public interface HashFunctionFactory<T> {
      * @param tableId Table ID.
      * @return Function to compute a composite hash of a row, given the types and values of the fields.
      */
-    RowHashFunction<T> create(int[] fields, UUID tableId);
+    RowHashFunction<T> create(int[] fields, int tableId);
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactoryImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactoryImpl.java
index fdb6521b1c..4ad5484257 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactoryImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactoryImpl.java
@@ -21,7 +21,6 @@ import static org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactoryIm
 import static org.apache.ignite.lang.IgniteStringFormatter.format;
 
 import java.util.Objects;
-import java.util.UUID;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.ignite.internal.schema.NativeType;
 import org.apache.ignite.internal.schema.NativeTypeSpec;
@@ -46,7 +45,7 @@ public class HashFunctionFactoryImpl<T> implements HashFunctionFactory<T> {
 
     /** {@inheritDoc} */
     @Override
-    public RowHashFunction<T> create(int[] fields, UUID tableId) {
+    public RowHashFunction<T> create(int[] fields, int tableId) {
         int fieldCnt = fields.length;
         NativeType[] fieldTypes = new NativeType[fieldCnt];
         TableDescriptor tblDesc = sqlSchemaManager.tableById(tableId).descriptor();
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java
index 3c490600f7..8e8ede4832 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/StopCalciteModuleTest.java
@@ -39,7 +39,6 @@ import java.lang.management.ThreadMXBean;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
 import java.util.concurrent.Flow;
@@ -149,7 +148,7 @@ public class StopCalciteModuleTest {
 
     private final ClusterNode localNode = new ClusterNode("mock-node-id", NODE_NAME, null);
 
-    private UUID tblId = UUID.randomUUID();
+    private final int tblId = 1;
 
     /**
      * Before.
@@ -172,7 +171,7 @@ public class StopCalciteModuleTest {
 
         when(tbl.name()).thenReturn("TEST");
 
-        when(schemaManager.schemaRegistry(anyLong(), any())).thenReturn(completedFuture(schemaReg));
+        when(schemaManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(schemaReg));
 
         // Mock create table (notify on register listener).
         doAnswer(invocation -> {
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java
index 28b85ee041..9d9ee9916d 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImplTest.java
@@ -34,6 +34,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
@@ -481,7 +482,7 @@ public class ExecutionServiceImplTest {
 
         when(topologyService.localMember()).thenReturn(clusterNode);
 
-        when(schemaManagerMock.tableById(any())).thenReturn(table);
+        when(schemaManagerMock.tableById(anyInt())).thenReturn(table);
 
         when(schemaManagerMock.actualSchemaAsync(isA(long.class))).thenReturn(CompletableFuture.completedFuture(null));
 
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
index a7679278f1..20837d2995 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
@@ -150,7 +150,7 @@ public class TableScanNodeExecutionTest extends AbstractExecutionTest {
         ) {
             super(
                     "test",
-                    UUID.randomUUID(),
+                    1,
                     Int2ObjectMaps.singleton(0, mock(RaftGroupService.class)),
                     PART_CNT,
                     addr -> mock(ClusterNode.class),
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
index 49cd20ee66..d4fa19a9af 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
@@ -26,7 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
 import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertSame;
-import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.mock;
@@ -72,7 +72,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
  */
 @ExtendWith(MockitoExtension.class)
 public class SqlSchemaManagerTest {
-    private final UUID tableId = UUID.randomUUID();
+    private final int tableId = 1;
 
     private final UUID indexId = UUID.randomUUID();
 
@@ -141,7 +141,7 @@ public class SqlSchemaManagerTest {
         when(schemaRegistry.schema()).thenReturn(schemaDescriptor);
         when(schemaRegistry.lastSchemaVersion()).thenReturn(schemaDescriptor.version());
 
-        when(schemaManager.schemaRegistry(anyLong(), any())).thenReturn(completedFuture(schemaRegistry));
+        when(schemaManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(schemaRegistry));
 
         sqlSchemaManager.onTableCreated("PUBLIC", tableId, testRevisionRegister.actualToken() + 1);
         testRevisionRegister.moveForward();
@@ -179,7 +179,7 @@ public class SqlSchemaManagerTest {
         when(table.tableId()).thenReturn(tableId);
         when(schemaRegistry.schema()).thenReturn(schemaDescriptor);
         when(schemaRegistry.lastSchemaVersion()).thenReturn(schemaDescriptor.version());
-        when(schemaManager.schemaRegistry(anyLong(), any())).thenReturn(completedFuture(schemaRegistry));
+        when(schemaManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(schemaRegistry));
 
         sqlSchemaManager.onTableCreated("PUBLIC", tableId, testRevisionRegister.actualToken() + 1);
         testRevisionRegister.moveForward();
@@ -235,7 +235,7 @@ public class SqlSchemaManagerTest {
         when(table.tableId()).thenReturn(tableId);
         when(schemaRegistry.schema()).thenReturn(schemaDescriptor);
         when(schemaRegistry.lastSchemaVersion()).thenReturn(schemaDescriptor.version());
-        when(schemaManager.schemaRegistry(anyLong(), any())).thenReturn(completedFuture(schemaRegistry));
+        when(schemaManager.schemaRegistry(anyLong(), anyInt())).thenReturn(completedFuture(schemaRegistry));
 
         sqlSchemaManager.onTableCreated("PUBLIC", table.tableId(), testRevisionRegister.actualToken() + 1);
         testRevisionRegister.moveForward();
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/NoOpTransaction.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/NoOpTransaction.java
index 1800dc02ef..431e0a0395 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/NoOpTransaction.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/NoOpTransaction.java
@@ -41,7 +41,7 @@ public final class NoOpTransaction implements InternalTransaction {
 
     private final IgniteBiTuple<ClusterNode, Long> tuple;
 
-    private final TablePartitionId groupId = new TablePartitionId(UUID.randomUUID(), 0);
+    private final TablePartitionId groupId = new TablePartitionId(1, 0);
 
     /**
      * Constructs the object.
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/PredefinedSchemaManager.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/PredefinedSchemaManager.java
index a34dff4c1f..0f897a47cd 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/PredefinedSchemaManager.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/PredefinedSchemaManager.java
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 import java.util.stream.Collectors;
@@ -43,7 +42,7 @@ import org.jetbrains.annotations.Nullable;
  */
 public class PredefinedSchemaManager implements SqlSchemaManager {
     private final SchemaPlus root;
-    private final Map<UUID, IgniteTable> tableById;
+    private final Map<Integer, IgniteTable> tableById;
 
     /** Constructs schema manager from a single schema. */
     PredefinedSchemaManager(IgniteSchema schema) {
@@ -80,7 +79,7 @@ public class PredefinedSchemaManager implements SqlSchemaManager {
 
     /** {@inheritDoc} */
     @Override
-    public IgniteTable tableById(UUID id) {
+    public IgniteTable tableById(int id) {
         return tableById.get(id);
     }
 }
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestTable.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestTable.java
index be2680c85b..76e833ce96 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestTable.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestTable.java
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
@@ -64,7 +63,7 @@ public class TestTable implements IgniteTable {
     private static final String DATA_PROVIDER_NOT_CONFIGURED_MESSAGE_TEMPLATE =
             "DataProvider is not configured [table={}, node={}]";
 
-    private final UUID id = UUID.randomUUID();
+    private final int id = 1;
     private final Map<String, IgniteIndex> indexes = new HashMap<>();
 
     private final String name;
@@ -122,7 +121,7 @@ public class TestTable implements IgniteTable {
 
     /** {@inheritDoc} */
     @Override
-    public UUID id() {
+    public int id() {
         return id;
     }
 
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractAggregatePlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractAggregatePlannerTest.java
index 69ba45133f..fbff6b9e03 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractAggregatePlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractAggregatePlannerTest.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Objects;
-import java.util.UUID;
 import java.util.function.Consumer;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
@@ -42,7 +41,6 @@ import org.apache.ignite.internal.sql.engine.rel.agg.IgniteReduceAggregateBase;
 import org.apache.ignite.internal.sql.engine.schema.IgniteIndex;
 import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
-import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.trait.TraitUtils;
 import org.hamcrest.Matchers;
 import org.junit.jupiter.api.AfterAll;
@@ -541,7 +539,7 @@ public abstract class AbstractAggregatePlannerTest extends AbstractPlannerTest {
     }
 
     private static IgniteDistribution hash() {
-        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+        return someAffinity();
     }
 
     private static Consumer<org.apache.ignite.internal.sql.engine.framework.TestTable> index(String name, int... cols) {
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
index 337a584a74..b8f152ec48 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
@@ -43,6 +43,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.Flow.Publisher;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.BiFunction;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
@@ -118,6 +119,7 @@ import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
 import org.apache.ignite.internal.sql.engine.schema.IgniteTable;
 import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.trait.TraitUtils;
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.util.BaseQueryContext;
@@ -143,6 +145,8 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
 
     protected static final int DEFAULT_ZONE_ID = 0;
 
+    private static final AtomicInteger NEXT_TABLE_ID = new AtomicInteger(2001);
+
     /** Last error message. */
     String lastErrorMsg;
 
@@ -180,6 +184,14 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
         n.childrenAccept(new TestRelVisitor(v));
     }
 
+    protected static IgniteDistribution someAffinity() {
+        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
+    }
+
+    protected static int nextTableId() {
+        return NEXT_TABLE_ID.getAndIncrement();
+    }
+
     /**
      * FindFirstNode.
      * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
@@ -807,7 +819,7 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
 
         private final TableDescriptor desc;
 
-        private final UUID id = UUID.randomUUID();
+        private final int id = nextTableId();
 
         /** Constructor. */
         public TestTable(RelDataType type) {
@@ -835,7 +847,7 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
 
         /** {@inheritDoc} */
         @Override
-        public UUID id() {
+        public int id() {
             return id;
         }
 
@@ -1197,7 +1209,7 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
     static class TestSortedIndex implements SortedIndex {
         private final UUID id = UUID.randomUUID();
 
-        private final UUID tableId = UUID.randomUUID();
+        private final int tableId = 1;
 
         private final SortedIndexDescriptor descriptor;
 
@@ -1237,7 +1249,7 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
 
         /** {@inheritDoc} */
         @Override
-        public UUID tableId() {
+        public int tableId() {
             return tableId;
         }
 
@@ -1278,12 +1290,12 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
     public static class TestHashIndex implements Index<IndexDescriptor> {
         private final UUID id = UUID.randomUUID();
 
-        private UUID tableId = UUID.randomUUID();
+        private int tableId = 1;
 
         private final IndexDescriptor descriptor;
 
         /** Create index. */
-        public static TestHashIndex create(List<String> indexedColumns, String name, UUID tableId) {
+        public static TestHashIndex create(List<String> indexedColumns, String name, int tableId) {
             var descriptor = new IndexDescriptor(name, indexedColumns);
 
             TestHashIndex idx = new TestHashIndex(descriptor);
@@ -1318,7 +1330,7 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
 
         /** {@inheritDoc} */
         @Override
-        public UUID tableId() {
+        public int tableId() {
             return tableId;
         }
 
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/DmlPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/DmlPlannerTest.java
index 6200ffa7cb..bd5a66e808 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/DmlPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/DmlPlannerTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.sql.engine.planner;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.stream.Stream;
 import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.ignite.internal.schema.NativeTypes;
@@ -82,7 +81,7 @@ public class DmlPlannerTest extends AbstractPlannerTest {
     @ParameterizedTest
     @MethodSource("distributions")
     public void testInsertSelectFrom(IgniteDistribution distribution) throws Exception {
-        IgniteDistribution anotherDistribution = IgniteDistributions.affinity(1, new UUID(1, 0), "0");
+        IgniteDistribution anotherDistribution = IgniteDistributions.affinity(1, 1, "0");
 
         IgniteTable test1 = newTestTable("TEST1", distribution);
         IgniteTable test2 = newTestTable("TEST2", anotherDistribution);
@@ -150,7 +149,7 @@ public class DmlPlannerTest extends AbstractPlannerTest {
         return Stream.of(
                 IgniteDistributions.single(),
                 IgniteDistributions.hash(List.of(0, 1)),
-                IgniteDistributions.affinity(0, new UUID(1, 1), "0")
+                IgniteDistributions.affinity(0, 2, "0")
         );
     }
 
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexPlannerTest.java
index 9f724d8ecc..717938a84c 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexPlannerTest.java
@@ -23,7 +23,6 @@ import static org.hamcrest.Matchers.notNullValue;
 import static org.hamcrest.Matchers.nullValue;
 
 import java.util.List;
-import java.util.UUID;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.ignite.internal.sql.engine.rel.IgniteIndexScan;
@@ -43,7 +42,7 @@ public class HashIndexPlannerTest extends AbstractPlannerTest {
 
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
+                someAffinity(),
                 "ID", Integer.class,
                 "VAL", Integer.class
         );
@@ -70,7 +69,7 @@ public class HashIndexPlannerTest extends AbstractPlannerTest {
 
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
+                someAffinity(),
                 "ID", Integer.class,
                 "VAL", Integer.class
         );
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexSpoolPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexSpoolPlannerTest.java
index 7d58a07b7a..0724c2699f 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexSpoolPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashIndexSpoolPlannerTest.java
@@ -23,7 +23,6 @@ import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
-import java.util.UUID;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexFieldAccess;
@@ -57,14 +56,14 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
                 .add("ID", f.createJavaType(Integer.class))
                 .add("JID", f.createJavaType(Integer.class))
                 .add("VAL", f.createJavaType(String.class))
-                .build(), IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID));
+                .build(), IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID));
 
         createTable(publicSchema, "T1",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("JID", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
-                        .build(), IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID))
+                        .build(), IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID))
                 .addIndex("t1_jid_idx", 1, 0);
 
         String sql = "select * "
@@ -107,7 +106,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 }
         );
@@ -123,7 +122,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 }
                         .addIndex("t1_jid0_idx", 1, 0)
@@ -170,7 +169,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 }
         );
@@ -185,7 +184,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 }
         );
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinColocationPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinColocationPlannerTest.java
index 9ada56654b..b8decb7a1f 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinColocationPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinColocationPlannerTest.java
@@ -25,7 +25,6 @@ import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.hasSize;
 
 import java.util.List;
-import java.util.UUID;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelNode;
@@ -52,7 +51,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
     public void joinSameTableSimpleAff() throws Exception {
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
+                IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID),
                 "ID", Integer.class,
                 "VAL", String.class
         );
@@ -84,7 +83,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
     public void joinSameTableComplexAff() throws Exception {
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), UUID.randomUUID(), DEFAULT_ZONE_ID),
+                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), nextTableId(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -120,7 +119,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
         TestTable complexTbl = createTable(
                 "COMPLEX_TBL",
                 2 * DEFAULT_TBL_SIZE,
-                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), UUID.randomUUID(), DEFAULT_ZONE_ID),
+                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), nextTableId(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -131,7 +130,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
         TestTable simpleTbl = createTable(
                 "SIMPLE_TBL",
                 DEFAULT_TBL_SIZE,
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
+                IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID),
                 "ID", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -174,7 +173,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
         TestTable complexTblDirect = createTable(
                 "COMPLEX_TBL_DIRECT",
                 2 * DEFAULT_TBL_SIZE,
-                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), UUID.randomUUID(), DEFAULT_ZONE_ID),
+                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), nextTableId(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -186,7 +185,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
         TestTable complexTblIndirect = createTable(
                 "COMPLEX_TBL_INDIRECT",
                 DEFAULT_TBL_SIZE,
-                IgniteDistributions.affinity(ImmutableIntList.of(1, 0), UUID.randomUUID(), DEFAULT_ZONE_ID),
+                IgniteDistributions.affinity(ImmutableIntList.of(1, 0), nextTableId(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinCommutePlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinCommutePlannerTest.java
index ccf4042e13..bf59930835 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinCommutePlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/JoinCommutePlannerTest.java
@@ -25,7 +25,6 @@ import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.calcite.plan.RelOptCost;
@@ -72,7 +71,7 @@ public class JoinCommutePlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 },
                 new TestTable(
@@ -83,7 +82,7 @@ public class JoinCommutePlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 }
         );
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
index cb0c1014d1..2f887ffb83 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/PlannerTest.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Predicate;
 import org.apache.calcite.plan.RelOptUtil;
@@ -93,7 +92,7 @@ public class PlannerTest extends AbstractPlannerTest {
         NODES_WITH_TERM = new ArrayList<>(4);
 
         for (int i = 0; i < 4; i++) {
-            String nodeName = UUID.randomUUID().toString();
+            String nodeName = Integer.toString(nextTableId());
 
             NODES.add(nodeName);
             NODES_WITH_TERM.add(new NodeWithTerm(nodeName, 0L));
@@ -127,7 +126,7 @@ public class PlannerTest extends AbstractPlannerTest {
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -147,7 +146,7 @@ public class PlannerTest extends AbstractPlannerTest {
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -308,7 +307,7 @@ public class PlannerTest extends AbstractPlannerTest {
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -389,7 +388,7 @@ public class PlannerTest extends AbstractPlannerTest {
 
             @Override
             public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -471,7 +470,7 @@ public class PlannerTest extends AbstractPlannerTest {
 
             @Override
             public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -817,7 +816,7 @@ public class PlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                        return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
                     }
                 }
         );
@@ -852,7 +851,7 @@ public class PlannerTest extends AbstractPlannerTest {
                         .build(), "TAB0") {
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
+                return IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -871,10 +870,10 @@ public class PlannerTest extends AbstractPlannerTest {
     @Test
     public void checkTableHintsHandling() throws Exception {
         IgniteSchema publicSchema = createSchema(
-                createTable("PERSON", IgniteDistributions.affinity(0, UUID.randomUUID(), Integer.MIN_VALUE),
+                createTable("PERSON", IgniteDistributions.affinity(0, nextTableId(), Integer.MIN_VALUE),
                         "PK", Integer.class, "ORG_ID", Integer.class
                 ),
-                createTable("COMPANY", IgniteDistributions.affinity(0, UUID.randomUUID(), Integer.MIN_VALUE),
+                createTable("COMPANY", IgniteDistributions.affinity(0, nextTableId(), Integer.MIN_VALUE),
                         "PK", Integer.class, "ID", Integer.class
                 )
         );
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SetOpPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SetOpPlannerTest.java
index c20db5c0a6..c2fd38c0ff 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SetOpPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SetOpPlannerTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.sql.engine.planner;
 
 import java.util.List;
-import java.util.UUID;
 import org.apache.calcite.rel.RelDistribution.Type;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -74,11 +73,11 @@ public class SetOpPlannerTest extends AbstractPlannerTest {
         createTable(publicSchema, "BROADCAST_TBL2", type, IgniteDistributions.broadcast());
         createTable(publicSchema, "SINGLE_TBL1", type, IgniteDistributions.single());
         createTable(publicSchema, "SINGLE_TBL2", type, IgniteDistributions.single());
-        createTable(publicSchema, "AFFINITY_TBL1", type, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID));
+        createTable(publicSchema, "AFFINITY_TBL1", type, IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID));
         createTable(publicSchema, "HASH_TBL1", type, IgniteDistributions.hash(List.of(0)));
-        createTable(publicSchema, "AFFINITY_TBL2", type, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID));
-        createTable(publicSchema, "AFFINITY_TBL3", type, IgniteDistributions.affinity(1, UUID.randomUUID(), DEFAULT_ZONE_ID));
-        createTable(publicSchema, "AFFINITY_TBL4", type, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID + 1));
+        createTable(publicSchema, "AFFINITY_TBL2", type, IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID));
+        createTable(publicSchema, "AFFINITY_TBL3", type, IgniteDistributions.affinity(1, nextTableId(), DEFAULT_ZONE_ID));
+        createTable(publicSchema, "AFFINITY_TBL4", type, IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID + 1));
     }
 
     /**
@@ -247,7 +246,7 @@ public class SetOpPlannerTest extends AbstractPlannerTest {
 
         assertPlan(sql, publicSchema, isInstanceOf(IgniteExchange.class)
                 .and(input(isInstanceOf(setOp.colocated)
-                        .and(hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)))
+                        .and(hasDistribution(IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID)))
                         .and(input(0, isTableScan("affinity_tbl1")))
                         .and(input(1, isTableScan("affinity_tbl2")))
                 ))
@@ -271,7 +270,7 @@ public class SetOpPlannerTest extends AbstractPlannerTest {
 
         assertPlan(sql, publicSchema, isInstanceOf(IgniteExchange.class)
                 .and(input(isInstanceOf(setOp.colocated)
-                        .and(hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)))
+                        .and(hasDistribution(IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID)))
                         .and(input(0, isTableScan("affinity_tbl1")))
                         .and(input(1, isInstanceOf(IgniteTrimExchange.class)
                                 .and(input(isTableScan("broadcast_tbl1")))
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortedIndexSpoolPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortedIndexSpoolPlannerTest.java
index 16735a59ee..c5749bb3e0 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortedIndexSpoolPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortedIndexSpoolPlannerTest.java
@@ -24,7 +24,6 @@ import static org.junit.jupiter.api.Assertions.assertNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
-import java.util.UUID;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -37,7 +36,6 @@ import org.apache.ignite.internal.sql.engine.rel.IgniteCorrelatedNestedLoopJoin;
 import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
 import org.apache.ignite.internal.sql.engine.rel.IgniteSortedIndexSpool;
 import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
-import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.trait.TraitUtils;
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.util.Commons;
@@ -65,7 +63,7 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
                         .add("JID", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                someAffinity()
         ).addIndex("t0_jid_idx", 1, 0);
 
         createTable(publicSchema,
@@ -75,7 +73,7 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
                         .add("JID", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                someAffinity()
         ).addIndex("t1_jid_idx", 1, 0);
 
         String sql = "select * "
@@ -116,7 +114,7 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
                         .add("JID1", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                someAffinity()
         );
 
         createTable(publicSchema,
@@ -127,7 +125,7 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
                         .add("JID1", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                someAffinity()
         ).addIndex("t1_jid0_idx", 2, 1);
 
         String sql = "select * "
@@ -161,10 +159,10 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
     @Test
     public void testDescFields() throws Exception {
         IgniteSchema publicSchema = createSchema(
-                createTable("T0", 10, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
+                createTable("T0", 10, someAffinity(),
                         "ID", Integer.class, "JID", Integer.class, "VAL", String.class)
                         .addIndex("t0_jid_idx", 1),
-                createTable("T1", 100, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
+                createTable("T1", 100, someAffinity(),
                         "ID", Integer.class, "JID", Integer.class, "VAL", String.class)
                         .addIndex(RelCollations.of(TraitUtils.createFieldCollation(1, ColumnCollation.DESC_NULLS_LAST)), "t1_jid_idx")
         );
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/TableSpoolPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/TableSpoolPlannerTest.java
index e86d0c95f1..5e46300b7a 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/TableSpoolPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/TableSpoolPlannerTest.java
@@ -19,13 +19,11 @@ package org.apache.ignite.internal.sql.engine.planner;
 
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 
-import java.util.UUID;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
 import org.apache.ignite.internal.sql.engine.rel.IgniteTableSpool;
 import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
-import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.junit.jupiter.api.Disabled;
@@ -54,7 +52,7 @@ public class TableSpoolPlannerTest extends AbstractPlannerTest {
                         .add("JID", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                someAffinity()
         );
 
         createTable(publicSchema,
@@ -64,7 +62,7 @@ public class TableSpoolPlannerTest extends AbstractPlannerTest {
                         .add("JID", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                someAffinity()
         );
 
         String sql = "select * "
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/UnionPlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/UnionPlannerTest.java
index abf870cd8e..48b9b763c9 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/UnionPlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/UnionPlannerTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.sql.engine.planner;
 
-import java.util.UUID;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.ignite.internal.sql.engine.rel.IgniteUnionAll;
@@ -88,7 +87,7 @@ public class UnionPlannerTest extends AbstractPlannerTest {
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID)
         );
 
         createTable(publicSchema,
@@ -98,10 +97,9 @@ public class UnionPlannerTest extends AbstractPlannerTest {
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID)
         );
 
-
         createTable(publicSchema,
                 "TABLE3",
                 new RelDataTypeFactory.Builder(f)
@@ -109,7 +107,7 @@ public class UnionPlannerTest extends AbstractPlannerTest {
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
                         .build(),
-                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+                IgniteDistributions.affinity(0, nextTableId(), DEFAULT_ZONE_ID)
         );
 
         return publicSchema;
diff --git a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
index 399a0f5852..02bc244f71 100644
--- a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
+++ b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
@@ -49,7 +49,7 @@ public interface MvPartitionStorage extends ManuallyCloseable {
 
     /**
      * Closure for executing write operations on the storage. All write operations, such as
-     * {@link #addWrite(RowId, BinaryRow, UUID, UUID, int)} or {@link #commitWrite(RowId, HybridTimestamp)},
+     * {@link #addWrite(RowId, BinaryRow, UUID, int, int)} or {@link #commitWrite(RowId, HybridTimestamp)},
      * as well as {@link #scanVersions(RowId)}, and operations like {@link #committedGroupConfiguration(byte[])}, must be executed inside
      * of the write closure. Also, each operation that involves modifying rows (and {@link #scanVersions(RowId)}) must hold lock on
      * the corresponding row ID, by either calling {@link Locker#lock(RowId)} or calling {@link Locker#tryLock(RowId)} and checking the
@@ -182,7 +182,7 @@ public interface MvPartitionStorage extends ManuallyCloseable {
      * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
      * @throws StorageException If failed to write data to the storage.
      */
-    @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId)
+    @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId, int commitPartitionId)
             throws TxIdMismatchException, StorageException;
 
     /**
diff --git a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/ReadResult.java b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/ReadResult.java
index 3539fa8084..8ab1a45454 100644
--- a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/ReadResult.java
+++ b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/ReadResult.java
@@ -39,7 +39,7 @@ public class ReadResult {
     private final @Nullable UUID transactionId;
 
     /** Commit table id. Not {@code null} iff this is a write-intent. */
-    private final @Nullable UUID commitTableId;
+    private final @Nullable Integer commitTableId;
 
     /** Commit table id. If this is not a write-intent it is equal to {@link #UNDEFINED_COMMIT_PARTITION_ID}. */
     private final int commitPartitionId;
@@ -61,7 +61,7 @@ public class ReadResult {
             RowId rowId,
             @Nullable BinaryRow binaryRow,
             @Nullable UUID transactionId,
-            @Nullable UUID commitTableId,
+            @Nullable Integer commitTableId,
             @Nullable HybridTimestamp commitTs,
             @Nullable HybridTimestamp newestCommitTs,
             int commitPartitionId
@@ -92,7 +92,7 @@ public class ReadResult {
         return new ReadResult(rowId, null, null, null, null, null, UNDEFINED_COMMIT_PARTITION_ID);
     }
 
-    public static ReadResult createFromWriteIntent(RowId rowId, @Nullable BinaryRow binaryRow, UUID transactionId, UUID commitTableId,
+    public static ReadResult createFromWriteIntent(RowId rowId, @Nullable BinaryRow binaryRow, UUID transactionId, int commitTableId,
             int commitPartitionId, @Nullable HybridTimestamp lastCommittedTimestamp) {
         return new ReadResult(rowId, binaryRow, transactionId, commitTableId, null, lastCommittedTimestamp, commitPartitionId);
     }
@@ -137,7 +137,7 @@ public class ReadResult {
      * @return Commit table id part of the transaction state if this is a write-intent,
      *         {@code null} otherwise.
      */
-    public @Nullable UUID commitTableId() {
+    public @Nullable Integer commitTableId() {
         return commitTableId;
     }
 
diff --git a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
index 247eef0b7c..284dca252c 100644
--- a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
+++ b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
@@ -22,6 +22,7 @@ import static java.util.stream.Collectors.toUnmodifiableList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
+import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.internal.schema.NativeType;
 import org.apache.ignite.internal.schema.configuration.ColumnView;
 import org.apache.ignite.internal.schema.configuration.ConfigurationToSchemaDescriptorConverter;
@@ -31,6 +32,7 @@ import org.apache.ignite.internal.schema.configuration.index.HashIndexView;
 import org.apache.ignite.internal.schema.configuration.index.TableIndexView;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.tostring.S;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Descriptor for creating a Hash Index Storage.
@@ -132,7 +134,7 @@ public class HashIndexDescriptor implements IndexDescriptor {
             ));
         }
 
-        TableView tableConfig = tablesConfig.tables().get(indexConfig.tableId());
+        TableView tableConfig = findTableById(indexConfig.tableId(), tablesConfig.tables());
 
         if (tableConfig == null) {
             throw new StorageException(String.format("Table configuration for \"%s\" could not be found", indexConfig.tableId()));
@@ -151,6 +153,16 @@ public class HashIndexDescriptor implements IndexDescriptor {
                 .collect(toUnmodifiableList());
     }
 
+    private @Nullable static TableView findTableById(int tableId, NamedListView<? extends TableView> tablesView) {
+        for (TableView table : tablesView) {
+            if (table.id() == tableId) {
+                return table;
+            }
+        }
+
+        return null;
+    }
+
     @Override
     public UUID id() {
         return id;
diff --git a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
index 1acc31ae02..1e22bab27a 100644
--- a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
+++ b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.schema.configuration.index.SortedIndexView;
 import org.apache.ignite.internal.schema.configuration.index.TableIndexView;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.tostring.S;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Descriptor for creating a Sorted Index Storage.
@@ -151,7 +152,7 @@ public class SortedIndexDescriptor implements IndexDescriptor {
             ));
         }
 
-        TableView tableConfig = tablesConfig.tables().get(indexConfig.tableId());
+        TableView tableConfig = findTableById(indexConfig.tableId(), tablesConfig.tables());
 
         if (tableConfig == null) {
             throw new StorageException(String.format("Table configuration for \"%s\" could not be found", indexConfig.tableId()));
@@ -180,6 +181,16 @@ public class SortedIndexDescriptor implements IndexDescriptor {
         return BinaryTupleSchema.create(elements);
     }
 
+    private @Nullable static TableView findTableById(int tableId, NamedListView<? extends TableView> tablesView) {
+        for (TableView table : tablesView) {
+            if (table.id() == tableId) {
+                return table;
+            }
+        }
+
+        return null;
+    }
+
     @Override
     public UUID id() {
         return id;
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
index 1de404bf06..0ab70eac81 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
@@ -766,7 +766,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
             locker.lock(rowId);
 
-            storage.addWrite(rowId, binaryRow, txId, UUID.randomUUID(), 0);
+            storage.addWrite(rowId, binaryRow, txId, 999, 0);
             commitWrite(rowId, clock.now());
 
             addWrite(rowId, binaryRow2, newTransactionId());
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
index 69056ef8f6..656ea9fda6 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
@@ -49,7 +49,6 @@ import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder;
-import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
 import org.apache.ignite.internal.distributionzones.configuration.DistributionZoneConfiguration;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.schema.BinaryRow;
@@ -100,6 +99,8 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
     /** Partition id for 1 storage. */
     protected static final int PARTITION_ID_1 = 1 << 8;
 
+    protected static final int COMMIT_TABLE_ID = 999;
+
     protected MvTableStorage tableStorage;
 
     protected SortedIndexDescriptor sortedIdx;
@@ -189,7 +190,7 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
         partitionStorage0.runConsistently(locker -> {
             locker.lock(rowId0);
 
-            return partitionStorage0.addWrite(rowId0, testData0, txId, UUID.randomUUID(), 0);
+            return partitionStorage0.addWrite(rowId0, testData0, txId, COMMIT_TABLE_ID, 0);
         });
 
         assertThat(unwrap(partitionStorage0.read(rowId0, HybridTimestamp.MAX_VALUE)), is(equalTo(unwrap(testData0))));
@@ -202,7 +203,7 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
         partitionStorage1.runConsistently(locker -> {
             locker.lock(rowId1);
 
-            return partitionStorage1.addWrite(rowId1, testData1, txId, UUID.randomUUID(), 0);
+            return partitionStorage1.addWrite(rowId1, testData1, txId, COMMIT_TABLE_ID, 0);
         });
 
         assertThrows(IllegalArgumentException.class, () -> partitionStorage0.read(rowId1, HybridTimestamp.MAX_VALUE));
@@ -828,7 +829,7 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
                         .build()
         );
 
-        UUID tableId = ConfigurationUtil.internalId(tablesConfig.tables().value(), "foo");
+        int tableId = tablesConfig.tables().value().get("foo").id();
 
         CompletableFuture<Void> indexCreateFut = tablesConfig.indexes().change(ch ->
                 indexDefinitions.forEach(idxDef -> ch.create(idxDef.name(),
@@ -883,7 +884,7 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
 
         BinaryRow binaryRow = binaryRow(new TestKey(0, "0"), new TestValue(1, "1"));
 
-        assertThrows(StorageClosedException.class, () -> storage.addWrite(rowId, binaryRow, UUID.randomUUID(), UUID.randomUUID(), partId));
+        assertThrows(StorageClosedException.class, () -> storage.addWrite(rowId, binaryRow, UUID.randomUUID(), COMMIT_TABLE_ID, partId));
         assertThrows(StorageClosedException.class, () -> storage.commitWrite(rowId, timestamp));
         assertThrows(StorageClosedException.class, () -> storage.abortWrite(rowId));
         assertThrows(StorageClosedException.class, () -> storage.addWriteCommitted(rowId, binaryRow, timestamp));
@@ -1042,7 +1043,7 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
                 locker.lock(rowId);
 
                 if ((finalI % 2) == 0) {
-                    mvPartitionStorage.addWrite(rowId, binaryRow, UUID.randomUUID(), UUID.randomUUID(), rowId.partitionId());
+                    mvPartitionStorage.addWrite(rowId, binaryRow, UUID.randomUUID(), COMMIT_TABLE_ID, rowId.partitionId());
 
                     mvPartitionStorage.commitWrite(rowId, timestamp);
                 } else {
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvPartitionStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvPartitionStorageTest.java
index aeb29927ba..9177bb6e2a 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvPartitionStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvPartitionStorageTest.java
@@ -41,7 +41,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
 public abstract class BaseMvPartitionStorageTest extends BaseMvStoragesTest {
     protected static final int PARTITION_ID = 1;
 
-    protected static final UUID COMMIT_TABLE_ID = UUID.randomUUID();
+    protected static final int COMMIT_TABLE_ID = 999;
 
     protected static final UUID TX_ID = newTransactionId();
 
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
index 198b9dda77..ba81744ea8 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java
@@ -81,12 +81,19 @@ public class TestMvPartitionStorage implements MvPartitionStorage {
         final @Nullable BinaryRow row;
         final @Nullable HybridTimestamp ts;
         final @Nullable UUID txId;
-        final @Nullable UUID commitTableId;
+        final @Nullable Integer commitTableId;
         final int commitPartitionId;
         volatile @Nullable VersionChain next;
 
-        VersionChain(RowId rowId, @Nullable BinaryRow row, @Nullable HybridTimestamp ts, @Nullable UUID txId, @Nullable UUID commitTableId,
-                int commitPartitionId, @Nullable VersionChain next) {
+        VersionChain(
+                RowId rowId,
+                @Nullable BinaryRow row,
+                @Nullable HybridTimestamp ts,
+                @Nullable UUID txId,
+                @Nullable Integer commitTableId,
+                int commitPartitionId,
+                @Nullable VersionChain next
+        ) {
             this.rowId = rowId;
             this.row = row;
             this.ts = ts;
@@ -96,7 +103,7 @@ public class TestMvPartitionStorage implements MvPartitionStorage {
             this.next = next;
         }
 
-        static VersionChain forWriteIntent(RowId rowId, @Nullable BinaryRow row, @Nullable UUID txId, @Nullable UUID commitTableId,
+        static VersionChain forWriteIntent(RowId rowId, @Nullable BinaryRow row, @Nullable UUID txId, @Nullable Integer commitTableId,
                 int commitPartitionId, @Nullable VersionChain next) {
             return new VersionChain(rowId, row, null, txId, commitTableId, commitPartitionId, next);
         }
@@ -202,7 +209,7 @@ public class TestMvPartitionStorage implements MvPartitionStorage {
             RowId rowId,
             @Nullable BinaryRow row,
             UUID txId,
-            UUID commitTableId,
+            int commitTableId,
             int commitPartitionId
     ) throws TxIdMismatchException {
         checkStorageClosed();
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
index 6bf4ef63dc..e3c5bedf27 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
@@ -23,10 +23,8 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.empty;
 
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.stream.Stream;
-import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
 import org.apache.ignite.internal.schema.configuration.TablesView;
 import org.apache.ignite.internal.schema.testutils.builder.SchemaBuilders;
 import org.apache.ignite.internal.schema.testutils.definition.ColumnType;
@@ -48,7 +46,7 @@ public abstract class AbstractHashIndexStorageTest extends AbstractIndexStorageT
 
         CompletableFuture<Void> createIndexFuture = tablesCfg.indexes()
                 .change(chg -> chg.create(indexDefinition.name(), idx -> {
-                    UUID tableId = ConfigurationUtil.internalId(tablesCfg.tables().value(), TABLE_NAME);
+                    int tableId = tablesCfg.tables().value().get(TABLE_NAME).id();
 
                     addIndex(indexDefinition, tableId, idx);
                 }));
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
index 0d130ba602..cd7a3a8078 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
@@ -45,12 +45,10 @@ import java.util.Collections;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Objects;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.stream.IntStream;
-import org.apache.ignite.internal.configuration.util.ConfigurationUtil;
 import org.apache.ignite.internal.logger.IgniteLogger;
 import org.apache.ignite.internal.logger.Loggers;
 import org.apache.ignite.internal.schema.BinaryTuplePrefix;
@@ -122,7 +120,7 @@ public abstract class AbstractSortedIndexStorageTest extends AbstractIndexStorag
     protected SortedIndexStorage createIndexStorage(ColumnarIndexDefinition indexDefinition) {
         CompletableFuture<Void> createIndexFuture =
                 tablesCfg.indexes().change(chg -> chg.create(indexDefinition.name(), idx -> {
-                    UUID tableId = ConfigurationUtil.internalId(tablesCfg.tables().value(), TABLE_NAME);
+                    int tableId = tablesCfg.tables().value().get(TABLE_NAME).id();
 
                     addIndex(indexDefinition, tableId, idx);
                 }));
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java
index 5c7a040f19..334d7fc316 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java
@@ -82,7 +82,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
 
         this.engine = engine;
         this.dataRegion = dataRegion;
-        this.tableId = tableCfg.tableId().value();
+        this.tableId = tableCfg.id().value();
     }
 
     /**
@@ -162,7 +162,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             assert !filePageStore.isMarkedToDestroy() : IgniteStringFormatter.format(
                     "Should not be marked for deletion: [tableName={}, tableId={}, partitionId={}]",
                     tableView.name(),
-                    tableView.tableId(),
+                    tableView.id(),
                     groupPartitionId.getPartitionId()
             );
 
@@ -212,7 +212,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             boolean initNew = false;
 
             if (meta.rowVersionFreeListRootPageId() == 0) {
-                long rootPageId = pageMemory.allocatePage(tableView.tableId(), partId, FLAG_AUX);
+                long rootPageId = pageMemory.allocatePage(tableView.id(), partId, FLAG_AUX);
 
                 meta.rowVersionFreeListRootPageId(lastCheckpointId(), rootPageId);
 
@@ -220,7 +220,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             }
 
             return new RowVersionFreeList(
-                    tableView.tableId(),
+                    tableView.id(),
                     partId,
                     dataRegion.pageMemory(),
                     null,
@@ -260,7 +260,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             boolean initNew = false;
 
             if (meta.indexColumnsFreeListRootPageId() == 0L) {
-                long rootPageId = pageMemory.allocatePage(tableView.tableId(), partitionId, FLAG_AUX);
+                long rootPageId = pageMemory.allocatePage(tableView.id(), partitionId, FLAG_AUX);
 
                 meta.indexColumnsFreeListRootPageId(lastCheckpointId(), rootPageId);
 
@@ -268,7 +268,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             }
 
             return new IndexColumnsFreeList(
-                    tableView.tableId(),
+                    tableView.id(),
                     partitionId,
                     pageMemory,
                     reuseList,
@@ -308,7 +308,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             boolean initNew = false;
 
             if (meta.versionChainTreeRootPageId() == 0) {
-                long rootPageId = pageMemory.allocatePage(tableView.tableId(), partId, FLAG_AUX);
+                long rootPageId = pageMemory.allocatePage(tableView.id(), partId, FLAG_AUX);
 
                 meta.versionChainTreeRootPageId(lastCheckpointId(), rootPageId);
 
@@ -316,7 +316,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             }
 
             return new VersionChainTree(
-                    tableView.tableId(),
+                    tableView.id(),
                     tableView.name(),
                     partId,
                     dataRegion.pageMemory(),
@@ -355,7 +355,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             boolean initNew = false;
 
             if (meta.indexTreeMetaPageId() == 0) {
-                long rootPageId = pageMemory.allocatePage(tableView.tableId(), partitionId, FLAG_AUX);
+                long rootPageId = pageMemory.allocatePage(tableView.id(), partitionId, FLAG_AUX);
 
                 meta.indexTreeMetaPageId(lastCheckpointId(), rootPageId);
 
@@ -363,7 +363,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             }
 
             return new IndexMetaTree(
-                    tableView.tableId(),
+                    tableView.id(),
                     tableView.name(),
                     partitionId,
                     dataRegion.pageMemory(),
@@ -402,7 +402,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             boolean initNew = false;
 
             if (meta.gcQueueMetaPageId() == 0) {
-                long rootPageId = pageMemory.allocatePage(tableView.tableId(), partitionId, FLAG_AUX);
+                long rootPageId = pageMemory.allocatePage(tableView.id(), partitionId, FLAG_AUX);
 
                 meta.gcQueueMetaPageId(lastCheckpointId(), rootPageId);
 
@@ -410,7 +410,7 @@ public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableSto
             }
 
             return new GcQueue(
-                    tableView.tableId(),
+                    tableView.id(),
                     tableView.name(),
                     partitionId,
                     dataRegion.pageMemory(),
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java
index 233942bfef..b864e59e2b 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java
@@ -87,8 +87,12 @@ public class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStora
         );
     }
 
+    private static int tableId(TableView tableView) {
+        return tableView.id();
+    }
+
     private IndexMetaTree createIndexMetaTree(int partitionId, TableView tableCfgView) {
-        int grpId = tableCfgView.tableId();
+        int grpId = tableId(tableCfgView);
 
         long metaPageId = dataRegion.pageMemory().allocatePage(grpId, partitionId, FLAG_AUX);
 
@@ -110,7 +114,7 @@ public class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStora
     }
 
     private GcQueue createGarbageCollectionTree(int partitionId, TableView tableCfgView) {
-        int grpId = tableCfgView.tableId();
+        int grpId = tableId(tableCfgView);
 
         long metaPageId = dataRegion.pageMemory().allocatePage(grpId, partitionId, FLAG_AUX);
 
@@ -149,7 +153,7 @@ public class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStora
      * @throws StorageException If failed.
      */
     private VersionChainTree createVersionChainTree(int partId, TableView tableView) throws StorageException {
-        int grpId = tableView.tableId();
+        int grpId = tableId(tableView);
 
         try {
             long metaPageId = dataRegion.pageMemory().allocatePage(grpId, partId, FLAG_AUX);
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
index 6a181a5374..c1dd67cb79 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
@@ -167,7 +167,7 @@ public abstract class AbstractPageMemoryMvPartitionStorage implements MvPartitio
 
         PageMemory pageMemory = tableStorage.dataRegion().pageMemory();
 
-        groupId = tableStorage.configuration().value().tableId();
+        groupId = tableStorage.configuration().value().id();
 
         rowVersionDataPageReader = new DataPageReader(pageMemory, groupId, IoStatisticsHolderNoOp.INSTANCE);
     }
@@ -515,7 +515,7 @@ public abstract class AbstractPageMemoryMvPartitionStorage implements MvPartitio
         assert rowVersion.isUncommitted();
 
         UUID transactionId = chain.transactionId();
-        UUID commitTableId = chain.commitTableId();
+        int commitTableId = chain.commitTableId();
         int commitPartitionId = chain.commitPartitionId();
 
         BinaryRow row = rowVersionToBinaryRow(rowVersion);
@@ -544,7 +544,7 @@ public abstract class AbstractPageMemoryMvPartitionStorage implements MvPartitio
     }
 
     @Override
-    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId)
+    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId, int commitPartitionId)
             throws TxIdMismatchException, StorageException {
         assert rowId.partitionId() == partitionId : rowId;
 
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteInvokeClosure.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteInvokeClosure.java
index 74f81301c5..e2fd214319 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteInvokeClosure.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AddWriteInvokeClosure.java
@@ -35,7 +35,7 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Implementation of {@link InvokeClosure} for {@link AbstractPageMemoryMvPartitionStorage#addWrite(RowId, BinaryRow, UUID, UUID, int)}.
+ * Implementation of {@link InvokeClosure} for {@link AbstractPageMemoryMvPartitionStorage#addWrite(RowId, BinaryRow, UUID, int, int)}.
  *
  * <p>See {@link AbstractPageMemoryMvPartitionStorage} about synchronization.
  *
@@ -49,7 +49,7 @@ class AddWriteInvokeClosure implements InvokeClosure<VersionChain> {
 
     private final UUID txId;
 
-    private final UUID commitTableId;
+    private final int commitTableId;
 
     private final int commitPartitionId;
 
@@ -65,7 +65,7 @@ class AddWriteInvokeClosure implements InvokeClosure<VersionChain> {
             RowId rowId,
             @Nullable BinaryRow row,
             UUID txId,
-            UUID commitTableId,
+            int commitTableId,
             int commitPartitionId,
             AbstractPageMemoryMvPartitionStorage storage
     ) {
@@ -118,7 +118,7 @@ class AddWriteInvokeClosure implements InvokeClosure<VersionChain> {
     }
 
     /**
-     * Returns the result for {@link MvPartitionStorage#addWrite(RowId, BinaryRow, UUID, UUID, int)}.
+     * Returns the result for {@link MvPartitionStorage#addWrite(RowId, BinaryRow, UUID, int, int)}.
      */
     @Nullable BinaryRow getPreviousUncommittedRowVersion() {
         return previousUncommittedRowVersion;
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java
index 8fa5d1b058..a90926c6a6 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java
@@ -131,7 +131,7 @@ public class PersistentPageMemoryMvPartitionStorage extends AbstractPageMemoryMv
         blobStorage = new BlobStorage(
                 rowVersionFreeList,
                 dataRegion.pageMemory(),
-                tableStorage.configuration().value().tableId(),
+                tableStorage.configuration().value().id(),
                 partitionId,
                 IoStatisticsHolderNoOp.INSTANCE
         );
@@ -395,7 +395,7 @@ public class PersistentPageMemoryMvPartitionStorage extends AbstractPageMemoryMv
         this.blobStorage = new BlobStorage(
                 rowVersionFreeList,
                 tableStorage.dataRegion().pageMemory(),
-                tableStorage.configuration().tableId().value(),
+                tableStorage.configuration().id().value(),
                 partitionId,
                 IoStatisticsHolderNoOp.INSTANCE
         );
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChain.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChain.java
index bd497f8aee..3ca1b03169 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChain.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChain.java
@@ -44,7 +44,7 @@ public class VersionChain extends VersionChainKey {
     private final @Nullable UUID transactionId;
 
     /** Commit table id (part of transaction state). */
-    private final @Nullable UUID commitTableId;
+    private final @Nullable Integer commitTableId;
 
     /** Commit partition id (part of transaction state). */
     private final int commitPartitionId;
@@ -52,7 +52,7 @@ public class VersionChain extends VersionChainKey {
     /**
      * Constructor.
      */
-    private VersionChain(RowId rowId, @Nullable UUID transactionId, @Nullable UUID commitTableId, int commitPartitionId, long headLink,
+    private VersionChain(RowId rowId, @Nullable UUID transactionId, @Nullable Integer commitTableId, int commitPartitionId, long headLink,
             long nextLink) {
         super(rowId);
         this.transactionId = transactionId;
@@ -66,7 +66,7 @@ public class VersionChain extends VersionChainKey {
         return new VersionChain(rowId, null, null, ReadResult.UNDEFINED_COMMIT_PARTITION_ID, headLink, nextLink);
     }
 
-    public static VersionChain createUncommitted(RowId rowId, UUID transactionId, UUID commitTableId, int commitPartitionId, long headLink,
+    public static VersionChain createUncommitted(RowId rowId, UUID transactionId, int commitTableId, int commitPartitionId, long headLink,
             long nextLink) {
         return new VersionChain(rowId, transactionId, commitTableId, commitPartitionId, headLink, nextLink);
     }
@@ -81,7 +81,7 @@ public class VersionChain extends VersionChainKey {
     /**
      * Returns a commit table id, associated with a chain's head, or {@code null} if head is already committed.
      */
-    public @Nullable UUID commitTableId() {
+    public @Nullable Integer commitTableId() {
         return commitTableId;
     }
 
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/VersionChainIo.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/VersionChainIo.java
index 7c8d6ddf82..04317befe6 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/VersionChainIo.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/VersionChainIo.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.storage.pagememory.mv.io;
 
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.getShort;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putInt;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.putLong;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.putShort;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.PARTITIONLESS_LINK_SIZE_BYTES;
@@ -54,14 +56,11 @@ public interface VersionChainIo {
     /** Offset of txId's least significant bits, 8 bytes. */
     int TX_ID_LSB_OFFSET = TX_ID_MSB_OFFSET + Long.BYTES;
 
-    /** Offset of commit table id's most significant bits. */
-    int COMMIT_TABLE_ID_MSB_OFFSET = TX_ID_LSB_OFFSET + Long.BYTES;
-
-    /** Offset of commit table id's least significant bits. */
-    int COMMIT_TABLE_ID_LSB_OFFSET = COMMIT_TABLE_ID_MSB_OFFSET + Long.BYTES;
+    /** Offset of commit table id. */
+    int COMMIT_TABLE_ID = TX_ID_LSB_OFFSET + Long.BYTES;
 
     /** Offset of commit partition id. */
-    int COMMIT_PARTITION_ID_OFFSET = COMMIT_TABLE_ID_LSB_OFFSET + Long.BYTES;
+    int COMMIT_PARTITION_ID_OFFSET = COMMIT_TABLE_ID + Integer.BYTES;
 
     /** Offset of partitionless head link, 6 bytes. */
     int HEAD_LINK_OFFSET = COMMIT_PARTITION_ID_OFFSET + Short.BYTES;
@@ -107,7 +106,7 @@ public interface VersionChainIo {
         putLong(pageAddr, off + ROW_ID_LSB_OFFSET, rowId.leastSignificantBits());
 
         UUID txId = row.transactionId();
-        UUID commitTableId = row.commitTableId();
+        Integer commitTableId = row.commitTableId();
         int commitPartitionId = row.commitPartitionId();
 
         if (txId == null) {
@@ -123,8 +122,7 @@ public interface VersionChainIo {
             putLong(pageAddr, off + TX_ID_MSB_OFFSET, txId.getMostSignificantBits());
             putLong(pageAddr, off + TX_ID_LSB_OFFSET, txId.getLeastSignificantBits());
 
-            putLong(pageAddr, off + COMMIT_TABLE_ID_MSB_OFFSET, commitTableId.getMostSignificantBits());
-            putLong(pageAddr, off + COMMIT_TABLE_ID_LSB_OFFSET, commitTableId.getLeastSignificantBits());
+            putInt(pageAddr, off + COMMIT_TABLE_ID, commitTableId);
             putShort(pageAddr, off + COMMIT_PARTITION_ID_OFFSET, (short) commitPartitionId);
         }
 
@@ -180,10 +178,7 @@ public interface VersionChainIo {
         long nextLink = readPartitionless(partitionId, pageAddr, offset + NEXT_LINK_OFFSET);
 
         if (txId != null) {
-            long commitTblIdMsb = getLong(pageAddr, offset + COMMIT_TABLE_ID_MSB_OFFSET);
-            long commitTblIdLsb = getLong(pageAddr, offset + COMMIT_TABLE_ID_LSB_OFFSET);
-
-            UUID commitTableId = new UUID(commitTblIdMsb, commitTblIdLsb);
+            int commitTableId = getInt(pageAddr, offset + COMMIT_TABLE_ID);
 
             int commitPartitionId = getShort(pageAddr, offset + COMMIT_PARTITION_ID_OFFSET) & 0xFFFF;
 
diff --git a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/PartitionDataHelper.java b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/PartitionDataHelper.java
index af8fb1408b..2570962a3f 100644
--- a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/PartitionDataHelper.java
+++ b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/PartitionDataHelper.java
@@ -59,7 +59,7 @@ public class PartitionDataHelper implements ManuallyCloseable {
     private static final int TX_ID_SIZE = 2 * Long.BYTES;
 
     /** Commit table id size (part of the transaction state). */
-    private static final int TABLE_ID_SIZE = 2 * Long.BYTES;
+    private static final int TABLE_ID_SIZE = Integer.BYTES;
 
     /** Size of the value header (transaction state). */
     static final int VALUE_HEADER_SIZE = TX_ID_SIZE + TABLE_ID_SIZE + PARTITION_ID_SIZE;
diff --git a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorage.java b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorage.java
index 28a20553c6..737628194b 100644
--- a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorage.java
+++ b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorage.java
@@ -44,6 +44,8 @@ import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
 import static org.apache.ignite.internal.util.ByteUtils.bytesToUuid;
 import static org.apache.ignite.internal.util.ByteUtils.longToBytes;
 import static org.apache.ignite.internal.util.ByteUtils.putUuidToBytes;
+import static org.apache.ignite.internal.util.GridUnsafe.getInt;
+import static org.apache.ignite.internal.util.GridUnsafe.getShort;
 import static org.rocksdb.ReadTier.PERSISTED_TIER;
 
 import java.nio.ByteBuffer;
@@ -401,7 +403,7 @@ public class RocksDbMvPartitionStorage implements MvPartitionStorage {
     }
 
     @Override
-    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId)
+    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId, int commitPartitionId)
             throws TxIdMismatchException, StorageException {
         return busy(() -> {
             @SuppressWarnings("resource") WriteBatchWithIndex writeBatch = PartitionDataHelper.requireWriteBatch();
@@ -436,8 +438,7 @@ public class RocksDbMvPartitionStorage implements MvPartitionStorage {
                         byte[] valueHeaderBytes = new byte[VALUE_HEADER_SIZE];
 
                         putUuidToBytes(txId, valueHeaderBytes, TX_ID_OFFSET);
-                        putUuidToBytes(commitTableId, valueHeaderBytes, TABLE_ID_OFFSET);
-                        putShort(valueHeaderBytes, PARTITION_ID_OFFSET, (short) commitPartitionId);
+                        putInt(valueHeaderBytes, TABLE_ID_OFFSET, commitTableId);
 
                         writeBatch.put(helper.partCf, keyBytes, valueHeaderBytes);
                     }
@@ -460,7 +461,7 @@ public class RocksDbMvPartitionStorage implements MvPartitionStorage {
      * @param txId Transaction id.
      * @throws RocksDBException If write failed.
      */
-    private void writeUnversioned(byte[] keyArray, BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId)
+    private void writeUnversioned(byte[] keyArray, BinaryRow row, UUID txId, int commitTableId, int commitPartitionId)
             throws RocksDBException {
         @SuppressWarnings("resource") WriteBatchWithIndex writeBatch = PartitionDataHelper.requireWriteBatch();
 
@@ -470,7 +471,7 @@ public class RocksDbMvPartitionStorage implements MvPartitionStorage {
         byte[] array = value.array();
 
         putUuidToBytes(txId, array, TX_ID_OFFSET);
-        putUuidToBytes(commitTableId, array, TABLE_ID_OFFSET);
+        putInt(array, TABLE_ID_OFFSET, commitTableId);
         putShort(array, PARTITION_ID_OFFSET, (short) commitPartitionId);
 
         value.position(VALUE_OFFSET).put(rowBytes);
@@ -1065,6 +1066,10 @@ public class RocksDbMvPartitionStorage implements MvPartitionStorage {
         GridUnsafe.putShort(array, GridUnsafe.BYTE_ARR_OFF + off, value);
     }
 
+    private static void putInt(byte[] array, int off, int value) {
+        GridUnsafe.putInt(array, GridUnsafe.BYTE_ARR_OFF + off, value);
+    }
+
     private static void validateTxId(byte[] valueBytes, UUID txId) {
         long msb = bytesToLong(valueBytes);
         long lsb = bytesToLong(valueBytes, Long.BYTES);
@@ -1121,8 +1126,8 @@ public class RocksDbMvPartitionStorage implements MvPartitionStorage {
      */
     private static ReadResult wrapUncommittedValue(RowId rowId, byte[] valueBytes, @Nullable HybridTimestamp newestCommitTs) {
         UUID txId = bytesToUuid(valueBytes, TX_ID_OFFSET);
-        UUID commitTableId = bytesToUuid(valueBytes, TABLE_ID_OFFSET);
-        int commitPartitionId = GridUnsafe.getShort(valueBytes, GridUnsafe.BYTE_ARR_OFF + PARTITION_ID_OFFSET) & 0xFFFF;
+        int commitTableId = getInt(valueBytes, GridUnsafe.BYTE_ARR_OFF + TABLE_ID_OFFSET);
+        int commitPartitionId = getShort(valueBytes, GridUnsafe.BYTE_ARR_OFF + PARTITION_ID_OFFSET) & 0xFFFF;
 
         BinaryRow row;
 
diff --git a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
index ddcf45901f..322a4ac918 100644
--- a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
+++ b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
@@ -170,7 +170,7 @@ public class RocksDbStorageEngine implements StorageEngine {
 
         RocksDbDataRegion dataRegion = regions.get(dataStorageView.dataRegion());
 
-        Path tablePath = storagePath.resolve(TABLE_DIR_PREFIX + tableCfg.tableId().value());
+        Path tablePath = storagePath.resolve(TABLE_DIR_PREFIX + tableCfg.id().value());
 
         try {
             Files.createDirectories(tablePath);
diff --git a/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvTableStorageTest.java b/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvTableStorageTest.java
index fc16dc69f6..d8e34f6fce 100644
--- a/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvTableStorageTest.java
+++ b/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvTableStorageTest.java
@@ -79,7 +79,7 @@ public class RocksDbMvTableStorageTest extends AbstractMvTableStorageTest {
         partitionStorage0.runConsistently(locker -> {
             locker.lock(rowId0);
 
-            return partitionStorage0.addWrite(rowId0, testData, txId, UUID.randomUUID(), 0);
+            return partitionStorage0.addWrite(rowId0, testData, txId, COMMIT_TABLE_ID, 0);
         });
 
         MvPartitionStorage partitionStorage1 = getOrCreateMvPartition(PARTITION_ID_1);
@@ -89,7 +89,7 @@ public class RocksDbMvTableStorageTest extends AbstractMvTableStorageTest {
         partitionStorage1.runConsistently(locker -> {
             locker.lock(rowId1);
 
-            return partitionStorage1.addWrite(rowId1, testData, txId, UUID.randomUUID(), 0);
+            return partitionStorage1.addWrite(rowId1, testData, txId, COMMIT_TABLE_ID, 0);
         });
 
         tableStorage.destroyPartition(PARTITION_ID_0).get(1, TimeUnit.SECONDS);
@@ -120,7 +120,7 @@ public class RocksDbMvTableStorageTest extends AbstractMvTableStorageTest {
         partitionStorage0.runConsistently(locker -> {
             locker.lock(rowId0);
 
-            return partitionStorage0.addWrite(rowId0, testData, txId, UUID.randomUUID(), 0);
+            return partitionStorage0.addWrite(rowId0, testData, txId, COMMIT_TABLE_ID, 0);
         });
 
         tableStorage.stop();
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
index c8006360a4..14906572da 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
@@ -35,7 +35,6 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BooleanSupplier;
@@ -194,7 +193,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
         table = new InternalTableImpl(
                 "table",
-                UUID.randomUUID(),
+                1,
                 Int2ObjectMaps.singleton(0, service),
                 1,
                 consistentIdToNode,
@@ -235,7 +234,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
 
                 UpdateCommand cmd = msgFactory.updateCommand()
                         .txId(req0.transactionId())
-                        .tablePartitionId(tablePartitionId(new TablePartitionId(UUID.randomUUID(), 0)))
+                        .tablePartitionId(tablePartitionId(new TablePartitionId(1, 0)))
                         .rowUuid(new RowId(0).uuid())
                         .rowBuffer(binaryRow == null ? null : binaryRow.byteBuffer())
                         .safeTimeLong(hybridClock.nowLong())
@@ -249,7 +248,7 @@ public class ItTablePersistenceTest extends ItAbstractListenerSnapshotTest<Parti
                         .txId(req0.txId())
                         .commit(req0.commit())
                         .commitTimestampLong(req0.commitTimestampLong())
-                        .tablePartitionIds(asList(tablePartitionId(new TablePartitionId(UUID.randomUUID(), 0))))
+                        .tablePartitionIds(asList(tablePartitionId(new TablePartitionId(1, 0))))
                         .safeTimeLong(hybridClock.nowLong())
                         .build();
 
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
index 4f467a8283..0712a827d3 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
@@ -353,8 +353,8 @@ public class ItTxDistributedTestSingleNode extends TxAbstractTest {
         final String accountsName = "accounts";
         final String customersName = "customers";
 
-        UUID accTblId = UUID.randomUUID();
-        UUID custTblId = UUID.randomUUID();
+        int accTblId = 1;
+        int custTblId = 2;
 
         accRaftClients = startTable(accTblId, ACCOUNTS_SCHEMA);
         custRaftClients = startTable(custTblId, CUSTOMERS_SCHEMA);
@@ -410,7 +410,7 @@ public class ItTxDistributedTestSingleNode extends TxAbstractTest {
      * @param schemaDescriptor Schema descriptor.
      * @return Groups map.
      */
-    private Int2ObjectOpenHashMap<RaftGroupService> startTable(UUID tblId, SchemaDescriptor schemaDescriptor) throws Exception {
+    private Int2ObjectOpenHashMap<RaftGroupService> startTable(int tblId, SchemaDescriptor schemaDescriptor) throws Exception {
         List<Set<Assignment>> calculatedAssignments = AffinityUtils.calculateAssignments(
                 cluster.stream().map(node -> node.topologyService().localMember().name()).collect(toList()),
                 1,
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ReplicaUnavailableTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ReplicaUnavailableTest.java
index 98b5fe10c6..fe4b881974 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ReplicaUnavailableTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ReplicaUnavailableTest.java
@@ -30,7 +30,6 @@ import static org.mockito.Mockito.when;
 
 import java.util.List;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
@@ -132,7 +131,7 @@ public class ReplicaUnavailableTest extends IgniteAbstractTest {
     public void testWithReplicaStartedAfterRequestSending() throws Exception {
         ClusterNode clusterNode = clusterService.topologyService().localMember();
 
-        TablePartitionId tablePartitionId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId tablePartitionId = new TablePartitionId(1, 1);
 
         ReadWriteSingleRowReplicaRequest request = tableMessagesFactory.readWriteSingleRowReplicaRequest()
                 .groupId(tablePartitionId)
@@ -172,7 +171,7 @@ public class ReplicaUnavailableTest extends IgniteAbstractTest {
     public void testWithNotStartedReplica() {
         ClusterNode clusterNode = clusterService.topologyService().localMember();
 
-        TablePartitionId tablePartitionId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId tablePartitionId = new TablePartitionId(1, 1);
 
         ReadWriteSingleRowReplicaRequest request = tableMessagesFactory.readWriteSingleRowReplicaRequest()
                 .groupId(tablePartitionId)
diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItColocationTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItColocationTest.java
index 14945612f8..879b357fd2 100644
--- a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItColocationTest.java
+++ b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/ItColocationTest.java
@@ -158,7 +158,7 @@ public class ItColocationTest {
         Int2ObjectMap<RaftGroupService> partRafts = new Int2ObjectOpenHashMap<>();
         Map<ReplicationGroupId, RaftGroupService> groupRafts = new HashMap<>();
 
-        UUID tblId = UUID.randomUUID();
+        int tblId = 1;
 
         for (int i = 0; i < PARTS; ++i) {
             TablePartitionId groupId = new TablePartitionId(tblId, i);
@@ -191,7 +191,7 @@ public class ItColocationTest {
 
         when(replicaService.invoke(any(ClusterNode.class), any())).thenAnswer(invocation -> {
             ReplicaRequest request = invocation.getArgument(1);
-            var commitPartId = new TablePartitionId(UUID.randomUUID(), 0);
+            var commitPartId = new TablePartitionId(2, 0);
 
             RaftGroupService r = groupRafts.get(request.groupId());
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/IgniteTablesInternal.java b/modules/table/src/main/java/org/apache/ignite/internal/table/IgniteTablesInternal.java
index cc44042e72..40486703aa 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/IgniteTablesInternal.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/IgniteTablesInternal.java
@@ -37,7 +37,7 @@ public interface IgniteTablesInternal extends IgniteTables {
      * @return Table or {@code null} when not exists.
      * @throws NodeStoppingException If an implementation stopped before the method was invoked.
      */
-    TableImpl table(UUID id) throws NodeStoppingException;
+    TableImpl table(int id) throws NodeStoppingException;
 
     /**
      * Gets a table future by id. If the table exists, the future will point to it, otherwise to {@code null}.
@@ -46,7 +46,7 @@ public interface IgniteTablesInternal extends IgniteTables {
      * @return Future representing pending completion of the operation.
      * @throws NodeStoppingException If an implementation stopped before the method was invoked.
      */
-    CompletableFuture<TableImpl> tableAsync(UUID id) throws NodeStoppingException;
+    CompletableFuture<TableImpl> tableAsync(int id) throws NodeStoppingException;
 
     // TODO: IGNITE-16750 - the following two methods look a bit ugly, separation of public/internal Table aspects should help
 
@@ -88,7 +88,7 @@ public interface IgniteTablesInternal extends IgniteTables {
      * @return List of the current assignments.
      */
     @Nullable
-    List<String> assignments(UUID tableId) throws NodeStoppingException;
+    List<String> assignments(int tableId) throws NodeStoppingException;
 
     /**
      * Adds a listener to track changes in {@link #assignments(UUID)}.
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
index 85fb0a8733..86c2676afe 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
@@ -58,7 +58,7 @@ public interface InternalTable extends ManuallyCloseable {
      *
      * @return Table id as UUID.
      */
-    UUID tableId();
+    int tableId();
 
     /**
      * Gets a name of the table.
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
index c4c7edd68d..36b8933810 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
@@ -106,7 +106,7 @@ public class TableImpl implements Table {
      *
      * @return Table id as UUID.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tbl.tableId();
     }
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java
index b5d1424242..10754bff31 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandler.java
@@ -114,7 +114,7 @@ public class StorageUpdateHandler {
         storage.runConsistently(locker -> {
             BinaryRow row = rowBuffer != null ? new ByteBufferRow(rowBuffer) : null;
             RowId rowId = new RowId(partitionId, rowUuid);
-            UUID commitTblId = commitPartitionId.tableId();
+            int commitTblId = commitPartitionId.tableId();
             int commitPartId = commitPartitionId.partitionId();
 
             locker.lock(rowId);
@@ -153,7 +153,7 @@ public class StorageUpdateHandler {
             @Nullable Consumer<Collection<RowId>> onReplication
     ) {
         storage.runConsistently(locker -> {
-            UUID commitTblId = commitPartitionId.tableId();
+            int commitTblId = commitPartitionId.tableId();
             int commitPartId = commitPartitionId.partitionId();
 
             if (!nullOrEmpty(rowsToUpdate)) {
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
index ac4a81c238..310c12c14d 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
@@ -73,6 +73,7 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.ignite.configuration.ConfigurationChangeException;
 import org.apache.ignite.configuration.ConfigurationProperty;
+import org.apache.ignite.configuration.NamedConfigurationTree;
 import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
 import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
@@ -246,16 +247,16 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private final PlacementDriver placementDriver;
 
     /** Here a table future stores during creation (until the table can be provided to client). */
-    private final Map<UUID, CompletableFuture<Table>> tableCreateFuts = new ConcurrentHashMap<>();
+    private final Map<Integer, CompletableFuture<Table>> tableCreateFuts = new ConcurrentHashMap<>();
 
     /** Versioned store for tables by id. */
-    private final IncrementalVersionedValue<Map<UUID, TableImpl>> tablesByIdVv;
+    private final IncrementalVersionedValue<Map<Integer, TableImpl>> tablesByIdVv;
 
     /**
      * {@link TableImpl} is created during update of tablesByIdVv, we store reference to it in case of updating of tablesByIdVv fails, so we
      * can stop resources associated with the table or to clean up table resources on {@code TableManager#stop()}.
      */
-    private final Map<UUID, TableImpl> pendingTables = new ConcurrentHashMap<>();
+    private final Map<Integer, TableImpl> pendingTables = new ConcurrentHashMap<>();
 
     /** Resolver that resolves a node consistent ID to cluster node. */
     private final Function<String, ClusterNode> clusterNodeResolver;
@@ -498,7 +499,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
                 HasDataRequest msg = (HasDataRequest) message;
 
-                UUID tableId = msg.tableId();
+                int tableId = msg.tableId();
                 int partitionId = msg.partitionId();
 
                 boolean contains = false;
@@ -532,7 +533,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      */
     private CompletableFuture<?> onTableCreate(ConfigurationNotificationEvent<TableView> ctx) {
         if (!busyLock.enterBusy()) {
-            UUID tblId = ((ExtendedTableView) ctx.newValue()).id();
+            int tblId = ctx.newValue().id();
 
             fireEvent(TableEvent.CREATE,
                     new TableEventParameters(ctx.storageRevision(), tblId),
@@ -546,7 +547,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
             return createTableLocally(
                     ctx.storageRevision(),
                     ctx.newValue().name(),
-                    ((ExtendedTableView) ctx.newValue()).id()
+                    ctx.newValue().id()
             );
         } finally {
             busyLock.leaveBusy();
@@ -561,7 +562,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      */
     private CompletableFuture<?> onTableDelete(ConfigurationNotificationEvent<TableView> ctx) {
         if (!busyLock.enterBusy()) {
-            UUID tblId = ((ExtendedTableView) ctx.oldValue()).id();
+            int tblId = ctx.oldValue().id();
 
             fireEvent(
                     TableEvent.DROP,
@@ -576,8 +577,8 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
             dropTableLocally(
                     ctx.storageRevision(),
                     ctx.oldValue().name(),
-                    ((ExtendedTableView) ctx.oldValue()).id(),
-                    ByteUtils.fromBytes(((ExtendedTableView) ctx.oldValue()).assignments())
+                    ctx.oldValue().id(),
+                    ByteUtils.fromBytes(ctx.oldValue(ExtendedTableView.class).assignments())
             );
         } finally {
             busyLock.leaveBusy();
@@ -619,7 +620,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
     private CompletableFuture<?> updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> assignmentsCtx) {
         ExtendedTableView tblCfg = assignmentsCtx.newValue(ExtendedTableView.class);
 
-        UUID tblId = tblCfg.id();
+        int tblId = tblCfg.id();
 
         DistributionZoneConfiguration dstCfg = getZoneById(distributionZonesConfiguration, tblCfg.zoneId());
 
@@ -885,7 +886,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @param peers Raft peers.
      * @return A future that will hold the quantity of data nodes.
      */
-    private CompletableFuture<Long> queryDataNodesCount(UUID tblId, int partId, Collection<Peer> peers) {
+    private CompletableFuture<Long> queryDataNodesCount(int tblId, int partId, Collection<Peer> peers) {
         HasDataRequest request = TABLE_MESSAGES_FACTORY.hasDataRequest().tableId(tblId).partitionId(partId).build();
 
         //noinspection unchecked
@@ -953,7 +954,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
         metaStorageMgr.unregisterWatch(stableAssignmentsRebalanceListener);
         metaStorageMgr.unregisterWatch(assignmentsSwitchRebalanceListener);
 
-        Map<UUID, TableImpl> tablesToStop = Stream.concat(tablesByIdVv.latest().entrySet().stream(), pendingTables.entrySet().stream())
+        Map<Integer, TableImpl> tablesToStop = Stream.concat(tablesByIdVv.latest().entrySet().stream(), pendingTables.entrySet().stream())
                 .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (v1, v2) -> v1));
 
         cleanUpTablesResources(tablesToStop);
@@ -977,7 +978,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      *
      * @param tables Tables to stop.
      */
-    private void cleanUpTablesResources(Map<UUID, TableImpl> tables) {
+    private void cleanUpTablesResources(Map<Integer, TableImpl> tables) {
         for (TableImpl table : tables.values()) {
             table.beforeClose();
 
@@ -1050,7 +1051,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override
-    public List<String> assignments(UUID tableId) throws NodeStoppingException {
+    public List<String> assignments(int tableId) throws NodeStoppingException {
         if (!busyLock.enterBusy()) {
             throw new NodeStoppingException();
         }
@@ -1091,7 +1092,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @param tblId Table id.
      * @return Future that will be completed when local changes related to the table creation are applied.
      */
-    private CompletableFuture<?> createTableLocally(long causalityToken, String name, UUID tblId) {
+    private CompletableFuture<?> createTableLocally(long causalityToken, String name, int tblId) {
         LOG.trace("Creating local table: name={}, id={}, token={}", name, tblId, causalityToken);
 
         TableConfiguration tableCfg = tablesCfg.tables().get(name);
@@ -1168,7 +1169,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      */
     protected TxStateTableStorage createTxStateTableStorage(
             TableConfiguration tableCfg, DistributionZoneConfiguration distributionZoneCfg) {
-        Path path = storagePath.resolve(TX_STATE_DIR + tableCfg.value().tableId());
+        Path path = storagePath.resolve(TX_STATE_DIR + tableCfg.value().id());
 
         try {
             Files.createDirectories(path);
@@ -1215,7 +1216,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @param tblId Table id.
      * @param assignment Affinity assignment.
      */
-    private void dropTableLocally(long causalityToken, String name, UUID tblId, List<Set<ClusterNode>> assignment) {
+    private void dropTableLocally(long causalityToken, String name, int tblId, List<Set<ClusterNode>> assignment) {
         try {
             int partitions = assignment.size();
 
@@ -1430,10 +1431,11 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
                 var extConfCh = ((ExtendedTableChange) tableChange);
 
-                int intTableId = tablesChange.globalIdCounter() + 1;
-                tablesChange.changeGlobalIdCounter(intTableId);
+                int tableId = tablesChange.globalIdCounter() + 1;
 
-                extConfCh.changeTableId(intTableId);
+                extConfCh.changeId(tableId);
+
+                tablesChange.changeGlobalIdCounter(tableId);
 
                 extConfCh.changeSchemaId(INITIAL_SCHEMA_VERSION);
 
@@ -1603,7 +1605,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                             })
                             .changeIndexes(idxChg -> {
                                 for (TableIndexView index : idxChg) {
-                                    if (index.tableId().equals(tbl.tableId())) {
+                                    if (index.tableId() == tbl.tableId()) {
                                         idxChg.delete(index.name());
                                     }
                                 }
@@ -1652,7 +1654,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
                     var i = 0;
 
-                    for (UUID tblId : tableIds) {
+                    for (int tblId : tableIds) {
                         tableFuts[i++] = tableAsyncInternal(tblId, false);
                     }
 
@@ -1677,8 +1679,10 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      *
      * @return A list of direct table ids.
      */
-    private List<UUID> directTableIds() {
-        return directProxy(tablesCfg.tables()).internalIds();
+    private List<Integer> directTableIds() {
+        return directProxy(tablesCfg.tables()).value().stream()
+                .map(TableView::id)
+                .collect(toList());
     }
 
     /**
@@ -1697,9 +1701,9 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @return Direct id of the table, or {@code null} if the table with the {@code tblName} has not been found.
      */
     @Nullable
-    private UUID directTableId(String tblName) {
+    private Integer directTableId(String tblName) {
         try {
-            ExtendedTableConfiguration exTblCfg = ((ExtendedTableConfiguration) directProxy(tablesCfg.tables()).get(tblName));
+            TableConfiguration exTblCfg = directProxy(tablesCfg.tables()).get(tblName);
 
             if (exTblCfg == null) {
                 return null;
@@ -1717,7 +1721,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @return Actual tables map.
      */
     @TestOnly
-    public Map<UUID, TableImpl> latestTables() {
+    public Map<Integer, TableImpl> latestTables() {
         return unmodifiableMap(tablesByIdVv.latest());
     }
 
@@ -1729,7 +1733,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override
-    public TableImpl table(UUID id) throws NodeStoppingException {
+    public TableImpl table(int id) throws NodeStoppingException {
         return join(tableAsync(id));
     }
 
@@ -1747,7 +1751,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @param id Table id.
      * @return Future.
      */
-    public CompletableFuture<TableImpl> tableAsync(long causalityToken, UUID id) {
+    public CompletableFuture<TableImpl> tableAsync(long causalityToken, int id) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
@@ -1760,7 +1764,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
     /** {@inheritDoc} */
     @Override
-    public CompletableFuture<TableImpl> tableAsync(UUID id) {
+    public CompletableFuture<TableImpl> tableAsync(int id) {
         if (!busyLock.enterBusy()) {
             throw new IgniteException(new NodeStoppingException());
         }
@@ -1817,7 +1821,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      *         otherwise.
      * @return Future representing pending completion of the operation.
      */
-    public CompletableFuture<TableImpl> tableAsyncInternal(UUID id, boolean checkConfiguration) {
+    public CompletableFuture<TableImpl> tableAsyncInternal(int id, boolean checkConfiguration) {
         CompletableFuture<Boolean> tblCfgFut = checkConfiguration
                 // TODO: IGNITE-16288 isTableConfigured should use async configuration API
                 ? supplyAsync(() -> inBusyLock(busyLock, () -> isTableConfigured(id)), ioExecutor)
@@ -1828,7 +1832,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                 return completedFuture(null);
             }
 
-            var tbl = tablesByIdVv.latest().get(id);
+            TableImpl tbl = tablesByIdVv.latest().get(id);
 
             if (tbl != null) {
                 return completedFuture(tbl);
@@ -1836,7 +1840,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
 
             CompletableFuture<TableImpl> getTblFut = new CompletableFuture<>();
 
-            CompletionListener<Map<UUID, TableImpl>> tablesListener = (token, tables, th) -> {
+            CompletionListener<Map<Integer, TableImpl>> tablesListener = (token, tables, th) -> {
                 if (th == null) {
                     TableImpl table = tables.get(id);
 
@@ -1870,14 +1874,16 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
      * @param id Table id.
      * @return True when the table is configured into cluster, false otherwise.
      */
-    private boolean isTableConfigured(UUID id) {
-        try {
-            ((ExtendedTableConfiguration) directProxy(tablesCfg.tables()).get(id)).id().value();
+    private boolean isTableConfigured(int id) {
+        NamedConfigurationTree<TableConfiguration, TableView, TableChange> tables = directProxy(tablesCfg.tables());
 
-            return true;
-        } catch (NoSuchElementException e) {
-            return false;
+        for (TableView tableConfig : tables.value()) {
+            if (tableConfig.id() == id) {
+                return true;
+            }
         }
+
+        return false;
     }
 
     /**
@@ -1951,7 +1957,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
         }
 
         int partId = extractPartitionNumber(pendingAssignmentsWatchEntry.key());
-        UUID tblId = extractTableId(pendingAssignmentsWatchEntry.key(), PENDING_ASSIGNMENTS_PREFIX);
+        int tblId = extractTableId(pendingAssignmentsWatchEntry.key(), PENDING_ASSIGNMENTS_PREFIX);
 
         var replicaGrpId = new TablePartitionId(tblId, partId);
 
@@ -2086,8 +2092,6 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                                     groupOptions
                             );
 
-                            UUID tblId = tbl.tableId();
-
                             replicaMgr.startReplica(
                                     replicaGrpId,
                                     allOf(
@@ -2101,7 +2105,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                                             lockMgr,
                                             scanRequestExecutor,
                                             partId,
-                                            tblId,
+                                            tbl.tableId(),
                                             tbl.indexesLockers(partId),
                                             new Lazy<>(() -> tbl.indexStorageAdapters(partId).get().get(tbl.pkId())),
                                             () -> tbl.indexStorageAdapters(partId).get(),
@@ -2111,7 +2115,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                                             placementDriver,
                                             partitionUpdateHandlers.storageUpdateHandler,
                                             new NonHistoricSchemas(schemaManager),
-                                            completedFuture(schemaManager.schemaRegistry(tblId)),
+                                            completedFuture(schemaManager.schemaRegistry(tbl.tableId())),
                                             localNode(),
                                             internalTable.storage(),
                                             indexBuilder
@@ -2198,7 +2202,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
                     byte[] key = evt.entryEvent().newEntry().key();
 
                     int partitionNumber = extractPartitionNumber(key);
-                    UUID tblId = extractTableId(key, ASSIGNMENTS_SWITCH_REDUCE_PREFIX);
+                    int tblId = extractTableId(key, ASSIGNMENTS_SWITCH_REDUCE_PREFIX);
 
                     TablePartitionId replicaGrpId = new TablePartitionId(tblId, partitionNumber);
 
@@ -2313,7 +2317,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
         }
 
         int partitionId = extractPartitionNumber(stableAssignmentsWatchEvent.key());
-        UUID tableId = extractTableId(stableAssignmentsWatchEvent.key(), STABLE_ASSIGNMENTS_PREFIX);
+        int tableId = extractTableId(stableAssignmentsWatchEvent.key(), STABLE_ASSIGNMENTS_PREFIX);
 
         TablePartitionId tablePartitionId = new TablePartitionId(tableId, partitionId);
 
@@ -2380,7 +2384,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
         }
     }
 
-    private Collection<UUID> collectTableIndexes(UUID tableId) {
+    private Collection<UUID> collectTableIndexes(int tableId) {
         NamedListView<? extends TableIndexView> indexes = tablesCfg.value().indexes();
 
         List<UUID> indexIds = new ArrayList<>();
@@ -2388,7 +2392,7 @@ public class TableManager extends Producer<TableEvent, TableEventParameters> imp
         for (int i = 0; i < indexes.size(); i++) {
             TableIndexView indexConfig = indexes.get(i);
 
-            if (indexConfig.tableId().equals(tableId)) {
+            if (indexConfig.tableId() == tableId) {
                 indexIds.add(indexConfig.id());
             }
         }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TablePartitionIdMessage.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TablePartitionIdMessage.java
index c68eb55730..94497d2182 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TablePartitionIdMessage.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/TablePartitionIdMessage.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.table.distributed.command;
 
 import java.io.Serializable;
-import java.util.UUID;
 import org.apache.ignite.internal.replicator.TablePartitionId;
 import org.apache.ignite.internal.table.distributed.TableMessageGroup.Commands;
 import org.apache.ignite.network.NetworkMessage;
@@ -29,7 +28,7 @@ import org.apache.ignite.network.annotations.Transferable;
  */
 @Transferable(Commands.TABLE_PARTITION_ID)
 public interface TablePartitionIdMessage extends NetworkMessage, Serializable {
-    UUID tableId();
+    int tableId();
 
     int partitionId();
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
index ca55ed314c..333973c26c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
@@ -24,19 +24,19 @@ import org.apache.ignite.internal.tostring.S;
  * {@link IndexBuildTask} ID.
  */
 class IndexBuildTaskId {
-    private final UUID tableId;
+    private final int tableId;
 
     private final int partitionId;
 
     private final UUID indexId;
 
-    IndexBuildTaskId(UUID tableId, int partitionId, UUID indexId) {
+    IndexBuildTaskId(int tableId, int partitionId, UUID indexId) {
         this.tableId = tableId;
         this.partitionId = partitionId;
         this.indexId = indexId;
     }
 
-    public UUID getTableId() {
+    public int getTableId() {
         return tableId;
     }
 
@@ -59,12 +59,12 @@ class IndexBuildTaskId {
 
         IndexBuildTaskId that = (IndexBuildTaskId) o;
 
-        return partitionId == that.partitionId && tableId.equals(that.tableId) && indexId.equals(that.indexId);
+        return partitionId == that.partitionId && tableId == that.tableId && indexId.equals(that.indexId);
     }
 
     @Override
     public int hashCode() {
-        int result = tableId.hashCode();
+        int result = tableId;
         result = 31 * result + partitionId;
         result = 31 * result + indexId.hashCode();
         return result;
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
index be19759424..29c22897c5 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
@@ -87,7 +87,7 @@ public class IndexBuilder implements ManuallyCloseable {
      */
     // TODO: IGNITE-19498 Perhaps we need to start building the index only once
     public void startBuildIndex(
-            UUID tableId,
+            int tableId,
             int partitionId,
             UUID indexId,
             IndexStorage indexStorage,
@@ -123,7 +123,7 @@ public class IndexBuilder implements ManuallyCloseable {
      * @param partitionId Partition ID.
      * @param indexId Index ID.
      */
-    public void stopBuildIndex(UUID tableId, int partitionId, UUID indexId) {
+    public void stopBuildIndex(int tableId, int partitionId, UUID indexId) {
         inBusyLock(() -> {
             IndexBuildTask removed = indexBuildTaskById.remove(new IndexBuildTaskId(tableId, partitionId, indexId));
 
@@ -139,7 +139,7 @@ public class IndexBuilder implements ManuallyCloseable {
      * @param tableId Table ID.
      * @param partitionId Partition ID.
      */
-    public void stopBuildIndexes(UUID tableId, int partitionId) {
+    public void stopBuildIndexes(int tableId, int partitionId) {
         for (Iterator<Entry<IndexBuildTaskId, IndexBuildTask>> it = indexBuildTaskById.entrySet().iterator(); it.hasNext(); ) {
             if (!busyLock.enterBusy()) {
                 return;
@@ -150,7 +150,7 @@ public class IndexBuilder implements ManuallyCloseable {
 
                 IndexBuildTaskId taskId = entry.getKey();
 
-                if (tableId.equals(taskId.getTableId()) && partitionId == taskId.getPartitionId()) {
+                if (tableId == taskId.getTableId() && partitionId == taskId.getPartitionId()) {
                     it.remove();
 
                     entry.getValue().stop();
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/message/HasDataRequest.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/message/HasDataRequest.java
index 139044139b..e5be334932 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/message/HasDataRequest.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/message/HasDataRequest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.table.distributed.message;
 
-import java.util.UUID;
 import org.apache.ignite.internal.table.distributed.TableMessageGroup;
 import org.apache.ignite.network.NetworkMessage;
 import org.apache.ignite.network.annotations.Transferable;
@@ -28,7 +27,7 @@ import org.apache.ignite.network.annotations.Transferable;
 @Transferable(TableMessageGroup.HAS_DATA_REQUEST)
 public interface HasDataRequest extends NetworkMessage {
     /** ID of the table. */
-    UUID tableId();
+    int tableId();
 
     /** ID of the partition. */
     int partitionId();
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionDataStorage.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionDataStorage.java
index 4d2fc32f17..50f5110d1a 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionDataStorage.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionDataStorage.java
@@ -127,9 +127,9 @@ public interface PartitionDataStorage extends ManuallyCloseable {
      *     exists before this call
      * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
      * @throws StorageException If failed to write data to the storage.
-     * @see MvPartitionStorage#addWrite(RowId, BinaryRow, UUID, UUID, int)
+     * @see MvPartitionStorage#addWrite(RowId, BinaryRow, UUID, int, int)
      */
-    @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId)
+    @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId, int commitPartitionId)
             throws TxIdMismatchException, StorageException;
 
     /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccess.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccess.java
index 7514167b1d..5b166af1b0 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccess.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccess.java
@@ -98,7 +98,7 @@ public interface PartitionAccess {
      * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
      * @throws StorageException If failed to write data.
      */
-    void addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId);
+    void addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId, int commitPartitionId);
 
     /**
      * Creates a committed version. In details: - if there is no uncommitted version, a new committed version is added - if there is an
@@ -147,7 +147,7 @@ public interface PartitionAccess {
      *         <li>{@link #maxLastAppliedTerm()};</li>
      *         <li>{@link #committedGroupConfiguration()};</li>
      *         <li>{@link #addTxMeta(UUID, TxMeta)};</li>
-     *         <li>{@link #addWrite(RowId, BinaryRow, UUID, UUID, int)};</li>
+     *         <li>{@link #addWrite(RowId, BinaryRow, UUID, int, int)};</li>
      *         <li>{@link #addWriteCommitted(RowId, BinaryRow, HybridTimestamp)}.</li>
      *     </ul></li>
      * </ul>
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImpl.java
index 6d7cbd3635..6f2683336c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImpl.java
@@ -135,7 +135,7 @@ public class PartitionAccessImpl implements PartitionAccess {
     }
 
     @Override
-    public void addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId, int commitPartitionId) {
+    public void addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId, int commitPartitionId) {
         MvPartitionStorage mvPartitionStorage = getMvPartitionStorage(partitionId());
 
         mvPartitionStorage.runConsistently(locker -> {
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionKey.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionKey.java
index 683fa67fce..65cf82b170 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionKey.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionKey.java
@@ -18,21 +18,20 @@
 package org.apache.ignite.internal.table.distributed.raft.snapshot;
 
 import java.util.Objects;
-import java.util.UUID;
 import org.apache.ignite.internal.tostring.S;
 
 /**
  * Uniquely identifies a partition. This is a pair of internal table ID and partition number (aka partition ID).
  */
 public class PartitionKey {
-    private final UUID tableId;
+    private final int tableId;
 
     private final int partitionId;
 
     /**
      * Returns ID of the table.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
@@ -46,9 +45,7 @@ public class PartitionKey {
     /**
      * Constructs a new partition key.
      */
-    public PartitionKey(UUID tableId, int partitionId) {
-        Objects.requireNonNull(tableId, "tableId cannot be null");
-
+    public PartitionKey(int tableId, int partitionId) {
         this.tableId = tableId;
         this.partitionId = partitionId;
     }
@@ -62,7 +59,7 @@ public class PartitionKey {
             return false;
         }
         PartitionKey that = (PartitionKey) o;
-        return partitionId == that.partitionId && tableId.equals(that.tableId);
+        return partitionId == that.partitionId && tableId == that.tableId;
     }
 
     @Override
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/message/SnapshotMvDataResponse.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/message/SnapshotMvDataResponse.java
index 5294de7efb..523b0f4834 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/message/SnapshotMvDataResponse.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/message/SnapshotMvDataResponse.java
@@ -60,7 +60,7 @@ public interface SnapshotMvDataResponse extends NetworkMessage {
         @Nullable UUID txId();
 
         /** Commit table id for write-intent if it's present. */
-        @Nullable UUID commitTableId();
+        @Nullable Integer commitTableId();
 
         /** Commit partition id for write-intent if it's present. {@link ReadResult#UNDEFINED_COMMIT_PARTITION_ID} otherwise. */
         int commitPartitionId();
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshot.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshot.java
index bdd0c40ad8..5e947e32dc 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshot.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshot.java
@@ -324,7 +324,7 @@ public class OutgoingSnapshot {
         long[] commitTimestamps = new long[commitTimestampsCount];
 
         UUID transactionId = null;
-        UUID commitTableId = null;
+        Integer commitTableId = null;
         int commitPartitionId = ReadResult.UNDEFINED_COMMIT_PARTITION_ID;
 
         for (int i = count - 1, j = 0; i >= 0; i--) {
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorage.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorage.java
index 458a3dae26..1f942de68a 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorage.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorage.java
@@ -110,7 +110,7 @@ public class SnapshotAwarePartitionDataStorage implements PartitionDataStorage {
     }
 
     @Override
-    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId,
+    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId,
             int commitPartitionId) throws TxIdMismatchException, StorageException {
         handleSnapshotInterference(rowId);
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/CompatValidationResult.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/CompatValidationResult.java
index def1742451..a6b2e54d45 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/CompatValidationResult.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/CompatValidationResult.java
@@ -17,19 +17,14 @@
 
 package org.apache.ignite.internal.table.distributed.replicator;
 
-import java.util.Objects;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
 /**
  * Result of a schema compatibility validation.
  */
 public class CompatValidationResult {
-    private static final CompatValidationResult SUCCESS = new CompatValidationResult(true, null, -1, -1);
+    private static final CompatValidationResult SUCCESS = new CompatValidationResult(true, -1, -1, -1);
 
     private final boolean successful;
-    @Nullable
-    private final UUID failedTableId;
+    private final int failedTableId;
     private final int fromSchemaVersion;
     private final int toSchemaVersion;
 
@@ -50,11 +45,11 @@ public class CompatValidationResult {
      * @param toSchemaVersion Version number of the schema to which an incompatible transition tried to be made.
      * @return A validation result for a failure.
      */
-    public static CompatValidationResult failure(UUID failedTableId, int fromSchemaVersion, int toSchemaVersion) {
+    public static CompatValidationResult failure(int failedTableId, int fromSchemaVersion, int toSchemaVersion) {
         return new CompatValidationResult(false, failedTableId, fromSchemaVersion, toSchemaVersion);
     }
 
-    private CompatValidationResult(boolean successful, @Nullable UUID failedTableId, int fromSchemaVersion, int toSchemaVersion) {
+    private CompatValidationResult(boolean successful, int failedTableId, int fromSchemaVersion, int toSchemaVersion) {
         this.successful = successful;
         this.failedTableId = failedTableId;
         this.fromSchemaVersion = fromSchemaVersion;
@@ -76,8 +71,10 @@ public class CompatValidationResult {
      *
      * @return Table ID.
      */
-    public UUID failedTableId() {
-        return Objects.requireNonNull(failedTableId);
+    public int failedTableId() {
+        assert !successful : "Should not be called on a successful result";
+
+        return failedTableId;
     }
 
     /**
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
index eac16412b8..03e4247077 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
@@ -38,6 +38,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
@@ -257,7 +258,7 @@ public class PartitionReplicaListener implements ReplicaListener {
             LockManager lockManager,
             Executor scanRequestExecutor,
             int partId,
-            UUID tableId,
+            int tableId,
             Supplier<Map<UUID, IndexLocker>> indexesLockers,
             Lazy<TableSchemaAwareIndexStorage> pkIndexStorage,
             Supplier<Map<UUID, TableSchemaAwareIndexStorage>> secondaryIndexStorages,
@@ -1425,10 +1426,10 @@ public class PartitionReplicaListener implements ReplicaListener {
         ReadResult writeIntent = findAny(writeIntents).orElseThrow();
 
         for (ReadResult wi : writeIntents) {
-            assert wi.transactionId().equals(writeIntent.transactionId())
+            assert Objects.equals(wi.transactionId(), writeIntent.transactionId())
                     : "Unexpected write intent, tx1=" + writeIntent.transactionId() + ", tx2=" + wi.transactionId();
 
-            assert wi.commitTableId().equals(writeIntent.commitTableId())
+            assert Objects.equals(wi.commitTableId(), writeIntent.commitTableId())
                     : "Unexpected write intent, commitTableId1=" + writeIntent.commitTableId() + ", commitTableId2=" + wi.commitTableId();
 
             assert wi.commitPartitionId() == writeIntent.commitPartitionId()
@@ -2432,7 +2433,7 @@ public class PartitionReplicaListener implements ReplicaListener {
                 inBusyLock(() -> {
                     TableIndexView tableIndexView = ctx.newValue();
 
-                    if (tableId().equals(tableIndexView.tableId())) {
+                    if (tableId() == tableIndexView.tableId()) {
                         startBuildIndex(createIndexDescriptor(ctx.newValue(TablesView.class), tableIndexView.id()));
                     }
                 });
@@ -2450,7 +2451,7 @@ public class PartitionReplicaListener implements ReplicaListener {
                 inBusyLock(() -> {
                     TableIndexView tableIndexView = ctx.oldValue();
 
-                    if (tableId().equals(tableIndexView.tableId())) {
+                    if (tableId() == tableIndexView.tableId()) {
                         indexBuilder.stopBuildIndex(tableId(), partId(), tableIndexView.id());
                     }
                 });
@@ -2480,7 +2481,7 @@ public class PartitionReplicaListener implements ReplicaListener {
 
     private List<UUID> collectIndexIds(TablesView tablesView) {
         return tablesView.indexes().stream()
-                .filter(tableIndexView -> replicationGroupId.tableId().equals(tableIndexView.tableId()))
+                .filter(tableIndexView -> replicationGroupId.tableId() == tableIndexView.tableId())
                 .map(TableIndexView::id)
                 .collect(toList());
     }
@@ -2489,7 +2490,7 @@ public class PartitionReplicaListener implements ReplicaListener {
         return replicationGroupId.partitionId();
     }
 
-    private UUID tableId() {
+    private int tableId() {
         return replicationGroupId.tableId();
     }
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/SchemaCompatValidator.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/SchemaCompatValidator.java
index a9c5513c56..738c85f3f3 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/SchemaCompatValidator.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/SchemaCompatValidator.java
@@ -58,7 +58,7 @@ class SchemaCompatValidator {
     ) {
         HybridTimestamp beginTimestamp = TransactionIds.beginTimestamp(txId);
 
-        Set<UUID> tableIds = enlistedGroupIds.stream()
+        Set<Integer> tableIds = enlistedGroupIds.stream()
                 .map(TablePartitionId::tableId)
                 .collect(toSet());
 
@@ -73,11 +73,11 @@ class SchemaCompatValidator {
     }
 
     private CompatValidationResult validateForwardSchemasCompatibility(
-            Set<UUID> tableIds,
+            Set<Integer> tableIds,
             HybridTimestamp commitTimestamp,
             HybridTimestamp beginTimestamp
     ) {
-        for (UUID tableId : tableIds) {
+        for (int tableId : tableIds) {
             CompatValidationResult validationResult = validateForwardSchemaCompatibility(beginTimestamp, commitTimestamp, tableId);
 
             if (!validationResult.isSuccessful()) {
@@ -91,7 +91,7 @@ class SchemaCompatValidator {
     private CompatValidationResult validateForwardSchemaCompatibility(
             HybridTimestamp beginTimestamp,
             HybridTimestamp commitTimestamp,
-            UUID tableId
+            int tableId
     ) {
         List<FullTableSchema> tableSchemas = schemas.tableSchemaVersionsBetween(tableId, beginTimestamp, commitTimestamp);
 
@@ -130,7 +130,7 @@ class SchemaCompatValidator {
      * @param txId ID of the transaction that gets validated.
      * @return Future of validation result.
      */
-    CompletableFuture<CompatValidationResult> validateBackwards(int tupleSchemaVersion, UUID tableId, UUID txId) {
+    CompletableFuture<CompatValidationResult> validateBackwards(int tupleSchemaVersion, int tableId, UUID txId) {
         HybridTimestamp beginTimestamp = TransactionIds.beginTimestamp(txId);
 
         return schemas.waitForSchemasAvailability(beginTimestamp)
@@ -140,7 +140,7 @@ class SchemaCompatValidator {
 
     private CompatValidationResult validateBackwardSchemaCompatibility(
             int tupleSchemaVersion,
-            UUID tableId,
+            int tableId,
             HybridTimestamp beginTimestamp
     ) {
         List<FullTableSchema> tableSchemas = schemas.tableSchemaVersionsBetween(tableId, beginTimestamp, tupleSchemaVersion);
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/NonHistoricSchemas.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/NonHistoricSchemas.java
index 66f38fe125..49b622b8a1 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/NonHistoricSchemas.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/NonHistoricSchemas.java
@@ -21,7 +21,6 @@ import static java.util.concurrent.CompletableFuture.completedFuture;
 import static java.util.stream.Collectors.toList;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.catalog.commands.DefaultValue;
 import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
@@ -36,7 +35,7 @@ import org.apache.ignite.internal.schema.SchemaRegistry;
  *
  * <ul>
  *     <li>It imitates historicity, but always takes the latest known schema</li>
- *     <li>{@link #tableSchemaVersionsBetween(UUID, HybridTimestamp, HybridTimestamp)} always returns a single schema to avoid
+ *     <li>{@link #tableSchemaVersionsBetween(int, HybridTimestamp, HybridTimestamp)} always returns a single schema to avoid
  *     validation failures</li>
  * </ul>
  *
@@ -57,12 +56,12 @@ public class NonHistoricSchemas implements Schemas {
     }
 
     @Override
-    public CompletableFuture<?> waitForSchemaAvailability(UUID tableId, int schemaVersion) {
+    public CompletableFuture<?> waitForSchemaAvailability(int tableId, int schemaVersion) {
         return completedFuture(null);
     }
 
     @Override
-    public List<FullTableSchema> tableSchemaVersionsBetween(UUID tableId, HybridTimestamp fromIncluding, HybridTimestamp toIncluding) {
+    public List<FullTableSchema> tableSchemaVersionsBetween(int tableId, HybridTimestamp fromIncluding, HybridTimestamp toIncluding) {
         SchemaRegistry schemaRegistry = schemaManager.schemaRegistry(tableId);
         SchemaDescriptor schemaDescriptor = schemaRegistry.schema();
 
@@ -87,7 +86,7 @@ public class NonHistoricSchemas implements Schemas {
     }
 
     @Override
-    public List<FullTableSchema> tableSchemaVersionsBetween(UUID tableId, HybridTimestamp fromIncluding, int toIncluding) {
+    public List<FullTableSchema> tableSchemaVersionsBetween(int tableId, HybridTimestamp fromIncluding, int toIncluding) {
         // Returning an empty list makes sure that backward validation never fails, which is what we want before
         // we switch to CatalogService completely.
         return List.of();
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/Schemas.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/Schemas.java
index 2ed9db14ac..54ff6d659a 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/Schemas.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/schema/Schemas.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.table.distributed.schema;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 
@@ -43,7 +42,7 @@ public interface Schemas {
      * @param schemaVersion ID of the schema version.
      * @return Future that completes when the given schema version becomes available.
      */
-    CompletableFuture<?> waitForSchemaAvailability(UUID tableId, int schemaVersion);
+    CompletableFuture<?> waitForSchemaAvailability(int tableId, int schemaVersion);
 
     /**
      * Returns all schema versions between (including) the two that were effective at the given timestamps.
@@ -53,7 +52,7 @@ public interface Schemas {
      * @param toIncluding End timestamp.
      * @return All schema versions between (including) the two that were effective at the given timestamps.
      */
-    List<FullTableSchema> tableSchemaVersionsBetween(UUID tableId, HybridTimestamp fromIncluding, HybridTimestamp toIncluding);
+    List<FullTableSchema> tableSchemaVersionsBetween(int tableId, HybridTimestamp fromIncluding, HybridTimestamp toIncluding);
 
     /**
      * Returns all schema versions between (including) the one that was effective at the given timestamp and
@@ -65,5 +64,5 @@ public interface Schemas {
      * @param toIncluding End schema version ID.
      * @return All schema versions between (including) the given timestamp and schema version.
      */
-    List<FullTableSchema> tableSchemaVersionsBetween(UUID tableId, HybridTimestamp fromIncluding, int toIncluding);
+    List<FullTableSchema> tableSchemaVersionsBetween(int tableId, HybridTimestamp fromIncluding, int toIncluding);
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
index 077476e8c4..4d7f369aee 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
@@ -113,7 +113,7 @@ public class InternalTableImpl implements InternalTable {
     private final String tableName;
 
     /** Table identifier. */
-    private final UUID tableId;
+    private final int tableId;
 
     /** Resolver that resolves a node consistent ID to cluster node. */
     private final Function<String, ClusterNode> clusterNodeResolver;
@@ -160,7 +160,7 @@ public class InternalTableImpl implements InternalTable {
      */
     public InternalTableImpl(
             String tableName,
-            UUID tableId,
+            int tableId,
             Int2ObjectMap<RaftGroupService> partMap,
             int partitions,
             Function<String, ClusterNode> clusterNodeResolver,
@@ -197,7 +197,7 @@ public class InternalTableImpl implements InternalTable {
 
     /** {@inheritDoc} */
     @Override
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java b/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
index 97c7f3272d..4fb91b631c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/event/TableEventParameters.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.table.event;
 
-import java.util.UUID;
 import org.apache.ignite.internal.manager.EventParameters;
 
 /**
@@ -25,7 +24,7 @@ import org.apache.ignite.internal.manager.EventParameters;
  */
 public class TableEventParameters extends EventParameters {
     /** Table identifier. */
-    private final UUID tableId;
+    private final int tableId;
 
     /**
      * Constructor.
@@ -33,7 +32,7 @@ public class TableEventParameters extends EventParameters {
      * @param causalityToken Causality token.
      * @param tableId   Table identifier.
      */
-    public TableEventParameters(long causalityToken, UUID tableId) {
+    public TableEventParameters(long causalityToken, int tableId) {
         super(causalityToken);
         this.tableId = tableId;
     }
@@ -43,7 +42,7 @@ public class TableEventParameters extends EventParameters {
      *
      * @return Table id.
      */
-    public UUID tableId() {
+    public int tableId() {
         return tableId;
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java b/modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java
index 81050182af..ba5d2093e5 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/utils/RebalanceUtil.java
@@ -31,7 +31,6 @@ import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.affinity.AffinityUtils;
 import org.apache.ignite.internal.affinity.Assignment;
@@ -133,7 +132,7 @@ public class RebalanceUtil {
      * @param key Key.
      * @return Table id.
      */
-    public static UUID extractTableId(byte[] key) {
+    public static int extractTableId(byte[] key) {
         return extractTableId(key, "");
     }
 
@@ -144,10 +143,10 @@ public class RebalanceUtil {
      * @param prefix Key prefix.
      * @return Table id.
      */
-    public static UUID extractTableId(byte[] key, String prefix) {
+    public static int extractTableId(byte[] key, String prefix) {
         String strKey = new String(key, StandardCharsets.UTF_8);
 
-        return UUID.fromString(strKey.substring(prefix.length(), strKey.indexOf("_part_")));
+        return Integer.parseInt(strKey.substring(prefix.length(), strKey.indexOf("_part_")));
     }
 
     /**
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
index 8c39b45678..6953346121 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/RepeatedFinishReadWriteTransactionTest.java
@@ -63,7 +63,7 @@ public class RepeatedFinishReadWriteTransactionTest {
 
         ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, UUID.randomUUID());
 
-        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId partId = testTablePartitionId();
 
         tx.enlist(partId, new IgniteBiTuple<>(clusterNode, 1L));
 
@@ -100,6 +100,10 @@ public class RepeatedFinishReadWriteTransactionTest {
         assertTrue(rollbackFut.isDone());
     }
 
+    private static TablePartitionId testTablePartitionId() {
+        return new TablePartitionId(1, 1);
+    }
+
     @Test
     public void testRepeatedCommitRollbackAfterRollback() throws Exception {
         CountDownLatch txFinishStartedLatch = new CountDownLatch(1);
@@ -109,7 +113,7 @@ public class RepeatedFinishReadWriteTransactionTest {
 
         ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, UUID.randomUUID());
 
-        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId partId = testTablePartitionId();
 
         tx.enlist(partId, new IgniteBiTuple<>(clusterNode, 1L));
 
@@ -155,7 +159,7 @@ public class RepeatedFinishReadWriteTransactionTest {
 
         ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, UUID.randomUUID());
 
-        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId partId = testTablePartitionId();
 
         tx.enlist(partId, new IgniteBiTuple<>(null, null));
 
@@ -190,7 +194,7 @@ public class RepeatedFinishReadWriteTransactionTest {
 
         ReadWriteTransactionImpl tx = new ReadWriteTransactionImpl(txManager, UUID.randomUUID());
 
-        TablePartitionId partId = new TablePartitionId(UUID.randomUUID(), 1);
+        TablePartitionId partId = testTablePartitionId();
 
         tx.enlist(partId, new IgniteBiTuple<>(null, null));
 
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/AbstractMvStorageUpdateHandlerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/AbstractMvStorageUpdateHandlerTest.java
index 17c056bbff..a458d3a838 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/AbstractMvStorageUpdateHandlerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/AbstractMvStorageUpdateHandlerTest.java
@@ -164,7 +164,7 @@ abstract class AbstractMvStorageUpdateHandlerTest extends BaseMvStoragesTest {
         storage.runConsistently(locker -> {
             locker.lock(rowId);
 
-            storage.addWrite(rowId, row, UUID.randomUUID(), UUID.randomUUID(), PARTITION_ID);
+            storage.addWrite(rowId, row, UUID.randomUUID(), 999, PARTITION_ID);
 
             storage.commitWrite(rowId, timestamp);
 
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
index 9d93309663..da0be154a6 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
@@ -158,7 +158,7 @@ public abstract class IndexBaseTest extends BaseMvStoragesTest {
     }
 
     static void addWrite(StorageUpdateHandler handler, UUID rowUuid, @Nullable BinaryRow row) {
-        TablePartitionId partitionId = new TablePartitionId(UUID.randomUUID(), PARTITION_ID);
+        TablePartitionId partitionId = new TablePartitionId(333, PARTITION_ID);
 
         handler.handleUpdate(
                 TX_ID,
@@ -249,7 +249,7 @@ public abstract class IndexBaseTest extends BaseMvStoragesTest {
         };
 
         void addWrite(StorageUpdateHandler handler, UUID rowUuid, @Nullable BinaryRow row) {
-            TablePartitionId tablePartitionId = new TablePartitionId(UUID.randomUUID(), PARTITION_ID);
+            TablePartitionId tablePartitionId = new TablePartitionId(444, PARTITION_ID);
 
             addWrite(handler, tablePartitionId, rowUuid, row);
         }
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandlerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandlerTest.java
index 112bdbf087..8f135c7e33 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandlerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/StorageUpdateHandlerTest.java
@@ -109,7 +109,7 @@ public class StorageUpdateHandlerTest {
         storageUpdateHandler.handleUpdate(
                 UUID.randomUUID(),
                 UUID.randomUUID(),
-                new TablePartitionId(UUID.randomUUID(), PARTITION_ID),
+                new TablePartitionId(1, PARTITION_ID),
                 null,
                 null
         );
@@ -130,7 +130,7 @@ public class StorageUpdateHandlerTest {
         storageUpdateHandler.handleUpdateAll(
                 UUID.randomUUID(),
                 Map.of(),
-                new TablePartitionId(UUID.randomUUID(), PARTITION_ID),
+                new TablePartitionId(1, PARTITION_ID),
                 null
         );
 
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableManagerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableManagerTest.java
index d5ace866a5..852d37589a 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableManagerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/TableManagerTest.java
@@ -462,7 +462,7 @@ public class TableManagerTest extends IgniteAbstractTest {
         tableManager.beforeNodeStop();
         tableManager.stop();
 
-        UUID fakeTblId = UUID.randomUUID();
+        int fakeTblId = 1;
 
         assertThrows(IgniteException.class, () -> tableManager.table(fakeTblId));
         assertThrows(IgniteException.class, () -> tableManager.tableAsync(fakeTblId));
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/command/PartitionRaftCommandsSerializationTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/command/PartitionRaftCommandsSerializationTest.java
index 4651e9fcc5..60d728e269 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/command/PartitionRaftCommandsSerializationTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/command/PartitionRaftCommandsSerializationTest.java
@@ -79,7 +79,7 @@ public class PartitionRaftCommandsSerializationTest extends IgniteAbstractTest {
     public void testUpdateCommand() throws Exception {
         UpdateCommand cmd = msgFactory.updateCommand()
                 .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                        .tableId(UUID.randomUUID())
+                        .tableId(1)
                         .partitionId(1)
                         .build()
                 )
@@ -99,7 +99,7 @@ public class PartitionRaftCommandsSerializationTest extends IgniteAbstractTest {
     public void testRemoveCommand() throws Exception {
         UpdateCommand cmd = msgFactory.updateCommand()
                 .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                        .tableId(UUID.randomUUID())
+                        .tableId(1)
                         .partitionId(1)
                         .build()
                 )
@@ -124,7 +124,7 @@ public class PartitionRaftCommandsSerializationTest extends IgniteAbstractTest {
 
         var cmd = msgFactory.updateAllCommand()
                 .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                        .tableId(UUID.randomUUID())
+                        .tableId(1)
                         .partitionId(1)
                         .build()
                 )
@@ -156,7 +156,7 @@ public class PartitionRaftCommandsSerializationTest extends IgniteAbstractTest {
 
         var cmd = msgFactory.updateAllCommand()
                 .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                        .tableId(UUID.randomUUID())
+                        .tableId(1)
                         .partitionId(1)
                         .build()
                 )
@@ -198,7 +198,7 @@ public class PartitionRaftCommandsSerializationTest extends IgniteAbstractTest {
 
         for (int i = 0; i < 10; i++) {
             grps.add(msgFactory.tablePartitionIdMessage()
-                    .tableId(UUID.randomUUID())
+                    .tableId(1)
                     .partitionId(i)
                     .build());
         }
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/gc/MvGcTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/gc/MvGcTest.java
index f57eb0a755..06cfa85a8e 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/gc/MvGcTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/gc/MvGcTest.java
@@ -34,10 +34,10 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
 import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
@@ -62,6 +62,8 @@ public class MvGcTest {
 
     private MvGc gc;
 
+    private final AtomicInteger nextTableId = new AtomicInteger(1001);
+
     @BeforeEach
     void setUp(
             @InjectConfiguration("mock.gcThreads = 1")
@@ -367,7 +369,7 @@ public class MvGcTest {
     }
 
     private TablePartitionId createTablePartitionId() {
-        return new TablePartitionId(UUID.randomUUID(), PARTITION_ID);
+        return new TablePartitionId(nextTableId.getAndIncrement(), PARTITION_ID);
     }
 
     private GcUpdateHandler createWithCompleteFutureOnVacuum(CompletableFuture<Void> future, @Nullable HybridTimestamp exp) {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
index cba289ea05..59fa8ca452 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
@@ -513,7 +513,7 @@ public class PartitionCommandListenerTest {
         return msgFactory.buildIndexCommand()
                 .tablePartitionId(
                         msgFactory.tablePartitionIdMessage()
-                                .tableId(UUID.randomUUID())
+                                .tableId(1)
                                 .partitionId(PARTITION_ID)
                                 .build()
                 )
@@ -623,7 +623,7 @@ public class PartitionCommandListenerTest {
     private void insertAll() {
         Map<UUID, ByteBuffer> rows = new HashMap<>(KEY_COUNT);
         UUID txId = TestTransactionIds.newTransactionId();
-        var commitPartId = new TablePartitionId(txId, PARTITION_ID);
+        var commitPartId = new TablePartitionId(1, PARTITION_ID);
 
         for (int i = 0; i < KEY_COUNT; i++) {
             Row row = getTestRow(i, i);
@@ -659,7 +659,7 @@ public class PartitionCommandListenerTest {
      */
     private void updateAll(Function<Integer, Integer> keyValueMapper) {
         UUID txId = TestTransactionIds.newTransactionId();
-        var commitPartId = new TablePartitionId(txId, PARTITION_ID);
+        var commitPartId = new TablePartitionId(1, PARTITION_ID);
         Map<UUID, ByteBuffer> rows = new HashMap<>(KEY_COUNT);
 
         for (int i = 0; i < KEY_COUNT; i++) {
@@ -694,7 +694,7 @@ public class PartitionCommandListenerTest {
      */
     private void deleteAll() {
         UUID txId = TestTransactionIds.newTransactionId();
-        var commitPartId = new TablePartitionId(txId, PARTITION_ID);
+        var commitPartId = new TablePartitionId(1, PARTITION_ID);
         Map<UUID, ByteBuffer> keyRows = new HashMap<>(KEY_COUNT);
 
         for (int i = 0; i < KEY_COUNT; i++) {
@@ -746,7 +746,7 @@ public class PartitionCommandListenerTest {
             when(clo.command()).thenReturn(
                     msgFactory.updateCommand()
                             .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                                    .tableId(txId)
+                                    .tableId(1)
                                     .partitionId(PARTITION_ID).build())
                             .rowUuid(rowId.uuid())
                             .rowBuffer(row.byteBuffer())
@@ -791,7 +791,7 @@ public class PartitionCommandListenerTest {
             when(clo.command()).thenReturn(
                     msgFactory.updateCommand()
                             .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                                    .tableId(txId)
+                                    .tableId(1)
                                     .partitionId(PARTITION_ID).build())
                             .rowUuid(rowId.uuid())
                             .txId(txId)
@@ -865,7 +865,7 @@ public class PartitionCommandListenerTest {
             when(clo.command()).thenReturn(
                     msgFactory.updateCommand()
                             .tablePartitionId(msgFactory.tablePartitionIdMessage()
-                                    .tableId(txId)
+                                    .tableId(1)
                                     .partitionId(PARTITION_ID).build())
                             .rowUuid(UUID.randomUUID())
                             .rowBuffer(row.byteBuffer())
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImplTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImplTest.java
index f1d0c25c73..59de512e2a 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImplTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/PartitionAccessImplTest.java
@@ -69,7 +69,7 @@ public class PartitionAccessImplTest {
         TxStateStorage txStateStorage = txStateTableStorage.getOrCreateTxStateStorage(TEST_PARTITION_ID);
 
         PartitionAccess partitionAccess = new PartitionAccessImpl(
-                new PartitionKey(UUID.randomUUID(), TEST_PARTITION_ID),
+                testPartitionKey(),
                 mvTableStorage,
                 txStateTableStorage,
                 mock(MvGc.class),
@@ -102,6 +102,10 @@ public class PartitionAccessImplTest {
         assertEquals(20, partitionAccess.maxLastAppliedIndex());
     }
 
+    private static PartitionKey testPartitionKey() {
+        return new PartitionKey(1, TEST_PARTITION_ID);
+    }
+
     @Test
     void testMinMaxLastAppliedTerm() {
         TestMvTableStorage mvTableStorage = new TestMvTableStorage(tablesConfig.tables().get("foo"), tablesConfig,
@@ -112,7 +116,7 @@ public class PartitionAccessImplTest {
         TxStateStorage txStateStorage = txStateTableStorage.getOrCreateTxStateStorage(TEST_PARTITION_ID);
 
         PartitionAccess partitionAccess = new PartitionAccessImpl(
-                new PartitionKey(UUID.randomUUID(), TEST_PARTITION_ID),
+                testPartitionKey(),
                 mvTableStorage,
                 txStateTableStorage,
                 mock(MvGc.class),
@@ -155,7 +159,7 @@ public class PartitionAccessImplTest {
         IndexUpdateHandler indexUpdateHandler = mock(IndexUpdateHandler.class);
 
         PartitionAccess partitionAccess = new PartitionAccessImpl(
-                new PartitionKey(UUID.randomUUID(), TEST_PARTITION_ID),
+                testPartitionKey(),
                 mvTableStorage,
                 new TestTxStateTableStorage(),
                 mock(MvGc.class),
@@ -166,7 +170,7 @@ public class PartitionAccessImplTest {
         RowId rowId = new RowId(TEST_PARTITION_ID);
         BinaryRow binaryRow = mock(BinaryRow.class);
         UUID txId = UUID.randomUUID();
-        UUID commitTableId = UUID.randomUUID();
+        int commitTableId = 999;
 
         partitionAccess.addWrite(rowId, binaryRow, txId, commitTableId, TEST_PARTITION_ID);
 
@@ -196,7 +200,7 @@ public class PartitionAccessImplTest {
         IndexUpdateHandler indexUpdateHandler = mock(IndexUpdateHandler.class);
 
         PartitionAccess partitionAccess = new PartitionAccessImpl(
-                new PartitionKey(UUID.randomUUID(), TEST_PARTITION_ID),
+                testPartitionKey(),
                 mvTableStorage,
                 new TestTxStateTableStorage(),
                 mock(MvGc.class),
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopierTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopierTest.java
index 1479c11f79..4ae4cb9413 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopierTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopierTest.java
@@ -144,7 +144,7 @@ public class IncomingSnapshotCopierTest {
 
     private final UUID snapshotId = UUID.randomUUID();
 
-    private final UUID tableId = UUID.randomUUID();
+    private final int tableId = 1;
 
     private final RaftGroupConfigurationConverter raftGroupConfigurationConverter = new RaftGroupConfigurationConverter();
 
@@ -326,7 +326,7 @@ public class IncomingSnapshotCopierTest {
                     storage.addWriteCommitted(rowIds.get(i), createRow("k" + i, "v" + i), HYBRID_CLOCK.now());
                 } else {
                     // Writes an intent to write (uncommitted version).
-                    storage.addWrite(rowIds.get(i), createRow("k" + i, "v" + i), UUID.randomUUID(), UUID.randomUUID(), TEST_PARTITION);
+                    storage.addWrite(rowIds.get(i), createRow("k" + i, "v" + i), UUID.randomUUID(), 999, TEST_PARTITION);
                 }
             }
 
@@ -346,7 +346,7 @@ public class IncomingSnapshotCopierTest {
     ) {
         assertEquals(0, txIds.size() % 2, "size=" + txIds.size());
 
-        UUID tableId = UUID.randomUUID();
+        int tableId = 2;
 
         for (int i = 0; i < txIds.size(); i++) {
             TxState txState = i % 2 == 0 ? COMMITED : ABORTED;
@@ -369,7 +369,7 @@ public class IncomingSnapshotCopierTest {
             long[] timestamps = new long[readResults.size() + (readResults.get(0).isWriteIntent() ? -1 : 0)];
 
             UUID txId = null;
-            UUID commitTableId = null;
+            Integer commitTableId = null;
             int commitPartitionId = ReadResult.UNDEFINED_COMMIT_PARTITION_ID;
 
             int j = 0;
@@ -528,7 +528,7 @@ public class IncomingSnapshotCopierTest {
 
             return null;
         }).when(partitionSnapshotStorage.partition())
-                .addWrite(any(RowId.class), any(BinaryRow.class), any(UUID.class), any(UUID.class), anyInt());
+                .addWrite(any(RowId.class), any(BinaryRow.class), any(UUID.class), anyInt(), anyInt());
 
         // Let's start rebalancing.
         SnapshotCopier snapshotCopier = partitionSnapshotStorage.startToCopyFrom(
@@ -588,7 +588,7 @@ public class IncomingSnapshotCopierTest {
 
         // Let's add an error on the rebalance.
         doThrow(StorageException.class).when(partitionSnapshotStorage.partition())
-                .addWrite(any(RowId.class), any(BinaryRow.class), any(UUID.class), any(UUID.class), anyInt());
+                .addWrite(any(RowId.class), any(BinaryRow.class), any(UUID.class), anyInt(), anyInt());
 
         // Let's start rebalancing.
         SnapshotCopier snapshotCopier = partitionSnapshotStorage.startToCopyFrom(
@@ -612,7 +612,7 @@ public class IncomingSnapshotCopierTest {
     void cancellationsFromMultipleThreadsDoNotBlockEachOther() throws Exception {
         PartitionSnapshotStorage partitionSnapshotStorage = mock(PartitionSnapshotStorage.class, Answers.RETURNS_DEEP_STUBS);
 
-        when(partitionSnapshotStorage.partition().partitionKey()).thenReturn(new PartitionKey(UUID.randomUUID(), 0));
+        when(partitionSnapshotStorage.partition().partitionKey()).thenReturn(new PartitionKey(1, 0));
 
         IncomingSnapshotCopier copier = new IncomingSnapshotCopier(
                 partitionSnapshotStorage,
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotCommonTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotCommonTest.java
index d54789d8b8..502ee53141 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotCommonTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotCommonTest.java
@@ -47,7 +47,7 @@ class OutgoingSnapshotCommonTest {
 
     private final TableMessagesFactory messagesFactory = new TableMessagesFactory();
 
-    private final PartitionKey partitionKey = new PartitionKey(UUID.randomUUID(), 1);
+    private final PartitionKey partitionKey = new PartitionKey(1, 1);
 
     @BeforeEach
     void createTestInstance() {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotMvDataStreamingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotMvDataStreamingTest.java
index 70b343d090..3fd8cbcef8 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotMvDataStreamingTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotMvDataStreamingTest.java
@@ -69,9 +69,9 @@ class OutgoingSnapshotMvDataStreamingTest {
     private final HybridClock clock = new HybridClockImpl();
 
     private final UUID transactionId = UUID.randomUUID();
-    private final UUID commitTableId = UUID.randomUUID();
+    private final int commitTableId = 999;
 
-    private final PartitionKey partitionKey = new PartitionKey(UUID.randomUUID(), 1);
+    private final PartitionKey partitionKey = new PartitionKey(1, 1);
 
     @BeforeEach
     void createTestInstance() {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotReaderTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotReaderTest.java
index 0a610c224b..cb6353a4a4 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotReaderTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotReaderTest.java
@@ -23,7 +23,6 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import java.util.UUID;
 import java.util.concurrent.Executor;
 import org.apache.ignite.internal.table.distributed.raft.RaftGroupConfiguration;
 import org.apache.ignite.internal.table.distributed.raft.snapshot.PartitionAccess;
@@ -42,7 +41,7 @@ public class OutgoingSnapshotReaderTest {
     void testForChoosingMaximumAppliedIndexForMeta() {
         PartitionAccess partitionAccess = mock(PartitionAccess.class);
 
-        when(partitionAccess.partitionKey()).thenReturn(new PartitionKey(UUID.randomUUID(), 0));
+        when(partitionAccess.partitionKey()).thenReturn(new PartitionKey(1, 0));
         when(partitionAccess.committedGroupConfiguration()).thenReturn(mock(RaftGroupConfiguration.class));
 
         OutgoingSnapshotsManager outgoingSnapshotsManager = mock(OutgoingSnapshotsManager.class);
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotTxDataStreamingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotTxDataStreamingTest.java
index 6c2cc8385d..5b5871076b 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotTxDataStreamingTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotTxDataStreamingTest.java
@@ -67,13 +67,13 @@ class OutgoingSnapshotTxDataStreamingTest {
 
     private final HybridClock clock = new HybridClockImpl();
 
-    private final TablePartitionId partition1Id = new TablePartitionId(UUID.randomUUID(), 1);
-    private final TablePartitionId partition2Id = new TablePartitionId(UUID.randomUUID(), 2);
+    private final TablePartitionId partition1Id = new TablePartitionId(1, 1);
+    private final TablePartitionId partition2Id = new TablePartitionId(2, 2);
 
     private final TxMeta meta1 = new TxMeta(TxState.ABORTED, List.of(partition1Id), clock.now());
     private final TxMeta meta2 = new TxMeta(TxState.COMMITED, List.of(partition1Id, partition2Id), clock.now());
 
-    private final PartitionKey partitionKey = new PartitionKey(UUID.randomUUID(), 1);
+    private final PartitionKey partitionKey = new PartitionKey(1, 1);
 
     @BeforeEach
     void createTestInstance() {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotsManagerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotsManagerTest.java
index 07b5b1da50..c40c6baac9 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotsManagerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/OutgoingSnapshotsManagerTest.java
@@ -43,7 +43,7 @@ class OutgoingSnapshotsManagerTest {
     @Mock
     private PartitionAccess partitionAccess;
 
-    private final PartitionKey partitionKey = new PartitionKey(UUID.randomUUID(), 1);
+    private final PartitionKey partitionKey = new PartitionKey(1, 1);
 
     @SuppressWarnings("EmptyTryBlock")
     @Test
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorageTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorageTest.java
index 60a379dab2..12a98c9eff 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorageTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/snapshot/outgoing/SnapshotAwarePartitionDataStorageTest.java
@@ -62,7 +62,7 @@ class SnapshotAwarePartitionDataStorageTest {
     private PartitionsSnapshots partitionsSnapshots;
 
     @Spy
-    private final PartitionKey partitionKey = new PartitionKey(UUID.randomUUID(), PARTITION_ID);
+    private final PartitionKey partitionKey = new PartitionKey(1, PARTITION_ID);
 
     @InjectMocks
     private SnapshotAwarePartitionDataStorage testedStorage;
@@ -145,11 +145,11 @@ class SnapshotAwarePartitionDataStorageTest {
     void delegatesAddWrite() {
         BinaryRow resultRow = mock(BinaryRow.class);
 
-        when(partitionStorage.addWrite(any(), any(), any(), any(), anyInt())).thenReturn(resultRow);
+        when(partitionStorage.addWrite(any(), any(), any(), anyInt(), anyInt())).thenReturn(resultRow);
 
         BinaryRow argumentRow = mock(BinaryRow.class);
         UUID txId = UUID.randomUUID();
-        UUID commitTableId = UUID.randomUUID();
+        int commitTableId = 999;
 
         assertThat(testedStorage.addWrite(rowId, argumentRow, txId, commitTableId, 42), is(resultRow));
         verify(partitionStorage).addWrite(rowId, argumentRow, txId, commitTableId, 42);
@@ -288,7 +288,7 @@ class SnapshotAwarePartitionDataStorageTest {
         ADD_WRITE {
             @Override
             void executeOn(SnapshotAwarePartitionDataStorage storage, RowId rowId) {
-                storage.addWrite(rowId, mock(BinaryRow.class), UUID.randomUUID(), UUID.randomUUID(), 42);
+                storage.addWrite(rowId, mock(BinaryRow.class), UUID.randomUUID(), 999, 42);
             }
         },
         ABORT_WRITE {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
index 46077dbea8..402ec89e30 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
@@ -104,7 +104,7 @@ import org.junit.jupiter.params.provider.MethodSource;
 @ExtendWith(ConfigurationExtension.class)
 public class PartitionReplicaListenerIndexLockingTest extends IgniteAbstractTest {
     private static final int PART_ID = 0;
-    private static final UUID TABLE_ID = new UUID(0L, 0L);
+    private static final int TABLE_ID = 1;
     private static final UUID PK_INDEX_ID = new UUID(0L, 1L);
     private static final UUID HASH_INDEX_ID = new UUID(0L, 2L);
     private static final UUID SORTED_INDEX_ID = new UUID(0L, 3L);
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
index eea21facbb..6ba6468f4a 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
@@ -178,7 +178,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
     private static final int FUTURE_SCHEMA_ROW_INDEXED_VALUE = 0;
 
     /** Table id. */
-    private final UUID tblId = UUID.randomUUID();
+    private final int tblId = 1;
 
     private final Map<UUID, Set<RowId>> pendingRows = new ConcurrentHashMap<>();
 
@@ -344,7 +344,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
         lenient().when(safeTimeClock.waitFor(any())).thenReturn(completedFuture(null));
 
         lenient().when(schemas.waitForSchemasAvailability(any())).thenReturn(completedFuture(null));
-        lenient().when(schemas.waitForSchemaAvailability(any(), anyInt())).thenReturn(completedFuture(null));
+        lenient().when(schemas.waitForSchemaAvailability(anyInt(), anyInt())).thenReturn(completedFuture(null));
 
         UUID pkIndexId = UUID.randomUUID();
         UUID sortedIndexId = UUID.randomUUID();
@@ -991,18 +991,22 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
                 .requestType(requestType)
                 .binaryRow(binaryRow)
                 .term(1L)
-                .commitPartitionId(new TablePartitionId(UUID.randomUUID(), partId))
+                .commitPartitionId(commitPartitionId())
                 .build()
         );
     }
 
+    private static TablePartitionId commitPartitionId() {
+        return new TablePartitionId(999, partId);
+    }
+
     private CompletableFuture<?> doMultiRowRequest(UUID txId, Collection<BinaryRow> binaryRows, RequestType requestType) {
         return partitionReplicaListener.invoke(TABLE_MESSAGES_FACTORY.readWriteMultiRowReplicaRequest()
                 .transactionId(txId)
                 .requestType(requestType)
                 .binaryRows(binaryRows)
                 .term(1L)
-                .commitPartitionId(new TablePartitionId(UUID.randomUUID(), partId))
+                .commitPartitionId(commitPartitionId())
                 .build()
         );
     }
@@ -1022,7 +1026,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
                             .requestType(RequestType.RW_INSERT)
                             .binaryRow(binaryRow)
                             .term(1L)
-                            .commitPartitionId(new TablePartitionId(UUID.randomUUID(), partId))
+                            .commitPartitionId(commitPartitionId())
                             .build();
                 },
                 () -> checkRowInMvStorage(binaryRow(0), true)
@@ -1048,7 +1052,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
                             .requestType(RequestType.RW_UPSERT_ALL)
                             .binaryRows(asList(binaryRow0, binaryRow1))
                             .term(1L)
-                            .commitPartitionId(new TablePartitionId(UUID.randomUUID(), partId))
+                            .commitPartitionId(commitPartitionId())
                             .build();
                 },
                 () -> checkRowInMvStorage(binaryRow(0), true)
@@ -1266,7 +1270,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
 
     @Test
     public void commitsOnSameSchemaSuccessfully() {
-        when(schemas.tableSchemaVersionsBetween(any(), any(), any()))
+        when(schemas.tableSchemaVersionsBetween(anyInt(), any(), any(HybridTimestamp.class)))
                 .thenReturn(List.of(
                         tableSchema(CURRENT_SCHEMA_VERSION, List.of(nullableColumn("col")))
                 ));
@@ -1328,7 +1332,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
     @Test
     @Disabled("IGNITE-19229")
     public void commitsOnCompatibleSchemaChangeSuccessfully() {
-        when(schemas.tableSchemaVersionsBetween(any(), any(), any()))
+        when(schemas.tableSchemaVersionsBetween(anyInt(), any(), any(HybridTimestamp.class)))
                 .thenReturn(List.of(
                         tableSchema(CURRENT_SCHEMA_VERSION, List.of(nullableColumn("col1"))),
                         tableSchema(FUTURE_SCHEMA_VERSION, List.of(nullableColumn("col1"), nullableColumn("col2")))
@@ -1360,12 +1364,12 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
     }
 
     private void simulateForwardIncompatibleSchemaChange(int fromSchemaVersion, int toSchemaVersion) {
-        when(schemas.tableSchemaVersionsBetween(any(), any(), any()))
+        when(schemas.tableSchemaVersionsBetween(anyInt(), any(), any(HybridTimestamp.class)))
                 .thenReturn(incompatibleSchemaVersions(fromSchemaVersion, toSchemaVersion));
     }
 
     private void simulateBackwardIncompatibleSchemaChange(int fromSchemaVersion, int toSchemaVersion) {
-        when(schemas.tableSchemaVersionsBetween(any(), any(), anyInt()))
+        when(schemas.tableSchemaVersionsBetween(anyInt(), any(), anyInt()))
                 .thenReturn(incompatibleSchemaVersions(fromSchemaVersion, toSchemaVersion));
     }
 
@@ -1463,7 +1467,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
                         .oldBinaryRow(binaryRow(key, new TestValue(1, "v1"), kvMarshaller))
                         .binaryRow(binaryRow(key, new TestValue(3, "v3"), kvMarshaller))
                         .term(1L)
-                        .commitPartitionId(new TablePartitionId(UUID.randomUUID(), partId))
+                        .commitPartitionId(commitPartitionId())
                         .build()
                 )
         );
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImplTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImplTest.java
index f2a4dda60b..6c1b529c6d 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImplTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImplTest.java
@@ -24,7 +24,6 @@ import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 
 import it.unimi.dsi.fastutil.ints.Int2ObjectMaps;
-import java.util.UUID;
 import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.replicator.ReplicaService;
@@ -43,7 +42,7 @@ public class InternalTableImplTest {
     void testUpdatePartitionTrackers() {
         InternalTableImpl internalTable = new InternalTableImpl(
                 "test",
-                UUID.randomUUID(),
+                1,
                 Int2ObjectMaps.emptyMap(),
                 1,
                 s -> mock(ClusterNode.class),
diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/distributed/TestPartitionDataStorage.java b/modules/table/src/testFixtures/java/org/apache/ignite/distributed/TestPartitionDataStorage.java
index cffd10c270..744ea574bc 100644
--- a/modules/table/src/testFixtures/java/org/apache/ignite/distributed/TestPartitionDataStorage.java
+++ b/modules/table/src/testFixtures/java/org/apache/ignite/distributed/TestPartitionDataStorage.java
@@ -93,7 +93,7 @@ public class TestPartitionDataStorage implements PartitionDataStorage {
     }
 
     @Override
-    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, UUID commitTableId,
+    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId, int commitTableId,
             int commitPartitionId) throws TxIdMismatchException, StorageException {
         return partitionStorage.addWrite(rowId, row, txId, commitTableId, commitPartitionId);
     }
diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
index 27be15383c..b5c36b28d3 100644
--- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
+++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Function;
 import javax.naming.OperationNotSupportedException;
@@ -108,7 +109,7 @@ public class DummyInternalTableImpl extends InternalTableImpl {
 
     private static final HybridClock CLOCK = new HybridClockImpl();
 
-    private static final ReplicationGroupId crossTableGroupId = new TablePartitionId(UUID.randomUUID(), 0);
+    private static final ReplicationGroupId crossTableGroupId = new TablePartitionId(333, 0);
 
     private PartitionListener partitionListener;
 
@@ -119,6 +120,8 @@ public class DummyInternalTableImpl extends InternalTableImpl {
     /** The thread updates safe time on the dummy replica. */
     private Thread safeTimeUpdaterThread;
 
+    private static final AtomicInteger nextTableId = new AtomicInteger(10_001);
+
     /**
      * Creates a new local table.
      *
@@ -184,7 +187,7 @@ public class DummyInternalTableImpl extends InternalTableImpl {
     ) {
         super(
                 "test",
-                UUID.randomUUID(),
+                nextTableId.getAndIncrement(),
                 Int2ObjectMaps.singleton(PART_ID, mock(RaftGroupService.class)),
                 1,
                 name -> mock(ClusterNode.class),
@@ -257,7 +260,7 @@ public class DummyInternalTableImpl extends InternalTableImpl {
                 }
         ).when(svc).run(any());
 
-        UUID tableId = tableId();
+        int tableId = tableId();
         UUID indexId = UUID.randomUUID();
 
         Function<BinaryRow, BinaryTuple> row2Tuple = BinaryRowConverter.keyExtractor(schema);
diff --git a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummySchemas.java b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummySchemas.java
index b70bb900c7..6a1cc4efa7 100644
--- a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummySchemas.java
+++ b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummySchemas.java
@@ -21,7 +21,6 @@ import static java.util.concurrent.CompletableFuture.completedFuture;
 import static java.util.stream.Collectors.toList;
 
 import java.util.List;
-import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
@@ -48,12 +47,12 @@ public class DummySchemas implements Schemas {
     }
 
     @Override
-    public CompletableFuture<?> waitForSchemaAvailability(UUID tableId, int schemaVersion) {
+    public CompletableFuture<?> waitForSchemaAvailability(int tableId, int schemaVersion) {
         return completedFuture(null);
     }
 
     @Override
-    public List<FullTableSchema> tableSchemaVersionsBetween(UUID tableId, HybridTimestamp fromIncluding, HybridTimestamp toIncluding) {
+    public List<FullTableSchema> tableSchemaVersionsBetween(int tableId, HybridTimestamp fromIncluding, HybridTimestamp toIncluding) {
         SchemaDescriptor schemaDescriptor = schemaRegistry.schema();
 
         List<TableColumnDescriptor> columns = schemaDescriptor.columnNames().stream()
@@ -77,7 +76,7 @@ public class DummySchemas implements Schemas {
     }
 
     @Override
-    public List<FullTableSchema> tableSchemaVersionsBetween(UUID tableId, HybridTimestamp fromIncluding, int toIncluding) {
+    public List<FullTableSchema> tableSchemaVersionsBetween(int tableId, HybridTimestamp fromIncluding, int toIncluding) {
         // Returning an empty list makes sure that backward validation never fails, which is what we want before
         // we switch to CatalogService completely.
         return List.of();
diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java
index 30095c12cc..3ec60e170f 100644
--- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java
+++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/LockKey.java
@@ -17,13 +17,12 @@
 
 package org.apache.ignite.internal.tx;
 
-import java.util.UUID;
 import org.apache.ignite.internal.tostring.S;
 
 /** Lock key. */
 public class LockKey {
     /** Context identificator. */
-    private final UUID contextId;
+    private final Object contextId;
 
     /** Entity to lock. */
     private final Object key;
@@ -32,7 +31,7 @@ public class LockKey {
         this(null, key);
     }
 
-    public LockKey(UUID contextId, Object key) {
+    public LockKey(Object contextId, Object key) {
         this.contextId = contextId;
         this.key = key;
     }
@@ -42,7 +41,7 @@ public class LockKey {
      *
      * @return Context identificator.
      */
-    public UUID contextId() {
+    public Object contextId() {
         return contextId;
     }
 
diff --git a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
index a3398facb4..8017646882 100644
--- a/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
+++ b/modules/transactions/src/test/java/org/apache/ignite/internal/tx/TxManagerTest.java
@@ -102,7 +102,7 @@ public class TxManagerTest extends IgniteAbstractTest {
 
         InternalTransaction tx = txManager.begin();
 
-        TablePartitionId tablePartitionId = new TablePartitionId(UUID.randomUUID(), 0);
+        TablePartitionId tablePartitionId = new TablePartitionId(1, 0);
 
         ClusterNode node = mock(ClusterNode.class);
 
diff --git a/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/AbstractTxStateStorageTest.java b/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/AbstractTxStateStorageTest.java
index f1250aa4b4..1e2fd4cf04 100644
--- a/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/AbstractTxStateStorageTest.java
+++ b/modules/transactions/src/testFixtures/java/org/apache/ignite/internal/tx/storage/state/AbstractTxStateStorageTest.java
@@ -59,7 +59,7 @@ import org.junit.jupiter.api.function.Executable;
  * Abstract tx storage test.
  */
 public abstract class AbstractTxStateStorageTest {
-    private final UUID tableId = UUID.randomUUID();
+    private final int tableId = 1;
 
     protected TxStateTableStorage tableStorage;