You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pp...@apache.org on 2022/12/23 09:29:33 UTC

[ignite-3] branch main updated: IGNITE-18211 Sql. Adjust affinity distribution to the new colocation rules (#1434)

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

ppa 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 44baaddc07 IGNITE-18211 Sql. Adjust affinity distribution to the new colocation rules (#1434)
44baaddc07 is described below

commit 44baaddc0723d92d9450358c17bd487979333de8
Author: Pavel Pereslegin <xx...@gmail.com>
AuthorDate: Fri Dec 23 12:29:27 2022 +0300

    IGNITE-18211 Sql. Adjust affinity distribution to the new colocation rules (#1434)
---
 .../internal/sql/engine/ItAggregatesTest.java      |   2 -
 .../ignite/internal/sql/engine/ItSetOpTest.java    |   2 +-
 .../sql/engine/exec/ExecutionServiceImpl.java      |   9 +-
 .../sql/engine/exec/LogicalRelImplementor.java     |  17 ++-
 .../internal/sql/engine/externalize/RelJson.java   |  20 +++-
 .../sql/engine/metadata/AffinityService.java       |  34 ------
 .../sql/engine/schema/SqlSchemaManagerImpl.java    |   7 +-
 .../sql/engine/schema/TableDescriptorImpl.java     |  32 ++----
 .../internal/sql/engine/trait/AffinityAdapter.java |  55 ----------
 .../sql/engine/trait/DistributionFunction.java     |  96 +++++++----------
 .../sql/engine/trait/DistributionTrait.java        |   7 +-
 .../sql/engine/trait/IgniteDistribution.java       |   9 +-
 .../sql/engine/trait/IgniteDistributions.java      |  53 ++++-----
 .../internal/sql/engine/trait/Partitioned.java     |   3 +-
 .../internal/sql/engine/trait/TraitUtils.java      |   7 --
 .../sql/engine/util/HashFunctionFactory.java       |  43 ++++++++
 .../sql/engine/util/HashFunctionFactoryImpl.java   | 120 +++++++++++++++++++++
 .../sql/engine/exec/ExecutionServiceImplTest.java  |   7 +-
 .../planner/AbstractAggregatePlannerTest.java      |  29 ++---
 .../sql/engine/planner/AbstractPlannerTest.java    |  19 ++--
 .../sql/engine/planner/AggregatePlannerTest.java   |  11 +-
 .../engine/planner/HashAggregatePlannerTest.java   |  21 ++--
 .../sql/engine/planner/HashIndexPlannerTest.java   |   5 +-
 .../engine/planner/HashIndexSpoolPlannerTest.java  |  51 +++------
 .../engine/planner/JoinColocationPlannerTest.java  |  53 +++++----
 .../sql/engine/planner/JoinCommutePlannerTest.java |   5 +-
 .../internal/sql/engine/planner/PlannerTest.java   |  22 ++--
 .../sql/engine/planner/SetOpPlannerTest.java       |  31 ++----
 .../engine/planner/SortAggregatePlannerTest.java   |  20 ++--
 .../planner/SortedIndexSpoolPlannerTest.java       | 105 +++++++-----------
 .../sql/engine/planner/TableSpoolPlannerTest.java  |  34 +++---
 .../sql/engine/planner/UnionPlannerTest.java       |  45 ++++----
 .../sql/engine/util/HashFunctionsTest.java         | 111 +++++++++++++++++++
 33 files changed, 576 insertions(+), 509 deletions(-)

diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java
index 9b01c95e2e..656467fe55 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItAggregatesTest.java
@@ -24,7 +24,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 import org.apache.ignite.internal.sql.engine.util.QueryChecker;
 import org.apache.ignite.lang.IgniteException;
 import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
@@ -187,7 +186,6 @@ public class ItAggregatesTest extends AbstractBasicIntegrationTest {
     }
 
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void testColocatedAggregate() {
         sql("CREATE TABLE t1(id INT, val0 VARCHAR, val1 VARCHAR, val2 VARCHAR, PRIMARY KEY(id, val1)) "
                 + "COLOCATE BY (val1)");
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java
index c8aa0c2809..84e9d346be 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSetOpTest.java
@@ -186,8 +186,8 @@ public class ItSetOpTest extends AbstractBasicIntegrationTest {
         assertEquals(2, countIf(rows, r -> r.get(0).equals("Igor1")));
     }
 
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18426")
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void testSetOpColocated() {
         sql("CREATE TABLE emp(empid INTEGER, deptid INTEGER, name VARCHAR, PRIMARY KEY(empid, deptid)) COLOCATE BY (deptid)");
         sql("CREATE TABLE dept(deptid INTEGER, name VARCHAR, PRIMARY KEY(deptid))");
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 1d401738c7..37a243efce 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
@@ -29,7 +29,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Queue;
 import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
@@ -71,6 +70,7 @@ import org.apache.ignite.internal.sql.engine.prepare.QueryPlan;
 import org.apache.ignite.internal.sql.engine.schema.SqlSchemaManager;
 import org.apache.ignite.internal.sql.engine.util.BaseQueryContext;
 import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactoryImpl;
 import org.apache.ignite.internal.sql.engine.util.TypeUtils;
 import org.apache.ignite.internal.storage.DataStorageManager;
 import org.apache.ignite.internal.table.distributed.TableManager;
@@ -149,7 +149,12 @@ public class ExecutionServiceImpl<RowT> implements ExecutionService, TopologyEve
                 taskExecutor,
                 handler,
                 exchangeSrvc,
-                ctx -> new LogicalRelImplementor<>(ctx, cacheId -> Objects::hashCode, mailboxRegistry, exchangeSrvc)
+                ctx -> new LogicalRelImplementor<>(
+                        ctx,
+                        new HashFunctionFactoryImpl<>(sqlSchemaManager, handler),
+                        mailboxRegistry,
+                        exchangeSrvc
+                )
         );
     }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
index a19cd03695..12298c3425 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java
@@ -67,7 +67,6 @@ import org.apache.ignite.internal.sql.engine.exec.rel.SortNode;
 import org.apache.ignite.internal.sql.engine.exec.rel.TableScanNode;
 import org.apache.ignite.internal.sql.engine.exec.rel.TableSpoolNode;
 import org.apache.ignite.internal.sql.engine.exec.rel.UnionAllNode;
-import org.apache.ignite.internal.sql.engine.metadata.AffinityService;
 import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
 import org.apache.ignite.internal.sql.engine.prepare.bounds.SearchBounds;
 import org.apache.ignite.internal.sql.engine.rel.IgniteCorrelatedNestedLoopJoin;
@@ -108,6 +107,7 @@ import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
 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;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
 
 /**
  * Implements a query plan.
@@ -119,7 +119,7 @@ public class LogicalRelImplementor<RowT> implements IgniteRelVisitor<Node<RowT>>
 
     private final ExecutionContext<RowT> ctx;
 
-    private final AffinityService affSrvc;
+    private final HashFunctionFactory<RowT> hashFuncFactory;
 
     private final ExchangeService exchangeSvc;
 
@@ -131,17 +131,17 @@ public class LogicalRelImplementor<RowT> implements IgniteRelVisitor<Node<RowT>>
      * Constructor.
      *
      * @param ctx Root context.
-     * @param affSrvc Affinity service.
+     * @param hashFuncFactory Factory to create a hash function for the row, from which the destination nodes are calculated.
      * @param mailboxRegistry Mailbox registry.
      * @param exchangeSvc Exchange service.
      */
     public LogicalRelImplementor(
             ExecutionContext<RowT> ctx,
-            AffinityService affSrvc,
+            HashFunctionFactory<RowT> hashFuncFactory,
             MailboxRegistry mailboxRegistry,
             ExchangeService exchangeSvc
     ) {
-        this.affSrvc = affSrvc;
+        this.hashFuncFactory = hashFuncFactory;
         this.mailboxRegistry = mailboxRegistry;
         this.exchangeSvc = exchangeSvc;
         this.ctx = ctx;
@@ -154,11 +154,10 @@ public class LogicalRelImplementor<RowT> implements IgniteRelVisitor<Node<RowT>>
     public Node<RowT> visit(IgniteSender rel) {
         IgniteDistribution distribution = rel.distribution();
 
-        Destination<RowT> dest = distribution.destination(ctx, affSrvc, ctx.target());
+        Destination<RowT> dest = distribution.destination(hashFuncFactory, ctx.target());
 
         // Outbox fragment ID is used as exchange ID as well.
-        Outbox<RowT> outbox =
-                new Outbox<>(ctx, exchangeSvc, mailboxRegistry, rel.exchangeId(), rel.targetFragmentId(), dest);
+        Outbox<RowT> outbox = new Outbox<>(ctx, exchangeSvc, mailboxRegistry, rel.exchangeId(), rel.targetFragmentId(), dest);
 
         Node<RowT> input = visit(rel.getInput());
 
@@ -189,7 +188,7 @@ public class LogicalRelImplementor<RowT> implements IgniteRelVisitor<Node<RowT>>
         assert TraitUtils.distribution(rel).getType() == HASH_DISTRIBUTED;
 
         IgniteDistribution distr = rel.distribution();
-        Destination<RowT> dest = distr.destination(ctx, affSrvc, ctx.group(rel.sourceId()));
+        Destination<RowT> dest = distr.destination(hashFuncFactory, ctx.group(rel.sourceId()));
         String localNodeName = ctx.localNode().name();
 
         FilterNode<RowT> node = new FilterNode<>(ctx, r -> Objects.equals(localNodeName, first(dest.targets(r))));
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 c4e90df8d8..92a54957a3 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,6 +37,7 @@ 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;
@@ -110,6 +111,7 @@ import org.apache.ignite.internal.sql.engine.prepare.bounds.RangeBounds;
 import org.apache.ignite.internal.sql.engine.prepare.bounds.SearchBounds;
 import org.apache.ignite.internal.sql.engine.rel.InternalIgniteRel;
 import org.apache.ignite.internal.sql.engine.trait.DistributionFunction;
+import org.apache.ignite.internal.sql.engine.trait.DistributionFunction.AffinityDistribution;
 import org.apache.ignite.internal.sql.engine.trait.DistributionTrait;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
@@ -527,6 +529,13 @@ class RelJson {
 
                 map.put("keys", keys);
 
+                DistributionFunction function = distribution.function();
+
+                if (function.affinity()) {
+                    map.put("zoneId", ((AffinityDistribution) function).zoneId());
+                    map.put("tableId", ((AffinityDistribution) function).tableId().toString());
+                }
+
                 return map;
             default:
                 throw new AssertionError("Unexpected distribution type.");
@@ -657,10 +666,13 @@ class RelJson {
         }
 
         Map<String, Object> map = (Map<String, Object>) distribution;
-        Number cacheId = (Number) map.get("cacheId");
-        if (cacheId != null) {
-            return IgniteDistributions.hash(ImmutableIntList.copyOf((List<Integer>) map.get("keys")),
-                    DistributionFunction.affinity(cacheId.intValue(), cacheId));
+        String tableIdStr = (String) map.get("tableId");
+
+        if (tableIdStr != null) {
+            UUID tableId = UUID.fromString(tableIdStr);
+            Object zoneId = map.get("zoneId");
+
+            return IgniteDistributions.affinity((List<Integer>) map.get("keys"), tableId, zoneId);
         }
 
         return IgniteDistributions.hash(ImmutableIntList.copyOf((List<Integer>) map.get("keys")), DistributionFunction.hash());
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/AffinityService.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/AffinityService.java
deleted file mode 100644
index 1835ec5542..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/metadata/AffinityService.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.sql.engine.metadata;
-
-import java.util.function.ToIntFunction;
-
-/**
- * AffinityService interface.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
- */
-public interface AffinityService {
-    /**
-     * Creates a partition mapping function on the basis of affinity function of cache with given ID.
-     *
-     * @param cacheId Cache ID.
-     * @return Affinity function.
-     */
-    ToIntFunction<Object> affinity(int cacheId);
-}
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 fa15bd3d5c..b62f10b456 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
@@ -49,6 +49,8 @@ import org.apache.ignite.internal.schema.DefaultValueProvider.Type;
 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.sql.engine.trait.IgniteDistribution;
+import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.table.TableImpl;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.util.IgniteSpinBusyLock;
@@ -364,8 +366,11 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager {
             colocationColumns.add(column.columnOrder());
         }
 
+        // TODO Use the actual zone ID after implementing https://issues.apache.org/jira/browse/IGNITE-18426.
+        IgniteDistribution distribution = IgniteDistributions.affinity(colocationColumns, table.tableId(), table.tableId());
+
         return new IgniteTableImpl(
-                new TableDescriptorImpl(colDescriptors, colocationColumns),
+                new TableDescriptorImpl(colDescriptors, distribution),
                 table.internalTable(),
                 schemaRegistry
         );
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
index c06d7897e3..e6de86ebc3 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/TableDescriptorImpl.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.sql.engine.schema;
 
 import static org.apache.ignite.internal.sql.engine.util.TypeUtils.native2relationalType;
 
-import it.unimi.dsi.fastutil.ints.IntList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -35,7 +34,6 @@ import org.apache.calcite.sql2rel.InitializerContext;
 import org.apache.calcite.sql2rel.NullInitializerExpressionFactory;
 import org.apache.calcite.util.ImmutableBitSet;
 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.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.util.Commons;
 import org.apache.ignite.internal.sql.engine.util.TypeUtils;
@@ -48,33 +46,24 @@ import org.jetbrains.annotations.Nullable;
 public class TableDescriptorImpl extends NullInitializerExpressionFactory implements TableDescriptor {
     private static final ColumnDescriptor[] DUMMY = new ColumnDescriptor[0];
 
-    // TODO: IGNITE-18211
-    // Current affinity distribution is designed to be described by cacheId and some identity.
-    // This is subject to change in IGNITE-18211, but for now let's use a dummy description
-    // to create unique affinity for every table.
-    private static final int DUMMY_CACHE_ID = 0;
-
-    private final UUID dummyAffinityIdentity = UUID.randomUUID();
-
     private final ColumnDescriptor[] descriptors;
 
     private final Map<String, ColumnDescriptor> descriptorsMap;
 
-    private final IntList colocationColumns;
-
     private final ImmutableBitSet insertFields;
 
     private final ImmutableBitSet keyFields;
 
+    private final IgniteDistribution distribution;
+
     /**
      * Constructor.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+     *
+     * @param columnDescriptors Column descriptors.
+     * @param distribution Distribution specification.
      */
-    public TableDescriptorImpl(
-            List<ColumnDescriptor> columnDescriptors,
-            IntList colocationColumns
-    ) {
-        this.colocationColumns = colocationColumns;
+    public TableDescriptorImpl(List<ColumnDescriptor> columnDescriptors, IgniteDistribution distribution) {
+        this.distribution = distribution;
 
         ImmutableBitSet.Builder keyFieldsBuilder = ImmutableBitSet.builder();
 
@@ -141,12 +130,7 @@ public class TableDescriptorImpl extends NullInitializerExpressionFactory implem
     /** {@inheritDoc} */
     @Override
     public IgniteDistribution distribution() {
-        // TODO: IGNITE-18211
-        // affinity function should be redesigned in IGNITE-18211, but for now
-        // let's create a unique affinity distribution per every table. This is required
-        // for aggregation push down, because we need to verify that grouping columns
-        // is a superset of distribution keys.
-        return IgniteDistributions.affinity(colocationColumns, DUMMY_CACHE_ID, dummyAffinityIdentity);
+        return distribution;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/AffinityAdapter.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/AffinityAdapter.java
deleted file mode 100644
index 8d98ca0382..0000000000
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/AffinityAdapter.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.sql.engine.trait;
-
-import java.util.function.ToIntFunction;
-import org.apache.ignite.internal.sql.engine.exec.RowHandler;
-import org.apache.ignite.internal.util.IgniteUtils;
-
-/**
- * AffinityAdapter.
- * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
- */
-final class AffinityAdapter<RowT> implements ToIntFunction<RowT> {
-    private final ToIntFunction<Object> affinity;
-
-    private final int[] keys;
-
-    private final RowHandler<RowT> hndlr;
-
-    /**
-     * Constructor.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
-     */
-    AffinityAdapter(ToIntFunction<Object> affinity, int[] keys, RowHandler<RowT> hndlr) {
-        this.affinity = affinity;
-        this.keys = keys;
-        this.hndlr = hndlr;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public int applyAsInt(RowT r) {
-        int hash = 0;
-        for (int i = 0; i < keys.length; i++) {
-            hash = 31 * hash + affinity.applyAsInt(hndlr.get(keys[i], r));
-        }
-
-        return IgniteUtils.safeAbs(hash);
-    }
-}
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 697a54502e..d9ebc797fb 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
@@ -17,19 +17,18 @@
 
 package org.apache.ignite.internal.sql.engine.trait;
 
-import static org.apache.ignite.internal.sql.engine.Stubs.intFoo;
 import static org.apache.ignite.internal.util.CollectionUtils.first;
 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;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
-import org.apache.ignite.internal.sql.engine.metadata.AffinityService;
 import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
 import org.apache.ignite.internal.util.IgniteUtils;
 
 /**
@@ -62,21 +61,19 @@ public abstract class DistributionFunction {
         return false;
     }
 
-    public static DistributionFunction affinity(int cacheId, Object identity) {
-        return new AffinityDistribution(cacheId, identity);
+    public static DistributionFunction affinity(UUID tableId, Object zoneId) {
+        return new AffinityDistribution(tableId, zoneId);
     }
 
     /**
      * Creates a destination based on this function algorithm, given nodes mapping and given distribution keys.
      *
-     * @param ctx             Execution context.
-     * @param affinityService Affinity function source.
+     * @param hashFuncFactory Factory to create a hash function for the row, from which the destination nodes are calculated.
      * @param group           Target mapping.
      * @param keys            Distribution keys.
      * @return Destination function.
      */
-    abstract <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affinityService,
-            ColocationGroup group, ImmutableIntList keys);
+    abstract <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup group, ImmutableIntList keys);
 
     /**
      * Get function name. This name used for equality checking and in {@link RelNode#getDigest()}.
@@ -137,16 +134,12 @@ public abstract class DistributionFunction {
         }
 
         return f0 instanceof AffinityDistribution && f1 instanceof AffinityDistribution
-                && Objects.equals(((AffinityDistribution) f0).identity(), ((AffinityDistribution) f1).identity());
+                && Objects.equals(((AffinityDistribution) f0).zoneId(), ((AffinityDistribution) f1).zoneId());
     }
 
     private static final class AnyDistribution extends DistributionFunction {
         public static final DistributionFunction INSTANCE = new AnyDistribution();
 
-        public static DistributionFunction affinity(int cacheId, Object identity) {
-            return new AffinityDistribution(cacheId, identity);
-        }
-
         /** {@inheritDoc} */
         @Override
         public RelDistribution.Type type() {
@@ -155,8 +148,7 @@ public abstract class DistributionFunction {
 
         /** {@inheritDoc} */
         @Override
-        public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affinityService,
-                ColocationGroup m, ImmutableIntList k) {
+        public <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup m, ImmutableIntList k) {
             throw new IllegalStateException();
         }
     }
@@ -172,8 +164,7 @@ public abstract class DistributionFunction {
 
         /** {@inheritDoc} */
         @Override
-        public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affinityService,
-                ColocationGroup m, ImmutableIntList k) {
+        public <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup m, ImmutableIntList k) {
             assert m != null && !nullOrEmpty(m.nodeNames());
 
             return new AllNodes<>(m.nodeNames());
@@ -191,8 +182,7 @@ public abstract class DistributionFunction {
 
         /** {@inheritDoc} */
         @Override
-        public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affinityService,
-                ColocationGroup m, ImmutableIntList k) {
+        public <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup m, ImmutableIntList k) {
             assert m != null && !nullOrEmpty(m.nodeNames());
 
             return new RandomNode<>(m.nodeNames());
@@ -210,8 +200,7 @@ public abstract class DistributionFunction {
 
         /** {@inheritDoc} */
         @Override
-        public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affinityService,
-                ColocationGroup m, ImmutableIntList k) {
+        public <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup m, ImmutableIntList k) {
             if (m == null || m.nodeNames() == null || m.nodeNames().size() != 1) {
                 throw new IllegalStateException();
             }
@@ -220,7 +209,7 @@ public abstract class DistributionFunction {
         }
     }
 
-    private static final class HashDistribution extends DistributionFunction {
+    private static class HashDistribution extends DistributionFunction {
         public static final DistributionFunction INSTANCE = new HashDistribution();
 
         /** {@inheritDoc} */
@@ -231,8 +220,7 @@ public abstract class DistributionFunction {
 
         /** {@inheritDoc} */
         @Override
-        public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affSrvc,
-                ColocationGroup m, ImmutableIntList k) {
+        public <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup m, ImmutableIntList k) {
             assert m != null && !nullOrEmpty(m.assignments()) && !k.isEmpty();
 
             List<List<String>> assignments = m.assignments();
@@ -243,27 +231,31 @@ public abstract class DistributionFunction {
                 }
             }
 
-            AffinityAdapter<RowT> affinity = new AffinityAdapter<>(affSrvc.affinity(intFoo()/*CU.UNDEFINED_CACHE_ID*/), k.toIntArray(),
-                    ctx.rowHandler());
+            return destination(assignments, hashFuncFactory, k.toIntArray());
+        }
 
-            return new Partitioned<>(assignments, affinity);
+        protected <RowT> Destination<RowT> destination(List<List<String>> assignments, HashFunctionFactory<RowT> funcFactory, int[] keys) {
+            return new Partitioned<>(assignments, funcFactory.create(keys));
         }
     }
 
-    private static final class AffinityDistribution extends DistributionFunction {
-        private final int cacheId;
+    /**
+     * Affinity distribution.
+     */
+    public static final class AffinityDistribution extends HashDistribution {
+        private final UUID tableId;
 
-        private final Object identity;
+        private final Object zoneId;
 
         /**
          * Constructor.
          *
-         * @param cacheId  Cache ID.
-         * @param identity Affinity identity key.
+         * @param tableId Table ID.
+         * @param zoneId  Distribution zone ID.
          */
-        private AffinityDistribution(int cacheId, Object identity) {
-            this.cacheId = cacheId;
-            this.identity = identity;
+        private AffinityDistribution(UUID tableId, Object zoneId) {
+            this.zoneId = zoneId;
+            this.tableId = tableId;
         }
 
         /** {@inheritDoc} */
@@ -272,39 +264,23 @@ public abstract class DistributionFunction {
             return true;
         }
 
-        /** {@inheritDoc} */
-        @Override
-        public RelDistribution.Type type() {
-            return RelDistribution.Type.HASH_DISTRIBUTED;
+        public UUID tableId() {
+            return tableId;
         }
 
-        /** {@inheritDoc} */
-        @Override
-        public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ctx, AffinityService affSrvc,
-                ColocationGroup m, ImmutableIntList k) {
-            assert m != null && !nullOrEmpty(m.assignments()) && k.size() == 1;
-
-            List<List<String>> assignments = m.assignments();
-
-            if (IgniteUtils.assertionsEnabled()) {
-                for (List<String> assignment : assignments) {
-                    assert nullOrEmpty(assignment) || assignment.size() == 1;
-                }
-            }
-
-            AffinityAdapter<RowT> affinity = new AffinityAdapter<>(affSrvc.affinity(cacheId), k.toIntArray(), ctx.rowHandler());
-
-            return new Partitioned<>(assignments, affinity);
+        public Object zoneId() {
+            return zoneId;
         }
 
-        public Object identity() {
-            return identity;
+        @Override
+        protected <RowT> Destination<RowT> destination(List<List<String>> assignments, HashFunctionFactory<RowT> funcFactory, int[] keys) {
+            return new Partitioned<>(assignments, funcFactory.create(keys, tableId));
         }
 
         /** {@inheritDoc} */
         @Override
         protected String name0() {
-            return "affinity[identity=" + identity + ", cacheId=" + cacheId + ']';
+            return "affinity[tableId=" + tableId + ", zoneId=" + zoneId + ']';
         }
     }
 }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionTrait.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionTrait.java
index e8ba3172e6..f73f16c70e 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionTrait.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/DistributionTrait.java
@@ -33,9 +33,8 @@ import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.Mappings;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
-import org.apache.ignite.internal.sql.engine.metadata.AffinityService;
 import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
 
 /**
  * Description of the physical distribution of a relational expression.
@@ -106,8 +105,8 @@ public final class DistributionTrait implements IgniteDistribution {
 
     /** {@inheritDoc} */
     @Override
-    public <RowT> Destination<RowT> destination(ExecutionContext<RowT> ectx, AffinityService affSrvc, ColocationGroup target) {
-        return function.destination(ectx, affSrvc, target, keys);
+    public <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup target) {
+        return function.destination(hashFuncFactory, target, keys);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistribution.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistribution.java
index 88d20c1b7c..99cfd8329a 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistribution.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/IgniteDistribution.java
@@ -20,9 +20,8 @@ package org.apache.ignite.internal.sql.engine.trait;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.mapping.Mappings;
-import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
-import org.apache.ignite.internal.sql.engine.metadata.AffinityService;
 import org.apache.ignite.internal.sql.engine.metadata.ColocationGroup;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
 
 /**
  * Ignite distribution trait.
@@ -36,13 +35,11 @@ public interface IgniteDistribution extends RelDistribution {
     /**
      * Creates a destination based on this function algorithm, given nodes mapping and distribution keys.
      *
-     * @param ectx            Execution context.
-     * @param affinityService Affinity function source.
+     * @param hashFuncFactory Factory to create a hash function for the row, from which the destination nodes are calculated.
      * @param targetGroup     Target mapping.
      * @return Destination function.
      */
-    <RowT> Destination<RowT> destination(ExecutionContext<RowT> ectx, AffinityService affinityService,
-            ColocationGroup targetGroup);
+    <RowT> Destination<RowT> destination(HashFunctionFactory<RowT> hashFuncFactory, ColocationGroup targetGroup);
 
     /** {@inheritDoc} */
     @Override
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 6bd5e060b8..11e865f67c 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,6 +18,7 @@
 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;
 
@@ -63,50 +64,35 @@ public class IgniteDistributions {
     }
 
     /**
-     * Affinity.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+     * Creates an affinity distribution that takes into account the zone ID and calculates the destinations
+     * based on a hash function which takes into account the key field types of the row.
      *
-     * @param key       Affinity key.
-     * @param cacheName Affinity cache name.
-     * @param identity  Affinity identity key.
+     * @param key     Affinity key ordinal.
+     * @param tableId Table ID.
+     * @param zoneId  Distribution zone ID.
      * @return Affinity distribution.
      */
-    public static IgniteDistribution affinity(int key, String cacheName, Object identity) {
-        // TODO: fix cacheId
-        return affinity(key, 0, identity);
+    public static IgniteDistribution affinity(int key, UUID tableId, Object zoneId) {
+        return hash(ImmutableIntList.of(key), DistributionFunction.affinity(tableId, zoneId));
     }
 
     /**
-     * Affinity.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+     * Creates an affinity distribution that takes into account the zone ID and calculates the destinations
+     * based on a hash function which takes into account the key field types of the row.
      *
-     * @param key      Affinity key.
-     * @param cacheId  Affinity cache ID.
-     * @param identity Affinity identity key.
+     * @param keys    Affinity keys ordinals.
+     * @param tableId Table ID.
+     * @param zoneId  Distribution zone ID.
      * @return Affinity distribution.
      */
-    public static IgniteDistribution affinity(int key, int cacheId, Object identity) {
-        return hash(ImmutableIntList.of(key), DistributionFunction.affinity(cacheId, identity));
+    public static IgniteDistribution affinity(List<Integer> keys, UUID tableId, Object zoneId) {
+        return hash(keys, DistributionFunction.affinity(tableId, zoneId));
     }
 
     /**
-     * Affinity.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+     * Creates a hash distribution that calculates destinations based on a composite hash of key field values of the row.
      *
-     * @param keys     Affinity keys.
-     * @param cacheId  Affinity cache ID.
-     * @param identity Affinity identity key.
-     * @return Affinity distribution.
-     */
-    public static IgniteDistribution affinity(List<Integer> keys, int cacheId, Object identity) {
-        return hash(keys, DistributionFunction.affinity(cacheId, identity));
-    }
-
-    /**
-     * Hash.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
-     *
-     * @param keys Distribution keys.
+     * @param keys Distribution keys ordinals.
      * @return Hash distribution.
      */
     public static IgniteDistribution hash(List<Integer> keys) {
@@ -114,10 +100,9 @@ public class IgniteDistributions {
     }
 
     /**
-     * Hash.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+     * Creates a hash distribution that calculates destinations based on a composite hash of key field values of the row.
      *
-     * @param keys     Distribution keys.
+     * @param keys     Distribution keys ordinals.
      * @param function Specific hash function.
      * @return Hash distribution.
      */
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/Partitioned.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/Partitioned.java
index e5f13fb48b..4403861d7c 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/Partitioned.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/Partitioned.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.function.ToIntFunction;
 import java.util.stream.Collectors;
+import org.apache.ignite.internal.util.IgniteUtils;
 
 /**
  * Partitioned.
@@ -43,7 +44,7 @@ public final class Partitioned<RowT> implements Destination<RowT> {
     /** {@inheritDoc} */
     @Override
     public List<String> targets(RowT row) {
-        return assignments.get(partFun.applyAsInt(row) % assignments.size());
+        return assignments.get(IgniteUtils.safeAbs(partFun.applyAsInt(row) % assignments.size()));
     }
 
     /** {@inheritDoc} */
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/TraitUtils.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/TraitUtils.java
index 67e051cab1..b309a9ce64 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/TraitUtils.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/trait/TraitUtils.java
@@ -184,13 +184,6 @@ public class TraitUtils {
             return rel;
         }
 
-        // TODO: remove in IGNITE-18211
-        // currently affinity function is not implemented properly,
-        // thus rehashing by affinity should by disabled for a while
-        if (toTrait.function().affinity()) {
-            return null;
-        }
-
         RelTraitSet traits = rel.getTraitSet().replace(toTrait);
         if (fromTrait.getType() == BROADCAST_DISTRIBUTED && toTrait.getType() == HASH_DISTRIBUTED) {
             return new IgniteTrimExchange(rel.getCluster(), traits, rel, toTrait);
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
new file mode 100644
index 0000000000..077b8aa88c
--- /dev/null
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine.util;
+
+import java.util.UUID;
+import java.util.function.ToIntFunction;
+
+/**
+ * Factory for creating a function to calculate the hash of the specified fields of the row.
+ */
+public interface HashFunctionFactory<T> {
+    /**
+     * Creates a hash function to compute a composite hash of a row, given the values of the fields.
+     *
+     * @param fields Field ordinals of the row from which the hash is to be calculated.
+     * @return Function to compute a composite hash of a row, given the values of the fields.
+     */
+    ToIntFunction<T> create(int[] fields);
+
+    /**
+     * Creates a hash function to compute a composite hash of a row, given the types and values of the fields.
+     *
+     * @param fields Field ordinals of the row from which the hash is to be calculated.
+     * @param tableId Table ID.
+     * @return Function to compute a composite hash of a row, given the types and values of the fields.
+     */
+    ToIntFunction<T> create(int[] fields, UUID 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
new file mode 100644
index 0000000000..9b4ae082b2
--- /dev/null
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/HashFunctionFactoryImpl.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine.util;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.schema.ColumnDescriptor;
+import org.apache.ignite.internal.sql.engine.schema.SqlSchemaManager;
+import org.apache.ignite.internal.sql.engine.schema.TableDescriptor;
+import org.apache.ignite.internal.util.ColocationUtils;
+import org.apache.ignite.internal.util.HashCalculator;
+
+/**
+ * Factory for creating a function to calculate the hash of the specified fields of a row.
+ */
+public class HashFunctionFactoryImpl<T> implements HashFunctionFactory<T> {
+    private final SqlSchemaManager sqlSchemaManager;
+    private final RowHandler<T> rowHandler;
+
+    public HashFunctionFactoryImpl(SqlSchemaManager sqlSchemaManager, RowHandler<T> rowHandler) {
+        this.sqlSchemaManager = sqlSchemaManager;
+        this.rowHandler = rowHandler;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public ToIntFunction<T> create(int[] fields, UUID tableId) {
+        int fieldCnt = fields.length;
+        NativeType[] fieldTypes = new NativeType[fieldCnt];
+        TableDescriptor tblDesc = sqlSchemaManager.tableById(tableId, -1).descriptor();
+        List<Integer> colocationColumns = tblDesc.distribution().getKeys();
+
+        assert colocationColumns.size() == fieldCnt : "fieldsCount=" + fieldCnt + ", colocationColumns=" + colocationColumns;
+
+        for (int i = 0; i < fieldCnt; i++) {
+            ColumnDescriptor colDesc = tblDesc.columnDescriptor(colocationColumns.get(i));
+
+            fieldTypes[i] = colDesc.physicalType();
+        }
+
+        return new TypesAwareHashFunction<>(fields, fieldTypes, rowHandler);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public ToIntFunction<T> create(int[] fields) {
+        return new SimpleHashFunction<>(fields, rowHandler);
+    }
+
+    /**
+     * Computes a composite hash of a row, given the values of the fields.
+     */
+    static class SimpleHashFunction<T> implements ToIntFunction<T> {
+        private final int[] fields;
+        private final RowHandler<T> rowHandler;
+
+        SimpleHashFunction(int[] fields, RowHandler<T> rowHandler) {
+            this.fields = fields;
+            this.rowHandler = rowHandler;
+        }
+
+        @Override
+        public int applyAsInt(T row) {
+            int hash = 0;
+
+            for (int idx : fields) {
+                hash = 31 * hash + Objects.hashCode(rowHandler.get(idx, row));
+            }
+
+            return hash;
+        }
+    }
+
+    /**
+     * Computes a composite hash of a row, given the types and values of the fields.
+     */
+    static class TypesAwareHashFunction<T> implements ToIntFunction<T> {
+        private final int[] fields;
+        private final NativeType[] fieldTypes;
+        private final RowHandler<T> rowHandler;
+
+        TypesAwareHashFunction(int[] fields, NativeType[] fieldTypes, RowHandler<T> rowHandler) {
+            this.fields = fields;
+            this.fieldTypes = fieldTypes;
+            this.rowHandler = rowHandler;
+        }
+
+        @Override
+        public int applyAsInt(T row) {
+            HashCalculator hashCalc = new HashCalculator();
+
+            for (int i = 0; i < fields.length; i++) {
+                Object obj = rowHandler.get(fields[i], row);
+
+                ColocationUtils.append(hashCalc, obj, fieldTypes[i]);
+            }
+
+            return hashCalc.hash();
+        }
+    }
+}
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 4931e073de..92e963db86 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
@@ -37,7 +37,6 @@ import static org.mockito.Mockito.when;
 
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
 import java.util.Queue;
 import java.util.UUID;
 import java.util.concurrent.CompletionException;
@@ -77,6 +76,8 @@ import org.apache.ignite.internal.sql.engine.trait.IgniteDistribution;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
 import org.apache.ignite.internal.sql.engine.util.BaseQueryContext;
 import org.apache.ignite.internal.sql.engine.util.Commons;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactory;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactoryImpl;
 import org.apache.ignite.internal.testframework.IgniteTestUtils.RunnableX;
 import org.apache.ignite.internal.util.ArrayUtils;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
@@ -511,7 +512,9 @@ public class ExecutionServiceImplTest {
                     MailboxRegistry mailboxRegistry,
                     ExchangeService exchangeService
             ) {
-                return new LogicalRelImplementor<>(ctx, cacheId -> Objects::hashCode, mailboxRegistry, exchangeService) {
+                HashFunctionFactory<Object[]> funcFactory = new HashFunctionFactoryImpl<>(mock(SqlSchemaManager.class), ctx.rowHandler());
+
+                return new LogicalRelImplementor<>(ctx, funcFactory, mailboxRegistry, exchangeService) {
                     @Override
                     public Node<Object[]> visit(IgniteTableScan rel) {
                         return new ScanNode<>(ctx, dataset) {
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 a684456741..547320af33 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
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.sql.engine.planner;
 
+import java.util.UUID;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-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.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeSystem;
@@ -36,21 +36,17 @@ public class AbstractAggregatePlannerTest extends AbstractPlannerTest {
     protected TestTable createBroadcastTable(String tblName) {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        TestTable tbl = new TestTable(
+        return createTable(tblName,
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("VAL0", f.createJavaType(Integer.class))
                         .add("VAL1", f.createJavaType(Integer.class))
                         .add("GRP0", f.createJavaType(Integer.class))
                         .add("GRP1", f.createJavaType(Integer.class))
-                        .build(), tblName) {
-
-            @Override
-            public IgniteDistribution distribution() {
-                return IgniteDistributions.broadcast();
-            }
-        };
-        return tbl;
+                        .build(),
+                DEFAULT_TBL_SIZE,
+                IgniteDistributions.broadcast()
+        );
     }
 
     /**
@@ -62,19 +58,16 @@ public class AbstractAggregatePlannerTest extends AbstractPlannerTest {
     protected AbstractPlannerTest.TestTable createAffinityTable(String tblName) {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        return new TestTable(
+        return createTable(tblName,
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("VAL0", f.createJavaType(Integer.class))
                         .add("VAL1", f.createJavaType(Integer.class))
                         .add("GRP0", f.createJavaType(Integer.class))
                         .add("GRP1", f.createJavaType(Integer.class))
-                        .build(), tblName) {
-
-            @Override
-            public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "test", "hash");
-            }
-        };
+                        .build(),
+                DEFAULT_TBL_SIZE,
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
     }
 }
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 e483175010..9d6fb2e7b1 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
@@ -141,6 +141,8 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
 
     protected static final String DEFAULT_SCHEMA = "PUBLIC";
 
+    protected static final int DEFAULT_ZONE_ID = 0;
+
     /** Last error message. */
     private String lastErrorMsg;
 
@@ -397,15 +399,12 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
         };
     }
 
-    protected static void createTable(IgniteSchema schema, String name, RelDataType type, IgniteDistribution distr) {
-        TestTable table = new TestTable(type, name) {
-            @Override
-            public IgniteDistribution distribution() {
-                return distr;
-            }
-        };
+    protected static TestTable createTable(IgniteSchema schema, String name, RelDataType type, IgniteDistribution distr) {
+        TestTable table = createTable(name, type, DEFAULT_TBL_SIZE, distr);
 
         schema.addTable(table);
+
+        return table;
     }
 
     /**
@@ -463,7 +462,11 @@ public abstract class AbstractPlannerTest extends IgniteAbstractTest {
             b.add((String) fields[i], TYPE_FACTORY.createJavaType((Class<?>) fields[i + 1]));
         }
 
-        return new TestTable(name, b.build(), size) {
+        return createTable(name, b.build(), size, distr);
+    }
+
+    protected static TestTable createTable(String name, RelDataType type, int size, IgniteDistribution distr) {
+        return new TestTable(name, type, size) {
             @Override
             public IgniteDistribution distribution() {
                 return distr;
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java
index 45abe1e4de..a281edf709 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AggregatePlannerTest.java
@@ -26,6 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.math.BigDecimal;
+import java.util.UUID;
 import java.util.stream.Stream;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.SingleRel;
@@ -342,14 +343,14 @@ public class AggregatePlannerTest extends AbstractAggregatePlannerTest {
     public void colocated(AggregateAlgorithm algo) throws Exception {
         IgniteSchema schema = createSchema(
                 createTable(
-                        "EMP", IgniteDistributions.affinity(1, "emp", "hash"),
+                        "EMP", IgniteDistributions.affinity(1, UUID.randomUUID(), DEFAULT_ZONE_ID),
                         "EMPID", Integer.class,
                         "DEPTID", Integer.class,
                         "NAME", String.class,
                         "SALARY", Integer.class
                 ).addIndex("DEPTID", 1),
                 createTable(
-                        "DEPT", IgniteDistributions.affinity(0, "dept", "hash"),
+                        "DEPT", IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                         "DEPTID", Integer.class,
                         "NAME", String.class
                 ).addIndex("DEPTID", 0)
@@ -358,7 +359,7 @@ public class AggregatePlannerTest extends AbstractAggregatePlannerTest {
         String sql = "SELECT SUM(SALARY) FROM emp GROUP BY deptid";
 
         assertPlan(sql, schema, hasChildThat(isInstanceOf(algo.colocated)
-                        .and(hasDistribution(IgniteDistributions.affinity(0, null, "hash")))),
+                        .and(hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)))),
                 algo.rulesToDisable);
 
         sql = "SELECT dept.deptid, agg.cnt "
@@ -366,8 +367,8 @@ public class AggregatePlannerTest extends AbstractAggregatePlannerTest {
                 + "JOIN (SELECT deptid, COUNT(*) AS cnt FROM emp GROUP BY deptid) AS agg ON dept.deptid = agg.deptid";
 
         assertPlan(sql, schema, hasChildThat(isInstanceOf(Join.class)
-                        .and(input(0, hasDistribution(IgniteDistributions.affinity(0, null, "hash"))))
-                        .and(input(1, hasDistribution(IgniteDistributions.affinity(0, null, "hash"))))),
+                        .and(input(0, hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID))))
+                        .and(input(1, hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID))))),
                 algo.rulesToDisable);
     }
 
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashAggregatePlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashAggregatePlannerTest.java
index f51674c6eb..bf4b8be044 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashAggregatePlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/HashAggregatePlannerTest.java
@@ -21,6 +21,7 @@ import static org.apache.ignite.internal.util.CollectionUtils.first;
 import static org.hamcrest.MatcherAssert.assertThat;
 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.calcite.sql.fun.SqlAvgAggFunction;
@@ -29,7 +30,6 @@ import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
 import org.apache.ignite.internal.sql.engine.rel.agg.IgniteMapHashAggregate;
 import org.apache.ignite.internal.sql.engine.rel.agg.IgniteReduceHashAggregate;
 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.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeSystem;
@@ -51,22 +51,17 @@ public class HashAggregatePlannerTest extends AbstractAggregatePlannerTest {
     public void subqueryWithAggregate() throws Exception {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        TestTable employer = new TestTable(
+        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
+
+        createTable(publicSchema,
+                "EMPS",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
-                        .build(), "EMPS") {
-
-            @Override
-            public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Employers", "hash");
-            }
-        };
-
-        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
-
-        publicSchema.addTable(employer);
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
 
         String sql = "SELECT * FROM emps WHERE emps.salary = (SELECT AVG(emps.salary) FROM emps)";
 
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 112bd4ca89..9f724d8ecc 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,6 +23,7 @@ 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;
@@ -42,7 +43,7 @@ public class HashIndexPlannerTest extends AbstractPlannerTest {
 
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(0, "default", "hash"),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID", Integer.class,
                 "VAL", Integer.class
         );
@@ -69,7 +70,7 @@ public class HashIndexPlannerTest extends AbstractPlannerTest {
 
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(0, "default", "hash"),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "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 9ae4feffed..5bb45b3660 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,6 +23,7 @@ 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;
@@ -50,37 +51,19 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
         IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        publicSchema.addTable(
-                new TestTable(
-                        new RelDataTypeFactory.Builder(f)
-                                .add("ID", f.createJavaType(Integer.class))
-                                .add("JID", f.createJavaType(Integer.class))
-                                .add("VAL", f.createJavaType(String.class))
-                                .build(), "T0") {
-
-                    @Override
-                    public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T0", "hash");
-                    }
-                }
-                        .addIndex("t0_jid_idx", 1, 0)
-        );
+        createTable(publicSchema, "T0", 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));
 
-        publicSchema.addTable(
-                new TestTable(
-                        new RelDataTypeFactory.Builder(f)
-                                .add("ID", f.createJavaType(Integer.class))
-                                .add("JID", f.createJavaType(Integer.class))
-                                .add("VAL", f.createJavaType(String.class))
-                                .build(), "T1") {
-
-                    @Override
-                    public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T1", "hash");
-                    }
-                }
-                        .addIndex("t1_jid_idx", 1, 0)
-        );
+        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))
+                .addIndex("t1_jid_idx", 1, 0);
 
         String sql = "select * "
                 + "from t0 "
@@ -122,7 +105,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T0", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
                     }
                 }
         );
@@ -138,7 +121,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T1", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
                     }
                 }
                         .addIndex("t1_jid0_idx", 1, 0)
@@ -185,7 +168,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T0", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
                     }
                 }
         );
@@ -200,7 +183,7 @@ public class HashIndexSpoolPlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T1", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), 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 0c207b3ff4..9ada56654b 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
@@ -21,12 +21,11 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.notNullValue;
-import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.hasSize;
 
 import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
+import java.util.UUID;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelNode;
@@ -35,10 +34,11 @@ import org.apache.ignite.internal.sql.engine.rel.IgniteExchange;
 import org.apache.ignite.internal.sql.engine.rel.IgniteIndexScan;
 import org.apache.ignite.internal.sql.engine.rel.IgniteMergeJoin;
 import org.apache.ignite.internal.sql.engine.rel.IgniteRel;
+import org.apache.ignite.internal.sql.engine.rel.IgniteSort;
+import org.apache.ignite.internal.sql.engine.rel.IgniteTableScan;
 import org.apache.ignite.internal.sql.engine.schema.IgniteIndex;
 import org.apache.ignite.internal.sql.engine.schema.IgniteSchema;
 import org.apache.ignite.internal.sql.engine.trait.IgniteDistributions;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
@@ -52,7 +52,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
     public void joinSameTableSimpleAff() throws Exception {
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(0, "default", "hash"),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID", Integer.class,
                 "VAL", String.class
         );
@@ -84,7 +84,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
     public void joinSameTableComplexAff() throws Exception {
         TestTable tbl = createTable(
                 "TEST_TBL",
-                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), ThreadLocalRandom.current().nextInt(), "hash"),
+                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -111,15 +111,16 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
     }
 
     /**
-     * Re-hashing based on simple affinity is possible, so bigger table with complex affinity should be sended to the smaller one.
+     * Re-hashing based on simple affinity.
+     *
+     * <p>The smaller table should be sent to the bigger one.
      */
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void joinComplexToSimpleAff() throws Exception {
         TestTable complexTbl = createTable(
                 "COMPLEX_TBL",
                 2 * DEFAULT_TBL_SIZE,
-                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), ThreadLocalRandom.current().nextInt(), "hash"),
+                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -130,8 +131,9 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
         TestTable simpleTbl = createTable(
                 "SIMPLE_TBL",
                 DEFAULT_TBL_SIZE,
-                IgniteDistributions.affinity(0, "default", "hash"),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID", Integer.class,
+                "ID2", Integer.class,
                 "VAL", String.class
         );
 
@@ -141,7 +143,7 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
 
         String sql = "select count(*) "
                 + "from COMPLEX_TBL t1 "
-                + "join SIMPLE_TBL t2 on t1.id1 = t2.id";
+                + "join SIMPLE_TBL t2 on t1.id1 = t2.id and t1.id2 = t2.id2";
 
         RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin");
 
@@ -156,19 +158,23 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
                 && ((IgniteRel) node).distribution().function().affinity());
 
         assertThat(invalidPlanMsg, exchanges, hasSize(1));
-        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0), instanceOf(IgniteIndexScan.class));
-        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0)
-                .getTable().unwrap(TestTable.class), equalTo(complexTbl));
+        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0), instanceOf(IgniteSort.class));
+        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0).getInput(0), instanceOf(IgniteTableScan.class));
+        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0).getInput(0)
+                .getTable().unwrap(TestTable.class), equalTo(simpleTbl));
     }
 
     /**
-     * Re-hashing for complex affinity is not supported.
+     * Re-hashing for complex affinity.
+     *
+     * <p>The smaller table should be sent to the bigger one.
      */
     @Test
     public void joinComplexToComplexAffWithDifferentOrder() throws Exception {
         TestTable complexTblDirect = createTable(
                 "COMPLEX_TBL_DIRECT",
-                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), ThreadLocalRandom.current().nextInt(), "hash"),
+                2 * DEFAULT_TBL_SIZE,
+                IgniteDistributions.affinity(ImmutableIntList.of(0, 1), UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -179,7 +185,8 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
 
         TestTable complexTblIndirect = createTable(
                 "COMPLEX_TBL_INDIRECT",
-                IgniteDistributions.affinity(ImmutableIntList.of(1, 0), ThreadLocalRandom.current().nextInt(), "hash"),
+                DEFAULT_TBL_SIZE,
+                IgniteDistributions.affinity(ImmutableIntList.of(1, 0), UUID.randomUUID(), DEFAULT_ZONE_ID),
                 "ID1", Integer.class,
                 "ID2", Integer.class,
                 "VAL", String.class
@@ -196,11 +203,19 @@ public class JoinColocationPlannerTest extends AbstractPlannerTest {
 
         RelNode phys = physicalPlan(sql, schema, "NestedLoopJoinConverter", "CorrelatedNestedLoopJoin");
 
-        IgniteMergeJoin exchange = findFirstNode(phys, node -> node instanceof IgniteExchange
-                && ((IgniteRel) node).distribution().function().affinity());
+        IgniteMergeJoin join = findFirstNode(phys, byClass(IgniteMergeJoin.class));
 
         String invalidPlanMsg = "Invalid plan:\n" + RelOptUtil.toString(phys);
 
-        assertThat(invalidPlanMsg, exchange, nullValue());
+        assertThat(invalidPlanMsg, join, notNullValue());
+        assertThat(invalidPlanMsg, join.distribution().function().affinity(), is(true));
+
+        List<IgniteExchange> exchanges = findNodes(phys, node -> node instanceof IgniteExchange
+                && ((IgniteRel) node).distribution().function().affinity());
+
+        assertThat(invalidPlanMsg, exchanges, hasSize(1));
+        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0), instanceOf(IgniteIndexScan.class));
+        assertThat(invalidPlanMsg, exchanges.get(0).getInput(0)
+                .getTable().unwrap(TestTable.class), equalTo(complexTblIndirect));
     }
 }
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 d98d9fe713..045daff6e9 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
@@ -23,6 +23,7 @@ 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.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptUtil;
@@ -58,7 +59,7 @@ public class JoinCommutePlannerTest extends AbstractPlannerTest {
                                 .build(), 1_000) {
 
                     @Override public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "HUGE", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
                     }
                 },
                 new TestTable(
@@ -68,7 +69,7 @@ public class JoinCommutePlannerTest extends AbstractPlannerTest {
                                 .build(), 10) {
 
                     @Override public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "SMALL", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), 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 4e74979a1b..b40172b112 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
@@ -68,7 +68,6 @@ import org.apache.ignite.internal.util.CollectionUtils;
 import org.apache.ignite.network.ClusterNode;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.condition.DisabledOnOs;
 import org.junit.jupiter.api.condition.OS;
@@ -121,7 +120,7 @@ public class PlannerTest extends AbstractPlannerTest {
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Developer", "hash");
+                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -141,7 +140,7 @@ public class PlannerTest extends AbstractPlannerTest {
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Project", "hash");
+                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -188,7 +187,6 @@ public class PlannerTest extends AbstractPlannerTest {
     }
 
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void testSplitterColocatedReplicatedReplicated() throws Exception {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
@@ -267,7 +265,6 @@ public class PlannerTest extends AbstractPlannerTest {
     }
 
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void testSplitterPartiallyColocatedReplicatedAndPartitioned() throws Exception {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
@@ -305,7 +302,7 @@ public class PlannerTest extends AbstractPlannerTest {
             }
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Project", "hash");
+                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -386,7 +383,7 @@ public class PlannerTest extends AbstractPlannerTest {
 
             @Override
             public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Project", "hash");
+                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -432,7 +429,6 @@ public class PlannerTest extends AbstractPlannerTest {
     }
 
     @Test
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void testSplitterPartiallyColocated2() throws Exception {
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
@@ -469,7 +465,7 @@ public class PlannerTest extends AbstractPlannerTest {
 
             @Override
             public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Project", "hash");
+                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -816,7 +812,7 @@ public class PlannerTest extends AbstractPlannerTest {
 
                     @Override
                     public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "TEST", "hash");
+                        return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
                     }
                 }
         );
@@ -851,7 +847,7 @@ public class PlannerTest extends AbstractPlannerTest {
                         .build(), "TAB0") {
 
             @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "tab0", "hash");
+                return IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID);
             }
         };
 
@@ -870,10 +866,10 @@ public class PlannerTest extends AbstractPlannerTest {
     @Test
     public void checkTableHintsHandling() throws Exception {
         IgniteSchema publicSchema = createSchema(
-                createTable("PERSON", IgniteDistributions.affinity(0, "ignored", "ignored"),
+                createTable("PERSON", IgniteDistributions.affinity(0, UUID.randomUUID(), Integer.MIN_VALUE),
                         "PK", Integer.class, "ORG_ID", Integer.class
                 ),
-                createTable("COMPANY", IgniteDistributions.affinity(0, "ignored", "ignored"),
+                createTable("COMPANY", IgniteDistributions.affinity(0, UUID.randomUUID(), 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 1970557e53..a9ee2c22f3 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,6 +18,7 @@
 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;
@@ -38,7 +39,6 @@ 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.type.IgniteTypeSystem;
 import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.TestInstance;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.EnumSource;
@@ -74,23 +74,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,
-                // TODO https://issues.apache.org/jira/browse/IGNITE-18211
-                // IgniteDistributions.affinity(0, "Test1", "hash"));
-                IgniteDistributions.hash(List.of(0)));
-
-
-        createTable(publicSchema, "AFFINITY_TBL2", type,
-                // TODO https://issues.apache.org/jira/browse/IGNITE-18211
-                // IgniteDistributions.affinity(0, "Test2", "hash"));
-                IgniteDistributions.hash(List.of(0)));
-
-        createTable(publicSchema, "AFFINITY_TBL3", type,
-                IgniteDistributions.affinity(1, "Test3", "hash"));
-
-        createTable(publicSchema, "AFFINITY_TBL4", type,
-                IgniteDistributions.affinity(0, "Test4", "hash2"));
+        createTable(publicSchema, "AFFINITY_TBL1", type, IgniteDistributions.affinity(0, UUID.randomUUID(), 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));
     }
 
     /**
@@ -259,8 +247,7 @@ public class SetOpPlannerTest extends AbstractPlannerTest {
 
         assertPlan(sql, publicSchema, isInstanceOf(IgniteExchange.class)
                 .and(input(isInstanceOf(setOp.colocated)
-                        // TODO https://issues.apache.org/jira/browse/IGNITE-18211
-                        // .and(hasDistribution(IgniteDistributions.affinity(0, null, "hash")))
+                        .and(hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)))
                         .and(input(0, isTableScan("affinity_tbl1")))
                         .and(input(1, isTableScan("affinity_tbl2")))
                 ))
@@ -284,8 +271,7 @@ public class SetOpPlannerTest extends AbstractPlannerTest {
 
         assertPlan(sql, publicSchema, isInstanceOf(IgniteExchange.class)
                 .and(input(isInstanceOf(setOp.colocated)
-                        // TODO https://issues.apache.org/jira/browse/IGNITE-18211
-                        // .and(hasDistribution(IgniteDistributions.affinity(0, null, "hash")))
+                        .and(hasDistribution(IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)))
                         .and(input(0, isTableScan("affinity_tbl1")))
                         .and(input(1, isInstanceOf(IgniteTrimExchange.class)
                                 .and(input(isTableScan("broadcast_tbl1")))
@@ -303,7 +289,6 @@ public class SetOpPlannerTest extends AbstractPlannerTest {
      */
     @ParameterizedTest
     @EnumSource
-    @Disabled("https://issues.apache.org/jira/browse/IGNITE-18211")
     public void testSetOpNonColocatedAffinity(SetOp setOp) throws Exception {
         String sql = "SELECT * FROM affinity_tbl1 "
                 + setOp(setOp)
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortAggregatePlannerTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortAggregatePlannerTest.java
index f0f1f489ba..cf8a66a3c7 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortAggregatePlannerTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/SortAggregatePlannerTest.java
@@ -25,6 +25,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.List;
+import java.util.UUID;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollations;
@@ -169,27 +170,20 @@ public class SortAggregatePlannerTest extends AbstractAggregatePlannerTest {
      */
     @Test
     public void collationPermuteMapReduce() throws Exception {
+        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        TestTable tbl = new TestTable(
+        createTable(publicSchema,
+                "TEST",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("VAL0", f.createJavaType(Integer.class))
                         .add("VAL1", f.createJavaType(Integer.class))
                         .add("GRP0", f.createJavaType(Integer.class))
                         .add("GRP1", f.createJavaType(Integer.class))
-                        .build(), "TEST") {
-
-            @Override
-            public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "test", "hash");
-            }
-        }
-                .addIndex("grp0_1", 3, 4);
-
-        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
-
-        publicSchema.addTable(tbl);
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        ).addIndex("grp0_1", 3, 4);
 
         String sql = "SELECT MIN(val0) FROM test GROUP BY grp1, grp0";
 
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 f9aed43182..1a61890522 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,6 +24,7 @@ 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 +38,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.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;
@@ -57,37 +57,25 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
         IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        publicSchema.addTable(
-                new TestTable(
-                        new RelDataTypeFactory.Builder(f)
-                                .add("ID", f.createJavaType(Integer.class))
-                                .add("JID", f.createJavaType(Integer.class))
-                                .add("VAL", f.createJavaType(String.class))
-                                .build(), "T0") {
-
-                    @Override
-                    public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T0", "hash");
-                    }
-                }
-                        .addIndex("t0_jid_idx", 1, 0)
-        );
-
-        publicSchema.addTable(
-                new TestTable(
-                        new RelDataTypeFactory.Builder(f)
-                                .add("ID", f.createJavaType(Integer.class))
-                                .add("JID", f.createJavaType(Integer.class))
-                                .add("VAL", f.createJavaType(String.class))
-                                .build(), "T1") {
-
-                    @Override
-                    public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T1", "hash");
-                    }
-                }
-                        .addIndex("t1_jid_idx", 1, 0)
-        );
+        createTable(publicSchema,
+                "T0",
+                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)
+        ).addIndex("t0_jid_idx", 1, 0);
+
+        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)
+        ).addIndex("t1_jid_idx", 1, 0);
 
         String sql = "select * "
                 + "from t0 "
@@ -119,38 +107,27 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
         IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        publicSchema.addTable(
-                new TestTable(
-                        new RelDataTypeFactory.Builder(f)
-                                .add("ID", f.createJavaType(Integer.class))
-                                .add("JID0", f.createJavaType(Integer.class))
-                                .add("JID1", f.createJavaType(Integer.class))
-                                .add("VAL", f.createJavaType(String.class))
-                                .build(), "T0") {
-
-                    @Override
-                    public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T0", "hash");
-                    }
-                }
+        createTable(publicSchema,
+                "T0",
+                new RelDataTypeFactory.Builder(f)
+                        .add("ID", f.createJavaType(Integer.class))
+                        .add("JID0", f.createJavaType(Integer.class))
+                        .add("JID1", f.createJavaType(Integer.class))
+                        .add("VAL", f.createJavaType(String.class))
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
         );
 
-        publicSchema.addTable(
-                new TestTable(
-                        new RelDataTypeFactory.Builder(f)
-                                .add("ID", f.createJavaType(Integer.class))
-                                .add("JID0", f.createJavaType(Integer.class))
-                                .add("JID1", f.createJavaType(Integer.class))
-                                .add("VAL", f.createJavaType(String.class))
-                                .build(), "T1") {
-
-                    @Override
-                    public IgniteDistribution distribution() {
-                        return IgniteDistributions.affinity(0, "T1", "hash");
-                    }
-                }
-                        .addIndex("t1_jid0_idx", 2, 1)
-        );
+        createTable(publicSchema,
+                "T1",
+                new RelDataTypeFactory.Builder(f)
+                        .add("ID", f.createJavaType(Integer.class))
+                        .add("JID0", f.createJavaType(Integer.class))
+                        .add("JID1", f.createJavaType(Integer.class))
+                        .add("VAL", f.createJavaType(String.class))
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        ).addIndex("t1_jid0_idx", 2, 1);
 
         String sql = "select * "
                 + "from t0 "
@@ -183,10 +160,10 @@ public class SortedIndexSpoolPlannerTest extends AbstractPlannerTest {
     @Test
     public void testDescFields() throws Exception {
         IgniteSchema publicSchema = createSchema(
-                createTable("T0", 10, IgniteDistributions.affinity(0, "T0", "hash"),
+                createTable("T0", 10, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                         "ID", Integer.class, "JID", Integer.class, "VAL", String.class)
                         .addIndex("t0_jid_idx", 1),
-                createTable("T1", 100, IgniteDistributions.affinity(0, "T1", "hash"),
+                createTable("T1", 100, IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID),
                         "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 25935217c2..e6ca578825 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,12 +19,12 @@ 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.IgniteDistribution;
 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.type.IgniteTypeSystem;
@@ -42,38 +42,28 @@ public class TableSpoolPlannerTest extends AbstractPlannerTest {
      */
     @Test
     public void tableSpoolDistributed() throws Exception {
+        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        TestTable t0 = new TestTable(
+        createTable(publicSchema,
+                "T0",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("JID", f.createJavaType(Integer.class))
                         .add("VAL", f.createJavaType(String.class))
-                        .build(), "T0") {
-
-            @Override
-            public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "T0", "hash");
-            }
-        };
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
 
-        TestTable t1 = new TestTable(
+        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(), "T1") {
-
-            @Override
-            public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "T1", "hash");
-            }
-        };
-
-        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
-
-        publicSchema.addTable(t0);
-        publicSchema.addTable(t1);
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
 
         String sql = "select * "
                 + "from t0 "
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 87f4360d74..a60ecf3833 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,12 +17,12 @@
 
 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;
 import org.apache.ignite.internal.sql.engine.rel.agg.IgniteReduceAggregateBase;
 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.type.IgniteTypeFactory;
 import org.apache.ignite.internal.sql.engine.type.IgniteTypeSystem;
@@ -77,49 +77,40 @@ public class UnionPlannerTest extends AbstractPlannerTest {
      * @return Ignite schema.
      */
     private IgniteSchema prepareSchema() {
+        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
+
         IgniteTypeFactory f = new IgniteTypeFactory(IgniteTypeSystem.INSTANCE);
 
-        TestTable tbl1 = new TestTable(
+        createTable(publicSchema,
+                "TABLE1",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
-                        .build(), "TABLE1") {
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
 
-            @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Table1", "hash");
-            }
-        };
-
-        TestTable tbl2 = new TestTable(
+        createTable(publicSchema,
+                "TABLE2",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
-                        .build(), "TABLE2") {
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
 
-            @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Table2", "hash");
-            }
-        };
 
-        TestTable tbl3 = new TestTable(
+        createTable(publicSchema,
+                "TABLE3",
                 new RelDataTypeFactory.Builder(f)
                         .add("ID", f.createJavaType(Integer.class))
                         .add("NAME", f.createJavaType(String.class))
                         .add("SALARY", f.createJavaType(Double.class))
-                        .build(), "TABLE3") {
-
-            @Override public IgniteDistribution distribution() {
-                return IgniteDistributions.affinity(0, "Table3", "hash");
-            }
-        };
-
-        IgniteSchema publicSchema = new IgniteSchema("PUBLIC");
-
-        publicSchema.addTable(tbl1);
-        publicSchema.addTable(tbl2);
-        publicSchema.addTable(tbl3);
+                        .build(),
+                IgniteDistributions.affinity(0, UUID.randomUUID(), DEFAULT_ZONE_ID)
+        );
 
         return publicSchema;
     }
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/HashFunctionsTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/HashFunctionsTest.java
new file mode 100644
index 0000000000..e8663d4418
--- /dev/null
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/util/HashFunctionsTest.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine.util;
+
+import java.util.Arrays;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.sql.engine.exec.ArrayRowHandler;
+import org.apache.ignite.internal.sql.engine.exec.RowHandler;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactoryImpl.SimpleHashFunction;
+import org.apache.ignite.internal.sql.engine.util.HashFunctionFactoryImpl.TypesAwareHashFunction;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.EnumSource;
+
+/**
+ * Basic tests for hash functions, which can be created using {@link HashFunctionFactory}.
+ */
+class HashFunctionsTest {
+    /**
+     * Ensures that the hash function accepts {@code null} values in row.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void checkNull(HashFunc func) {
+        func.hash(new Object[]{null}, 0);
+    }
+
+    /**
+     * Ensures that the hash function accepts empty row.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void checkEmpty(HashFunc func) {
+        Assertions.assertEquals(0, func.hash(new Object[]{}));
+    }
+
+    /**
+     * Ensures that the hash is computed according to the specified field ordinals
+     * and that the order of the ordinal numbers matters.
+     */
+    @ParameterizedTest
+    @EnumSource
+    public void checkOrder(HashFunc func) {
+        Object[] row = {100, 200, 100};
+
+        Assertions.assertNotEquals(
+                func.hash(row, 0, 1),
+                func.hash(row, 1, 0)
+        );
+
+        Assertions.assertEquals(
+                func.hash(row, 0, 2),
+                func.hash(row, 2, 0)
+        );
+    }
+
+    enum HashFunc {
+        SIMPLE,
+        TYPE_AWARE;
+
+        /**
+         * Compute hash.
+         *
+         * @param row Row to process.
+         * @param keys Ordinal numbers of key fields.
+         * @return Composite hash for the specified fields.
+         */
+        int hash(Object[] row, int... keys) {
+            RowHandler<Object[]> rowHandler = ArrayRowHandler.INSTANCE;
+            ToIntFunction<Object[]> func;
+
+            switch (this) {
+                case SIMPLE:
+                    func = new SimpleHashFunction<>(keys, rowHandler);
+
+                    break;
+
+                case TYPE_AWARE:
+                    NativeType[] fieldTypes = new NativeType[keys.length];
+
+                    Arrays.fill(fieldTypes, NativeTypes.INT32);
+
+                    func = new TypesAwareHashFunction<>(keys, fieldTypes, rowHandler);
+
+                    break;
+
+                default:
+                    throw new UnsupportedOperationException();
+            }
+
+            return func.applyAsInt(row);
+        }
+    }
+}