You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tk...@apache.org on 2023/08/17 08:34:46 UTC

[ignite-3] branch main updated: IGNITE-20243 Introduce a Matcher for BinaryRows (#2457)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new bb2c93aa6b IGNITE-20243 Introduce a Matcher for BinaryRows (#2457)
bb2c93aa6b is described below

commit bb2c93aa6b132beaa31934f5127fdab4295e6019
Author: Alexander Polovtcev <al...@gmail.com>
AuthorDate: Thu Aug 17 11:34:40 2023 +0300

    IGNITE-20243 Introduce a Matcher for BinaryRows (#2457)
---
 .../ignite/internal/table/ItRoReadsTest.java       |  44 +++-----
 modules/schema/build.gradle                        |   2 +
 .../ignite/internal/schema/BinaryRowMatcher.java   |  62 +++++++++++
 .../AbstractMvPartitionStorageConcurrencyTest.java |  19 ++--
 .../storage/AbstractMvPartitionStorageGcTest.java  |  21 ++--
 .../storage/AbstractMvPartitionStorageTest.java    | 117 +++++++++++----------
 .../storage/AbstractMvTableStorageTest.java        |   4 +-
 .../internal/storage/BaseMvStoragesTest.java       |  36 +++----
 .../AbstractPageMemoryMvPartitionStorageTest.java  |  11 +-
 ...PersistentPageMemoryMvPartitionStorageTest.java |   3 +-
 .../internal/table/distributed/IndexBaseTest.java  |  10 +-
 .../replication/PartitionReplicaListenerTest.java  |  41 ++++----
 12 files changed, 212 insertions(+), 158 deletions(-)

diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItRoReadsTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItRoReadsTest.java
index 99ef4e29ab..8b89bf2a37 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItRoReadsTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItRoReadsTest.java
@@ -20,10 +20,12 @@ package org.apache.ignite.internal.table;
 import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_REPLICA_COUNT;
 import static org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.createZone;
 import static org.apache.ignite.internal.runner.app.ItTablesApiTest.SCHEMA;
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
 import static org.apache.ignite.internal.schema.testutils.SchemaConfigurationConverter.convert;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.testNodeName;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.is;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNull;
@@ -34,7 +36,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Flow.Publisher;
 import java.util.concurrent.Flow.Subscriber;
 import java.util.concurrent.Flow.Subscription;
@@ -178,7 +179,7 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         res = internalTable.get(keyRow, node.clock().now(), node.node()).get();
 
-        assertRowEquals(res, keyValueRow);
+        assertThat(res, is(equalToRow(keyValueRow)));
     }
 
     @Test
@@ -210,7 +211,7 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         BinaryRow res = internalTable.get(keyRow, node.clock().now(), node.node()).get();
 
-        assertRowEquals(res, keyValueRow2);
+        assertThat(res, is(equalToRow(keyValueRow2)));
     }
 
     @Test
@@ -269,9 +270,7 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         latch.await();
 
-        assertEquals(1, list.size());
-
-        assertRowEquals(list.get(0), keyValueRow2);
+        assertThat(list, contains(equalToRow(keyValueRow2)));
     }
 
     @Test
@@ -300,13 +299,13 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         BinaryRow res = internalTable.get(keyRow, node.clock().now(), node.node()).get();
 
-        assertRowEquals(res, keyValueRow);
+        assertThat(res, is(equalToRow(keyValueRow)));
 
         tx2.commit();
 
         res = internalTable.get(keyRow, node.clock().now(), node.node()).get();
 
-        assertRowEquals(res, keyValueRow2);
+        assertThat(res, is(equalToRow(keyValueRow2)));
     }
 
     @Test
@@ -337,15 +336,11 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         res = internalTable.getAll(keyRows, node.clock().now(), node.node()).get();
 
-        assertEquals(3, res.size());
-
-        for (int i = 0; i < 3; i++) {
-            assertRowEquals(res.get(i), keyValueRows.get(i));
-        }
+        assertThat(res, contains(equalToRow(keyValueRows.get(0)), equalToRow(keyValueRows.get(1)), equalToRow(keyValueRows.get(2))));
     }
 
     @Test
-    public void testRoGetAllWithSeveralInserts() throws ExecutionException, InterruptedException {
+    public void testRoGetAllWithSeveralInserts() throws Exception {
         IgniteImpl node = node();
 
         InternalTable internalTable = ((TableImpl) table).internalTable();
@@ -368,11 +363,7 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         res = internalTable.getAll(keyRows, node.clock().now(), node.node()).get();
 
-        assertEquals(3, res.size());
-
-        for (int i = 0; i < 3; i++) {
-            assertRowEquals(res.get(i), keyValueRows.get(i));
-        }
+        assertThat(res, contains(equalToRow(keyValueRows.get(0)), equalToRow(keyValueRows.get(1)), equalToRow(keyValueRows.get(2))));
 
         node.transactions().runInTransaction(txs -> {
             for (int i = 0; i < 15; i++) {
@@ -386,11 +377,7 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
 
         res = internalTable.getAll(keyRows, node.clock().now(), node.node()).get();
 
-        assertEquals(3, res.size());
-
-        assertRowEquals(res.get(0), newKeyValueRow1);
-        assertRowEquals(res.get(1), newKeyValueRow2);
-        assertRowEquals(res.get(2), newKeyValueRow3);
+        assertThat(res, contains(equalToRow(newKeyValueRow1), equalToRow(newKeyValueRow2), equalToRow(newKeyValueRow3)));
     }
 
     @Test
@@ -498,9 +485,9 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
     }
 
     private static void putValue(KeyValueView<Tuple, Tuple> kv, int val, Transaction tx) {
-        Tuple tableKey = Tuple.create().set("key", Long.valueOf(val % 100));
+        Tuple tableKey = Tuple.create().set("key", (long) (val % 100));
 
-        Tuple value = Tuple.create().set("valInt", Integer.valueOf(val)).set("valStr", "some string row" + val);
+        Tuple value = Tuple.create().set("valInt", val).set("valStr", "some string row" + val);
 
         kv.put(tx, tableKey, value);
     }
@@ -553,9 +540,4 @@ public class ItRoReadsTest extends BaseIgniteAbstractTest {
     protected static IgniteImpl node() {
         return (IgniteImpl) NODE;
     }
-
-    private static void assertRowEquals(BinaryRow row1, BinaryRow row2) {
-        assertThat(row1.schemaVersion(), is(row2.schemaVersion()));
-        assertThat(row1.tupleSlice(), is(row2.tupleSlice()));
-    }
 }
diff --git a/modules/schema/build.gradle b/modules/schema/build.gradle
index b97177d6cd..7c2c191a1d 100644
--- a/modules/schema/build.gradle
+++ b/modules/schema/build.gradle
@@ -53,6 +53,8 @@ dependencies {
     testImplementation libs.jmh.core
     testImplementation libs.javax.annotations
 
+    testFixturesApi libs.hamcrest.core
+
     testFixturesImplementation project(':ignite-api')
     testFixturesImplementation project(':ignite-bytecode')
     testFixturesImplementation project(':ignite-configuration-api')
diff --git a/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/BinaryRowMatcher.java b/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/BinaryRowMatcher.java
new file mode 100644
index 0000000000..e628961184
--- /dev/null
+++ b/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/BinaryRowMatcher.java
@@ -0,0 +1,62 @@
+/*
+ * 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.schema;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.hamcrest.Description;
+import org.hamcrest.TypeSafeMatcher;
+
+/**
+ * Matcher for comparing {@link BinaryRow}s.
+ */
+public class BinaryRowMatcher extends TypeSafeMatcher<BinaryRow> {
+    private final BinaryRow row;
+
+    private BinaryRowMatcher(BinaryRow row) {
+        this.row = row;
+    }
+
+    public static BinaryRowMatcher equalToRow(BinaryRow row) {
+        return new BinaryRowMatcher(row);
+    }
+
+    @Override
+    protected boolean matchesSafely(BinaryRow item) {
+        return row.schemaVersion() == item.schemaVersion() && row.tupleSlice().equals(item.tupleSlice());
+    }
+
+    @Override
+    public void describeTo(Description description) {
+        description.appendValue(rowToString(row));
+    }
+
+    @Override
+    protected void describeMismatchSafely(BinaryRow item, Description mismatchDescription) {
+        mismatchDescription.appendText("was ").appendValue(rowToString(item));
+    }
+
+    private static String rowToString(BinaryRow row) {
+        ByteBuffer tupleSlice = row.tupleSlice();
+
+        byte[] array = new byte[tupleSlice.remaining()];
+        tupleSlice.get(array);
+
+        return String.format("{schemaVersion=%d tuple=%s}", row.schemaVersion(), Arrays.toString(array));
+    }
+}
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageConcurrencyTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageConcurrencyTest.java
index 53c9cc25e8..e9b0736ef5 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageConcurrencyTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageConcurrencyTest.java
@@ -17,17 +17,20 @@
 
 package org.apache.ignite.internal.storage;
 
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
 import static org.apache.ignite.internal.testframework.IgniteTestUtils.runRace;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.hasItem;
+import static org.hamcrest.Matchers.is;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
 import java.util.Collection;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
 import org.apache.ignite.internal.schema.BinaryRow;
+import org.hamcrest.Matcher;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.RepeatedTest;
 import org.junit.jupiter.api.Test;
@@ -69,7 +72,7 @@ public abstract class AbstractMvPartitionStorageConcurrencyTest extends BaseMvPa
                     () -> scanFirstEntry(HybridTimestamp.MAX_VALUE)
             );
 
-            assertRowMatches(read(ROW_ID, clock.now()), TABLE_ROW);
+            assertThat(read(ROW_ID, clock.now()), is(equalToRow(TABLE_ROW)));
         }
     }
 
@@ -85,7 +88,7 @@ public abstract class AbstractMvPartitionStorageConcurrencyTest extends BaseMvPa
                     () -> scanFirstEntry(HybridTimestamp.MAX_VALUE)
             );
 
-            assertRowMatches(read(ROW_ID, clock.now()), TABLE_ROW2);
+            assertThat(read(ROW_ID, clock.now()), is(equalToRow(TABLE_ROW2)));
         }
     }
 
@@ -140,7 +143,7 @@ public abstract class AbstractMvPartitionStorageConcurrencyTest extends BaseMvPa
                     () -> addWrite(ROW_ID, TABLE_ROW2, TX_ID)
             );
 
-            assertRowMatches(read(ROW_ID, HybridTimestamp.MAX_VALUE), TABLE_ROW2);
+            assertThat(read(ROW_ID, HybridTimestamp.MAX_VALUE), is(equalToRow(TABLE_ROW2)));
 
             abortWrite(ROW_ID);
 
@@ -163,7 +166,7 @@ public abstract class AbstractMvPartitionStorageConcurrencyTest extends BaseMvPa
                     () -> commitWrite(ROW_ID, clock.now())
             );
 
-            assertRowMatches(read(ROW_ID, HybridTimestamp.MAX_VALUE), TABLE_ROW2);
+            assertThat(read(ROW_ID, HybridTimestamp.MAX_VALUE), is(equalToRow(TABLE_ROW2)));
 
             assertNull(pollForVacuum(HybridTimestamp.MAX_VALUE));
 
@@ -221,7 +224,11 @@ public abstract class AbstractMvPartitionStorageConcurrencyTest extends BaseMvPa
     private static void assertRemoveRow(@Nullable BinaryRow rowBytes, Collection<BinaryRow> rows) {
         assertNotNull(rowBytes);
 
-        assertTrue(rows.remove(rowBytes), rowBytes.toString());
+        Matcher<BinaryRow> matcher = equalToRow(rowBytes);
+
+        assertThat(rows, hasItem(matcher));
+
+        rows.removeIf(matcher::matches);
     }
 
     @SuppressWarnings("ResultOfMethodCallIgnored")
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java
index bec02e5791..75062ea410 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageGcTest.java
@@ -17,6 +17,9 @@
 
 package org.apache.ignite.internal.storage;
 
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertNull;
 
@@ -58,13 +61,13 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
 
         assertNotNull(gcedRow);
 
-        assertRowMatches(gcedRow.binaryRow(), TABLE_ROW);
+        assertThat(gcedRow.binaryRow(), is(equalToRow(TABLE_ROW)));
 
         // Read from the old timestamp should return null.
         assertNull(read(ROW_ID, firstCommitTs));
 
         // Read from the newer timestamp should return last value.
-        assertRowMatches(read(ROW_ID, secondCommitTs), TABLE_ROW2);
+        assertThat(read(ROW_ID, secondCommitTs), is(equalToRow(TABLE_ROW2)));
     }
 
     @Test
@@ -75,7 +78,7 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
         BinaryRowAndRowId row = pollForVacuum(secondCommitTs);
 
         assertNotNull(row);
-        assertRowMatches(row.binaryRow(), TABLE_ROW);
+        assertThat(row.binaryRow(), is(equalToRow(TABLE_ROW)));
 
         assertNull(read(ROW_ID, secondCommitTs));
 
@@ -95,7 +98,7 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
         BinaryRowAndRowId row = pollForVacuum(lastCommitTs);
 
         assertNotNull(row);
-        assertRowMatches(row.binaryRow(), TABLE_ROW);
+        assertThat(row.binaryRow(), is(equalToRow(TABLE_ROW)));
 
         assertNull(read(ROW_ID, lastCommitTs));
 
@@ -118,13 +121,13 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
         BinaryRowAndRowId row = pollForVacuum(lowWatermark);
 
         assertNotNull(row);
-        assertRowMatches(row.binaryRow(), TABLE_ROW);
+        assertThat(row.binaryRow(), is(equalToRow(TABLE_ROW)));
 
         // Poll the next oldest row.
         row = pollForVacuum(lowWatermark);
 
         assertNotNull(row);
-        assertRowMatches(row.binaryRow(), TABLE_ROW2);
+        assertThat(row.binaryRow(), is(equalToRow(TABLE_ROW2)));
 
         // Nothing else to poll.
         assertNull(pollForVacuum(lowWatermark));
@@ -140,7 +143,8 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
 
         BinaryRowAndRowId row = pollForVacuum(HybridTimestamp.MAX_VALUE);
 
-        assertRowMatches(row.binaryRow(), TABLE_ROW);
+        assertNotNull(row);
+        assertThat(row.binaryRow(), is(equalToRow(TABLE_ROW)));
     }
 
     @Test
@@ -153,6 +157,7 @@ public abstract class AbstractMvPartitionStorageGcTest extends BaseMvPartitionSt
 
         BinaryRowAndRowId row = pollForVacuum(HybridTimestamp.MAX_VALUE);
 
-        assertRowMatches(row.binaryRow(), TABLE_ROW);
+        assertNotNull(row);
+        assertThat(row.binaryRow(), is(equalToRow(TABLE_ROW)));
     }
 }
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
index c9d7659512..422aaec831 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.storage;
 
 import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
@@ -96,7 +97,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         addWrite(rowId, binaryRow, txId);
 
         // Read with timestamp returns write-intent.
-        assertRowMatches(read(rowId, clock.now()), binaryRow);
+        assertThat(read(rowId, clock.now()), is(equalToRow(binaryRow)));
 
         // Remove write.
         addWrite(rowId, null, txId);
@@ -142,8 +143,8 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         assertNull(read(rowId, tsBefore));
 
         // Row is valid at the time during and after writing.
-        assertRowMatches(read(rowId, tsExact), binaryRow);
-        assertRowMatches(read(rowId, tsAfter), binaryRow);
+        assertThat(read(rowId, tsExact), is(equalToRow(binaryRow)));
+        assertThat(read(rowId, tsAfter), is(equalToRow(binaryRow)));
 
         TestValue newValue = new TestValue(30, "duh");
 
@@ -155,22 +156,22 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         // Same checks, but now there are two different versions.
         assertNull(read(rowId, tsBefore));
 
-        assertRowMatches(read(rowId, HybridTimestamp.MAX_VALUE), newRow);
+        assertThat(read(rowId, HybridTimestamp.MAX_VALUE), is(equalToRow(newRow)));
 
-        assertRowMatches(read(rowId, tsExact), binaryRow);
-        assertRowMatches(read(rowId, tsAfter), newRow);
-        assertRowMatches(read(rowId, clock.now()), newRow);
+        assertThat(read(rowId, tsExact), is(equalToRow(binaryRow)));
+        assertThat(read(rowId, tsAfter), is(equalToRow(newRow)));
+        assertThat(read(rowId, clock.now()), is(equalToRow(newRow)));
 
         // Only latest time behavior changes after commit.
         HybridTimestamp newRowCommitTs = clock.now();
         commitWrite(rowId, newRowCommitTs);
 
-        assertRowMatches(read(rowId, HybridTimestamp.MAX_VALUE), newRow);
+        assertThat(read(rowId, HybridTimestamp.MAX_VALUE), is(equalToRow(newRow)));
 
-        assertRowMatches(read(rowId, tsExact), binaryRow);
-        assertRowMatches(read(rowId, tsAfter), binaryRow);
+        assertThat(read(rowId, tsExact), is(equalToRow(binaryRow)));
+        assertThat(read(rowId, tsAfter), is(equalToRow(binaryRow)));
 
-        assertRowMatches(read(rowId, clock.now()), newRow);
+        assertThat(read(rowId, clock.now()), is(equalToRow(newRow)));
 
         // Remove.
         UUID removeTxId = newTransactionId();
@@ -181,9 +182,9 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         assertNull(read(rowId, HybridTimestamp.MAX_VALUE));
 
-        assertRowMatches(read(rowId, tsExact), binaryRow);
-        assertRowMatches(read(rowId, tsAfter), binaryRow);
-        assertRowMatches(read(rowId, newRowCommitTs), newRow);
+        assertThat(read(rowId, tsExact), is(equalToRow(binaryRow)));
+        assertThat(read(rowId, tsAfter), is(equalToRow(binaryRow)));
+        assertThat(read(rowId, newRowCommitTs), is(equalToRow(newRow)));
 
         assertNull(read(rowId, clock.now()));
 
@@ -197,8 +198,8 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         assertNull(read(rowId, removeTs));
         assertNull(read(rowId, clock.now()));
 
-        assertRowMatches(read(rowId, tsExact), binaryRow);
-        assertRowMatches(read(rowId, tsAfter), binaryRow);
+        assertThat(read(rowId, tsExact), is(equalToRow(binaryRow)));
+        assertThat(read(rowId, tsAfter), is(equalToRow(binaryRow)));
     }
 
     /**
@@ -269,7 +270,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
             assertFalse(cursor.hasNext());
             assertFalse(cursor.hasNext());
 
-            assertThrows(NoSuchElementException.class, () -> cursor.next());
+            assertThrows(NoSuchElementException.class, cursor::next);
 
             assertThat(res, containsInAnyOrder(value1, value2));
         }
@@ -336,18 +337,18 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
                     commitTs = commitTs2;
                 }
 
-                assertRowMatches(res.binaryRow(), expectedRow2);
+                assertThat(res.binaryRow(), is(equalToRow(expectedRow2)));
 
                 BinaryRow previousRow = cursor.committed(commitTs);
 
                 assertNotNull(previousRow);
-                assertRowMatches(previousRow, expectedRow1);
+                assertThat(previousRow, is(equalToRow(expectedRow1)));
             }
 
             assertFalse(cursor.hasNext());
             assertFalse(cursor.hasNext());
 
-            assertThrows(NoSuchElementException.class, () -> cursor.next());
+            assertThrows(NoSuchElementException.class, cursor::next);
 
             assertThrows(IllegalStateException.class, () -> cursor.committed(commitTs1));
         }
@@ -369,7 +370,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         ReadResult foundResult = storage.read(rowId, HybridTimestamp.MAX_VALUE);
 
         assertThat(foundResult.rowId(), is(rowId));
-        assertRowMatches(foundResult.binaryRow(), binaryRow);
+        assertThat(foundResult.binaryRow(), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -380,7 +381,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         ReadResult foundResult = storage.read(rowId, HybridTimestamp.MAX_VALUE);
 
         assertThat(foundResult.rowId(), is(rowId));
-        assertRowMatches(foundResult.binaryRow(), binaryRow);
+        assertThat(foundResult.binaryRow(), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -393,7 +394,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         ReadResult foundResult = storage.read(rowId2, HybridTimestamp.MAX_VALUE);
 
         assertThat(foundResult.rowId(), is(rowId2));
-        assertRowMatches(foundResult.binaryRow(), binaryRow2);
+        assertThat(foundResult.binaryRow(), is(equalToRow(binaryRow2)));
     }
 
     @Test
@@ -407,7 +408,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         ReadResult foundResult = storage.read(rowId2, HybridTimestamp.MAX_VALUE);
 
         assertThat(foundResult.rowId(), is(rowId2));
-        assertRowMatches(foundResult.binaryRow(), binaryRow2);
+        assertThat(foundResult.binaryRow(), is(equalToRow(binaryRow2)));
     }
 
     @Test
@@ -418,7 +419,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, commitTimestamp);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -430,7 +431,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         HybridTimestamp afterCommit = clock.now();
         BinaryRow foundRow = read(rowId, afterCommit);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -458,7 +459,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, secondVersionTs);
 
-        assertRowMatches(foundRow, binaryRow2);
+        assertThat(foundRow, is(equalToRow(binaryRow2)));
     }
 
     @Test
@@ -472,7 +473,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, firstVersionTs);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -488,7 +489,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, tsInBetween);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -500,7 +501,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         HybridTimestamp latestTs = clock.now();
         BinaryRow foundRow = read(rowId, latestTs);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -513,7 +514,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         HybridTimestamp latestTs = clock.now();
         BinaryRow foundRow = read(rowId, latestTs);
 
-        assertRowMatches(foundRow, binaryRow2);
+        assertThat(foundRow, is(equalToRow(binaryRow2)));
     }
 
     @Test
@@ -531,7 +532,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, HybridTimestamp.MAX_VALUE);
 
-        assertRowMatches(foundRow, binaryRow2);
+        assertThat(foundRow, is(equalToRow(binaryRow2)));
     }
 
     @Test
@@ -540,7 +541,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow returnedRow = addWrite(rowId, binaryRow2, txId);
 
-        assertRowMatches(returnedRow, binaryRow);
+        assertThat(returnedRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -556,7 +557,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
     @Test
     void addWriteCommittedTombstone() {
         addWriteCommitted(ROW_ID, binaryRow, clock.now());
-        assertRowMatches(read(ROW_ID, HybridTimestamp.MAX_VALUE), binaryRow);
+        assertThat(read(ROW_ID, HybridTimestamp.MAX_VALUE), is(equalToRow(binaryRow)));
 
         addWriteCommitted(ROW_ID, null, clock.now());
         assertNull(read(ROW_ID, HybridTimestamp.MAX_VALUE));
@@ -607,7 +608,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, firstTimestamp);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -616,7 +617,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow rowFromRemoval = addWrite(rowId, null, txId);
 
-        assertRowMatches(rowFromRemoval, binaryRow);
+        assertThat(rowFromRemoval, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -647,7 +648,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, clock.now());
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -657,11 +658,11 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         abortWrite(rowId);
 
-        assertRowMatches(read(rowId, HybridTimestamp.MAX_VALUE), binaryRow);
+        assertThat(read(rowId, HybridTimestamp.MAX_VALUE), is(equalToRow(binaryRow)));
 
         commitWrite(rowId, clock.now());
 
-        assertRowMatches(read(rowId, HybridTimestamp.MAX_VALUE), binaryRow);
+        assertThat(read(rowId, HybridTimestamp.MAX_VALUE), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -675,7 +676,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, HybridTimestamp.MAX_VALUE);
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -704,7 +705,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow returnedRow = abortWrite(rowId);
 
-        assertRowMatches(returnedRow, binaryRow);
+        assertThat(returnedRow, is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -715,7 +716,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         // We see the uncommitted row.
         assertThat(foundResult.rowId(), is(rowId));
-        assertRowMatches(foundResult.binaryRow(), binaryRow3);
+        assertThat(foundResult.binaryRow(), is(equalToRow(binaryRow3)));
     }
 
     @Test
@@ -747,7 +748,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         assertNull(res.transactionId());
         assertNull(res.commitTableId());
         assertEquals(ReadResult.UNDEFINED_COMMIT_PARTITION_ID, res.commitPartitionId());
-        assertRowMatches(res.binaryRow(), binaryRow);
+        assertThat(res.binaryRow(), is(equalToRow(binaryRow)));
 
         res = storage.read(rowId, clock.now());
 
@@ -756,7 +757,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         assertEquals(txId2, res.transactionId());
         assertEquals(COMMIT_TABLE_ID, res.commitTableId());
         assertEquals(PARTITION_ID, res.commitPartitionId());
-        assertRowMatches(res.binaryRow(), binaryRow2);
+        assertThat(res.binaryRow(), is(equalToRow(binaryRow2)));
     }
 
     private RowId commitAbortAndAddUncommitted() {
@@ -786,7 +787,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
             ReadResult result = cursor.next();
 
             assertThat(result.rowId(), is(rowId));
-            assertRowMatches(result.binaryRow(), binaryRow3);
+            assertThat(result.binaryRow(), is(equalToRow(binaryRow3)));
 
             assertFalse(cursor.hasNext());
         }
@@ -798,7 +799,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         BinaryRow foundRow = read(rowId, clock.now());
 
-        assertRowMatches(foundRow, binaryRow);
+        assertThat(foundRow, is(equalToRow(binaryRow)));
     }
 
     /**
@@ -866,21 +867,21 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
 
         assertNotNull(res);
         assertNull(res.newestCommitTimestamp());
-        assertRowMatches(res.binaryRow(), binaryRow);
+        assertThat(res.binaryRow(), is(equalToRow(binaryRow)));
 
         // Read between two commits.
         res = storage.read(rowId, betweenCommits);
 
         assertNotNull(res);
         assertNull(res.newestCommitTimestamp());
-        assertRowMatches(res.binaryRow(), binaryRow);
+        assertThat(res.binaryRow(), is(equalToRow(binaryRow)));
 
         // Read at exact time of second commit.
         res = storage.read(rowId, second);
 
         assertNotNull(res);
         assertNull(res.newestCommitTimestamp());
-        assertRowMatches(res.binaryRow(), binaryRow2);
+        assertThat(res.binaryRow(), is(equalToRow(binaryRow2)));
 
         // Read after second commit (write intent).
         res = storage.read(rowId, after);
@@ -888,7 +889,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         assertNotNull(res);
         assertNotNull(res.newestCommitTimestamp());
         assertEquals(second, res.newestCommitTimestamp());
-        assertRowMatches(res.binaryRow(), binaryRow3);
+        assertThat(res.binaryRow(), is(equalToRow(binaryRow3)));
     }
 
     @Test
@@ -939,7 +940,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
             return null;
         });
 
-        assertRowMatches(read(higherRowId, clock.now()), binaryRow);
+        assertThat(read(higherRowId, clock.now()), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -954,7 +955,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
             return null;
         });
 
-        assertRowMatches(read(lowerRowId, clock.now()), binaryRow);
+        assertThat(read(lowerRowId, clock.now()), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -1071,11 +1072,11 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
             assertTrue(next.isWriteIntent());
 
             assertThat(next.rowId(), is(rowIdAndCommitTs.get1()));
-            assertRowMatches(next.binaryRow(), binaryRow2);
+            assertThat(next.binaryRow(), is(equalToRow(binaryRow2)));
 
             BinaryRow committedRow = cursor.committed(rowIdAndCommitTs.get2());
 
-            assertRowMatches(committedRow, binaryRow);
+            assertThat(committedRow, is(equalToRow(binaryRow)));
         }
     }
 
@@ -1160,7 +1161,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         addWriteCommitted(rowId, binaryRow, clock.now());
 
         // Read with timestamp returns write-intent.
-        assertRowMatches(storage.read(rowId, clock.now()).binaryRow(), binaryRow);
+        assertThat(storage.read(rowId, clock.now()).binaryRow(), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -1172,8 +1173,8 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
         addWriteCommitted(rowId, binaryRow, ts1);
         addWriteCommitted(rowId, binaryRow2, clock.now());
 
-        assertRowMatches(storage.read(rowId, clock.now()).binaryRow(), binaryRow2);
-        assertRowMatches(storage.read(rowId, ts1).binaryRow(), binaryRow);
+        assertThat(storage.read(rowId, clock.now()).binaryRow(), is(equalToRow(binaryRow2)));
+        assertThat(storage.read(rowId, ts1).binaryRow(), is(equalToRow(binaryRow)));
     }
 
     @Test
@@ -1301,7 +1302,7 @@ public abstract class AbstractMvPartitionStorageTest extends BaseMvPartitionStor
             cursor.committed(commitTs1);
 
             ReadResult result2 = cursor.next();
-            assertRowMatches(result2.binaryRow(), binaryRow2);
+            assertThat(result2.binaryRow(), is(equalToRow(binaryRow2)));
 
             assertFalse(cursor.hasNext());
         }
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
index 7a3e5ebefd..852958372a 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.storage;
 
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toList;
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
 import static org.apache.ignite.internal.schema.CatalogDescriptorUtils.toHashIndexDescriptor;
 import static org.apache.ignite.internal.schema.CatalogDescriptorUtils.toSortedIndexDescriptor;
 import static org.apache.ignite.internal.schema.CatalogDescriptorUtils.toTableDescriptor;
@@ -1035,8 +1036,7 @@ public abstract class AbstractMvTableStorageTest extends BaseMvStoragesTest {
                 return toListOfBinaryRows(mvPartitionStorage.scanVersions(row.get1()));
             });
 
-            assertThat(allVersions, hasSize(1));
-            assertRowMatches(allVersions.get(0), row.get2());
+            assertThat(allVersions, contains(equalToRow(row.get2())));
 
             IndexRow hashIndexRow = indexRow(hashIndexStorage.indexDescriptor(), row.get2(), row.get1());
             IndexRow sortedIndexRow = indexRow(sortedIndexStorage.indexDescriptor(), row.get2(), row.get1());
diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
index ab68b9e1d6..c96a43c3cb 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
@@ -19,8 +19,6 @@ package org.apache.ignite.internal.storage;
 
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.notNullValue;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 import java.util.List;
@@ -31,7 +29,6 @@ import org.apache.ignite.internal.hlc.HybridClock;
 import org.apache.ignite.internal.hlc.HybridClockImpl;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.BinaryRowConverter;
-import org.apache.ignite.internal.schema.BinaryRowImpl;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.ColumnsExtractor;
 import org.apache.ignite.internal.schema.NativeTypes;
@@ -49,7 +46,6 @@ import org.apache.ignite.internal.tostring.IgniteToStringInclude;
 import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.internal.util.Cursor;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -57,10 +53,10 @@ import org.jetbrains.annotations.Nullable;
  */
 public abstract class BaseMvStoragesTest {
     /** Default reflection marshaller factory. */
-    protected static final MarshallerFactory marshallerFactory = new ReflectionMarshallerFactory();
+    private static final MarshallerFactory MARSHALLER_FACTORY = new ReflectionMarshallerFactory();
 
     /** Schema descriptor for tests. */
-    protected static final SchemaDescriptor schemaDescriptor = new SchemaDescriptor(1, new Column[]{
+    protected static final SchemaDescriptor SCHEMA_DESCRIPTOR = new SchemaDescriptor(1, new Column[]{
             new Column("INTKEY", NativeTypes.INT32, false),
             new Column("STRKEY", NativeTypes.STRING, false),
     }, new Column[]{
@@ -69,26 +65,24 @@ public abstract class BaseMvStoragesTest {
     });
 
     /** Key-value marshaller for tests. */
-    protected static final KvMarshaller<TestKey, TestValue> kvMarshaller
-            = marshallerFactory.create(schemaDescriptor, TestKey.class, TestValue.class);
+    private static final KvMarshaller<TestKey, TestValue> KV_MARSHALLER
+            = MARSHALLER_FACTORY.create(SCHEMA_DESCRIPTOR, TestKey.class, TestValue.class);
 
     /** Hybrid clock to generate timestamps. */
     protected final HybridClock clock = new HybridClockImpl();
 
     protected static BinaryRow binaryRow(TestKey key, TestValue value) {
         try {
-            Row row = kvMarshaller.marshal(key, value);
-
-            return new BinaryRowImpl(row.schemaVersion(), row.tupleSlice());
+            return KV_MARSHALLER.marshal(key, value);
         } catch (MarshallerException e) {
-            throw new IgniteException(e);
+            throw new IllegalArgumentException(e);
         }
     }
 
     protected static IndexRow indexRow(StorageIndexDescriptor indexDescriptor, BinaryRow binaryRow, RowId rowId) {
         int[] columnIndexes = indexDescriptor.columns().stream()
                 .mapToInt(indexColumnDescriptor -> {
-                    Column column = schemaDescriptor.column(indexColumnDescriptor.name());
+                    Column column = SCHEMA_DESCRIPTOR.column(indexColumnDescriptor.name());
 
                     assertNotNull(column, column.name());
 
@@ -96,24 +90,24 @@ public abstract class BaseMvStoragesTest {
                 })
                 .toArray();
 
-        ColumnsExtractor converter = BinaryRowConverter.columnsExtractor(schemaDescriptor, columnIndexes);
+        ColumnsExtractor converter = BinaryRowConverter.columnsExtractor(SCHEMA_DESCRIPTOR, columnIndexes);
         return new IndexRowImpl(converter.extractColumns(binaryRow), rowId);
     }
 
     protected static TestKey key(BinaryRow binaryRow) {
         try {
-            return kvMarshaller.unmarshalKey(Row.wrapBinaryRow(schemaDescriptor, binaryRow));
+            return KV_MARSHALLER.unmarshalKey(Row.wrapBinaryRow(SCHEMA_DESCRIPTOR, binaryRow));
         } catch (MarshallerException e) {
-            throw new IgniteException(e);
+            throw new IllegalArgumentException(e);
         }
     }
 
     @Nullable
     protected static TestValue value(BinaryRow binaryRow) {
         try {
-            return kvMarshaller.unmarshalValue(Row.wrapBinaryRow(schemaDescriptor, binaryRow));
+            return KV_MARSHALLER.unmarshalValue(Row.wrapBinaryRow(SCHEMA_DESCRIPTOR, binaryRow));
         } catch (MarshallerException e) {
-            throw new IgniteException(e);
+            throw new IllegalArgumentException(e);
         }
     }
 
@@ -145,12 +139,6 @@ public abstract class BaseMvStoragesTest {
         }
     }
 
-    protected static void assertRowMatches(@Nullable BinaryRow rowUnderQuestion, BinaryRow expectedRow) {
-        assertThat(rowUnderQuestion, is(notNullValue()));
-        assertThat(rowUnderQuestion.schemaVersion(), is(expectedRow.schemaVersion()));
-        assertThat(rowUnderQuestion.tupleSlice(), is(expectedRow.tupleSlice()));
-    }
-
     /**
      * Test pojo key.
      */
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java
index 74476e3b1b..aab1181582 100644
--- a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java
@@ -18,6 +18,9 @@
 package org.apache.ignite.internal.storage.pagememory.mv;
 
 import static java.util.stream.Collectors.joining;
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
 
 import java.util.stream.IntStream;
 import org.apache.ignite.internal.hlc.HybridTimestamp;
@@ -51,7 +54,7 @@ abstract class AbstractPageMemoryMvPartitionStorageTest extends AbstractMvPartit
 
         BinaryRow foundRow = read(rowId, HybridTimestamp.MAX_VALUE);
 
-        assertRowMatches(foundRow, longRow);
+        assertThat(foundRow, is(equalToRow(longRow)));
     }
 
     private BinaryRow rowStoredInFragments() {
@@ -77,7 +80,7 @@ abstract class AbstractPageMemoryMvPartitionStorageTest extends AbstractMvPartit
 
         BinaryRow foundRow = read(rowId, clock.now());
 
-        assertRowMatches(foundRow, longRow);
+        assertThat(foundRow, is(equalToRow(longRow)));
     }
 
     @Test
@@ -89,7 +92,7 @@ abstract class AbstractPageMemoryMvPartitionStorageTest extends AbstractMvPartit
         try (PartitionTimestampCursor cursor = storage.scan(HybridTimestamp.MAX_VALUE)) {
             BinaryRow foundRow = cursor.next().binaryRow();
 
-            assertRowMatches(foundRow, longRow);
+            assertThat(foundRow, is(equalToRow(longRow)));
         }
     }
 
@@ -104,7 +107,7 @@ abstract class AbstractPageMemoryMvPartitionStorageTest extends AbstractMvPartit
         try (PartitionTimestampCursor cursor = storage.scan(HybridTimestamp.MAX_VALUE)) {
             BinaryRow foundRow = cursor.next().binaryRow();
 
-            assertRowMatches(foundRow, longRow);
+            assertThat(foundRow, is(equalToRow(longRow)));
         }
     }
 }
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java
index 2dc0acd7f5..cd55784a08 100644
--- a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.storage.pagememory.mv;
 
 import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_PARTITION_COUNT;
 import static org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointState.FINISHED;
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
 import static org.apache.ignite.internal.storage.pagememory.configuration.schema.BasePageMemoryStorageEngineConfigurationSchema.DEFAULT_DATA_REGION_NAME;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.hamcrest.MatcherAssert.assertThat;
@@ -104,7 +105,7 @@ class PersistentPageMemoryMvPartitionStorageTest extends AbstractPageMemoryMvPar
 
         restartStorage();
 
-        assertRowMatches(binaryRow, read(rowId, HybridTimestamp.MAX_VALUE));
+        assertThat(read(rowId, HybridTimestamp.MAX_VALUE), is(equalToRow(binaryRow)));
     }
 
     private void restartStorage() throws Exception {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
index 1fbe5eb1f7..1e225e13a3 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
@@ -66,9 +66,9 @@ public abstract class IndexBaseTest extends BaseMvStoragesTest {
 
     private static final TableMessagesFactory MSG_FACTORY = new TableMessagesFactory();
 
-    private static final BinaryTupleSchema TUPLE_SCHEMA = BinaryTupleSchema.createRowSchema(schemaDescriptor);
+    private static final BinaryTupleSchema TUPLE_SCHEMA = BinaryTupleSchema.createRowSchema(SCHEMA_DESCRIPTOR);
 
-    private static final BinaryTupleSchema PK_INDEX_SCHEMA = BinaryTupleSchema.createKeySchema(schemaDescriptor);
+    private static final BinaryTupleSchema PK_INDEX_SCHEMA = BinaryTupleSchema.createKeySchema(SCHEMA_DESCRIPTOR);
 
     private static final ColumnsExtractor PK_INDEX_BINARY_TUPLE_CONVERTER = new ColumnsExtractor() {
         private final BinaryRowConverter converter = new BinaryRowConverter(TUPLE_SCHEMA, PK_INDEX_SCHEMA);
@@ -85,11 +85,11 @@ public abstract class IndexBaseTest extends BaseMvStoragesTest {
     };
 
     private static final int[] USER_INDEX_COLS = {
-            schemaDescriptor.column("INTVAL").schemaIndex(),
-            schemaDescriptor.column("STRVAL").schemaIndex()
+            SCHEMA_DESCRIPTOR.column("INTVAL").schemaIndex(),
+            SCHEMA_DESCRIPTOR.column("STRVAL").schemaIndex()
     };
 
-    private static final BinaryTupleSchema USER_INDEX_SCHEMA = BinaryTupleSchema.createSchema(schemaDescriptor, USER_INDEX_COLS);
+    private static final BinaryTupleSchema USER_INDEX_SCHEMA = BinaryTupleSchema.createSchema(SCHEMA_DESCRIPTOR, USER_INDEX_COLS);
 
     private static final ColumnsExtractor USER_INDEX_BINARY_TUPLE_CONVERTER = new ColumnsExtractor() {
         private final BinaryRowConverter converter = new BinaryRowConverter(TUPLE_SCHEMA, USER_INDEX_SCHEMA);
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
index 3011676259..83d5e31a09 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
@@ -17,16 +17,19 @@
 
 package org.apache.ignite.internal.table.distributed.replication;
 
+import static java.util.Collections.singletonList;
 import static java.util.concurrent.CompletableFuture.completedFuture;
 import static java.util.stream.Collectors.toList;
 import static org.apache.ignite.internal.distributionzones.DistributionZoneManager.DEFAULT_PARTITION_COUNT;
 import static org.apache.ignite.internal.hlc.HybridTimestamp.hybridTimestampToLong;
+import static org.apache.ignite.internal.schema.BinaryRowMatcher.equalToRow;
 import static org.apache.ignite.internal.testframework.asserts.CompletableFutureAssert.assertWillThrowFast;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowFast;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
 import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willSucceedFast;
 import static org.apache.ignite.internal.util.ArrayUtils.asList;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
@@ -46,7 +49,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
@@ -79,7 +81,6 @@ import org.apache.ignite.internal.raft.service.RaftGroupService;
 import org.apache.ignite.internal.replicator.TablePartitionId;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.BinaryRowConverter;
-import org.apache.ignite.internal.schema.BinaryRowImpl;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.ColumnsExtractor;
@@ -155,6 +156,7 @@ import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.network.TopologyService;
 import org.apache.ignite.sql.ColumnType;
 import org.apache.ignite.tx.TransactionException;
+import org.hamcrest.Matcher;
 import org.jetbrains.annotations.Nullable;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
@@ -333,11 +335,11 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
             if (txState == null) {
                 txMeta = null;
             } else if (txState == TxState.COMMITED) {
-                txMeta = new TxMeta(TxState.COMMITED, Collections.singletonList(grpId), txFixedTimestamp);
+                txMeta = new TxMeta(TxState.COMMITED, singletonList(grpId), txFixedTimestamp);
             } else {
                 assert txState == TxState.ABORTED : "Sate is " + txState;
 
-                txMeta = new TxMeta(TxState.ABORTED, Collections.singletonList(grpId), txFixedTimestamp);
+                txMeta = new TxMeta(TxState.ABORTED, singletonList(grpId), txFixedTimestamp);
             }
             return completedFuture(txMeta);
         });
@@ -474,7 +476,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
     public void testTxStateReplicaRequestCommitState() throws Exception {
         UUID txId = TestTransactionIds.newTransactionId();
 
-        txStateStorage.put(txId, new TxMeta(TxState.COMMITED, Collections.singletonList(grpId), clock.now()));
+        txStateStorage.put(txId, new TxMeta(TxState.COMMITED, singletonList(grpId), clock.now()));
 
         HybridTimestamp readTimestamp = clock.now();
 
@@ -1104,7 +1106,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
 
                 BinaryRow row = testMvPartitionStorage.read(rowId, HybridTimestamp.MAX_VALUE).binaryRow();
 
-                if (binaryRow.equals(row)) {
+                if (equalToRow(binaryRow).matches(row)) {
                     found = true;
                 }
             }
@@ -1246,25 +1248,28 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
         }
 
         if (multiple) {
-            Set<BinaryRow> allRows = insertFirst ? Set.of(br1Pk, br2Pk) : Set.of(br1Pk);
-            Set<BinaryRow> allRowsButModified = insertFirst ? Set.of(br2) : Set.of();
-            Set<BinaryRow> expected = committed
+            List<BinaryRow> allRowsPks = insertFirst ? List.of(br1Pk, br2Pk) : List.of(br1Pk);
+            List<BinaryRow> allRows = insertFirst ? List.of(br1, br2) : List.of(br1);
+            List<BinaryRow> allRowsButModified = insertFirst ? Arrays.asList(null, br2) : singletonList((BinaryRow) null);
+            List<BinaryRow> expected = committed
                     ? (upsertAfterDelete ? allRows : allRowsButModified)
-                    : (insertFirst ? allRows : Set.of());
-            Set<BinaryRow> res = new HashSet<>(roGetAll(allRows, clock.nowLong()));
+                    : (insertFirst ? allRows : singletonList((BinaryRow) null));
+            List<BinaryRow> res = roGetAll(allRowsPks, clock.nowLong());
 
             assertEquals(allRows.size(), res.size());
-            for (BinaryRow e : expected) {
-                // TODO: IGNITE-19430 - should there be an assertion in the next line?
-                res.contains(e);
-            }
+
+            List<Matcher<? super BinaryRow>> matchers = expected.stream()
+                    .map(row -> row == null ? nullValue(BinaryRow.class) : equalToRow(row))
+                    .collect(toList());
+
+            assertThat(res, contains(matchers));
         } else {
             BinaryRow res = roGet(br1Pk, clock.nowLong());
             BinaryRow expected = committed
                     ? (upsertAfterDelete ? br1 : null)
                     : (insertFirst ? br1 : null);
 
-            assertEquals(expected, res);
+            assertThat(res, is(expected == null ? nullValue(BinaryRow.class) : equalToRow(expected)));
         }
 
         cleanup(tx1);
@@ -1694,9 +1699,7 @@ public class PartitionReplicaListenerTest extends IgniteAbstractTest {
 
     private static BinaryRow binaryRow(TestKey key, TestValue value, KvMarshaller<TestKey, TestValue> marshaller) {
         try {
-            Row row = marshaller.marshal(key, value);
-
-            return new BinaryRowImpl(row.schemaVersion(), row.tupleSlice());
+            return marshaller.marshal(key, value);
         } catch (MarshallerException e) {
             throw new AssertionError(e);
         }