You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/05/19 15:34:08 UTC

[GitHub] [ignite-3] ibessonov commented on a diff in pull request #814: IGNITE-16933 PageMemory-based MV storage implementation

ibessonov commented on code in PR #814:
URL: https://github.com/apache/ignite-3/pull/814#discussion_r877147495


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/datapage/DataPageReader.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.pagememory.datapage;
+
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.itemId;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageId;
+import static org.apache.ignite.internal.util.GridUnsafe.NATIVE_BYTE_ORDER;
+import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.DataPagePayload;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Contains logic for reading values from page memory data pages (this includes handling multy-page values).
+ */
+public abstract class DataPageReader<T> {

Review Comment:
   Ok, it will be hard to describe my complaints, but I'll try. As far as I see, you tried to use the same abstraction for all data reads. Abstraction is good, but too much of it is bad for performance. Now, let me be more specific:
   - reading version row, generally speaking, does not require you reading the entire payload. You read timestamp and after that decide, whether you should read the payload or only return a link to the next version.
   - payload size is known when you finished reading the header, ByteArrayOutputStream will lead to extra allocation.
   
   RowId implementation doesn't support fragmentation, for example, but it must implement the method. Should we even attempt to introduce a common abstraction for two implementations? I think not. Even API should be different for them (chain reading needs a timestamp parameter).



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageMemoryDataRegion.java:
##########
@@ -33,4 +34,11 @@ public interface PageMemoryDataRegion {
      */
     @Nullable
     PageMemory pageMemory();
+
+    /**
+     * Returns page memory or throws an exception if not started.
+     */
+    default PageMemory requiredPageMemory() {

Review Comment:
   I believe that no methods should ever be called before start, except for constructor, of course. So, why would you use this new method then. I think that we should change javadoc and remove Nullable annotation, let the implementation of the original method throw NPE if pageMemory is null.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/StorableBase.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.pagememory;
+
+/**
+ * Base of {@link Storable} containing useful trivial implementations of some methods.
+ */
+public abstract class StorableBase implements Storable {

Review Comment:
   This one is weird, it has a single implementation. That implementation calls "link" method right after "super" constructor. Two question:
    - should constructor have two arguments?
    - should this class even exist?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -0,0 +1,515 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+
+import java.nio.ByteBuffer;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.PageMemoryDataRegion;
+import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.NoUncommittedVersionException;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.StorageUtils;
+import org.apache.ignite.internal.storage.TxIdMismatchException;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryDataRegion;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.IgniteCursor;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link MvPartitionStorage} using Page Memory.
+ *
+ * @see MvPartitionStorage
+ */
+public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
+    private static final byte[] TOMBSTONE_PAYLOAD = new byte[0];
+    private static final Predicate<BinaryRow> MATCH_ALL = row -> true;
+
+    private final int partitionId;
+    private final int groupId;
+
+    private final VersionChainFreeList versionChainFreeList;

Review Comment:
   Why don't you separate fields with empty lines?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -0,0 +1,515 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+
+import java.nio.ByteBuffer;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.PageMemoryDataRegion;
+import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.NoUncommittedVersionException;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.StorageUtils;
+import org.apache.ignite.internal.storage.TxIdMismatchException;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryDataRegion;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.IgniteCursor;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link MvPartitionStorage} using Page Memory.
+ *
+ * @see MvPartitionStorage
+ */
+public class PageMemoryMvPartitionStorage implements MvPartitionStorage {

Review Comment:
   I'd also call it volatile right now, to avoid confusion in the future.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/LinkRowId.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import org.apache.ignite.internal.pagememory.util.PageIdUtils;
+import org.apache.ignite.internal.storage.RowId;
+
+/**
+ * {@link RowId} implementation which identifies the row data using row link in page-memory.
+ *
+ * @see RowId
+ * @see PageIdUtils#link(long, int)
+ */
+public class LinkRowId implements RowId {
+    private final long rowLink;
+
+    public LinkRowId(long rowLink) {
+        this.rowLink = rowLink;
+    }
+
+    @Override
+    public int partitionId() {
+        long pageId = PageIdUtils.pageId(rowLink);
+        return PageIdUtils.partitionId(pageId);

Review Comment:
   You can pass rowLink directly into "partitionId", it's fine. We don't abuse data format here, it's well documented. Or, if you wish to leave it this way, please add a separating line :)



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/RowVersionFreeList.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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.INDEX_PARTITION;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.evict.PageEvictionTracker;
+import org.apache.ignite.internal.pagememory.freelist.AbstractFreeList;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.util.PageLockListener;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@link AbstractFreeList} for {@link RowVersion} instances.
+ */
+public class RowVersionFreeList extends AbstractFreeList<RowVersion> {
+    private static final IgniteLogger LOG = IgniteLogger.forClass(RowVersionFreeList.class);
+
+    private final IoStatisticsHolder statHolder;
+
+    /**
+     * Constructor.
+     *
+     * @param grpId              Group ID.
+     * @param pageMem            Page memory.
+     * @param lockLsnr           Page lock listener.
+     * @param metaPageId         Metadata page ID.
+     * @param initNew            {@code True} if new metadata should be initialized.
+     * @param pageListCacheLimit Page list cache limit.
+     * @param evictionTracker    Page eviction tracker.
+     * @param statHolder         Statistics holder to track IO operations.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public RowVersionFreeList(
+            int grpId,
+            PageMemory pageMem,
+            PageLockListener lockLsnr,
+            long metaPageId,
+            boolean initNew,
+            @Nullable AtomicLong pageListCacheLimit,
+            PageEvictionTracker evictionTracker,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        super(
+                grpId,
+                "RowVersionFreeList_" + grpId,
+                pageMem,
+                null,
+                lockLsnr,
+                FLAG_AUX,
+                LOG,
+                metaPageId,
+                initNew,
+                pageListCacheLimit,
+                evictionTracker
+        );
+
+        this.statHolder = statHolder;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    protected long allocatePageNoReuse() throws IgniteInternalCheckedException {
+        return pageMem.allocatePage(grpId, INDEX_PARTITION, defaultPageFlag);

Review Comment:
   I believe other structures store partitionId in a field, please check it. INDEX_PARTITION is only applicable in volatile case, persistent storage will need a valid partition number here.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChain.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import java.util.UUID;
+import org.apache.ignite.internal.pagememory.Storable;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.storage.pagememory.mv.io.VersionChainDataIo;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents row version chain: that is, all versions of the row plus some row-level metadata.
+ */
+public class VersionChain extends VersionChainLink implements Storable {
+    public static long NULL_UUID_COMPONENT = 0;
+
+    private final int partitionId;
+    @Nullable
+    private final UUID transactionId;
+    private final long headLink;
+
+    // TODO: IGNITE-17008 - add nextLink
+
+    /**
+     * Constructs a VersionChain without a transaction ID.
+     */
+    public static VersionChain withoutTxId(int partitionId, long link, long headLink) {
+        return new VersionChain(partitionId, link, null, headLink);
+    }
+
+    /**
+     * Constructor.
+     */
+    public VersionChain(int partitionId, @Nullable UUID transactionId, long headLink) {
+        this.partitionId = partitionId;
+        this.transactionId = transactionId;
+        this.headLink = headLink;
+    }
+
+    /**
+     * Constructor.
+     */
+    public VersionChain(int partitionId, long link, @Nullable UUID transactionId, long headLink) {
+        super(link);
+        this.partitionId = partitionId;
+        this.transactionId = transactionId;
+        this.headLink = headLink;
+    }
+
+    @Nullable
+    public UUID transactionId() {
+        return transactionId;
+    }
+
+    public long headLink() {
+        return headLink;
+    }
+
+    @Override
+    public final int partition() {
+        return partitionId;
+    }
+
+    @Override
+    public int size() {
+        return transactioIdStoreSize() + headLinkStoreSize();
+    }
+
+    private int transactioIdStoreSize() {
+        return 2 * Long.BYTES;
+    }
+
+    private int headLinkStoreSize() {
+        return PartitionlessLinks.PARTITIONLESS_LINK_SIZE_BYTES;
+    }
+
+    @Override
+    public int headerSize() {
+        return 0;
+    }
+
+    @Override
+    public IoVersions<? extends AbstractDataPageIo> ioVersions() {
+        return VersionChainDataIo.VERSIONS;
+    }
+
+    @Override
+    public String toString() {
+        return "VersionChain{"

Review Comment:
   Is this a auto-generated toString? Usually we use S.toString or manual formatting with brackets []



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainFreeList.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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.INDEX_PARTITION;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.evict.PageEvictionTracker;
+import org.apache.ignite.internal.pagememory.freelist.AbstractFreeList;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.util.PageLockListener;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@link AbstractFreeList} for {@link VersionChain} instances.
+ */
+public class VersionChainFreeList extends AbstractFreeList<VersionChain> {
+    private static final IgniteLogger LOG = IgniteLogger.forClass(VersionChainFreeList.class);
+
+    private final IoStatisticsHolder statHolder;
+
+    /**
+     * Constructor.
+     *
+     * @param grpId              Group ID.
+     * @param pageMem            Page memory.
+     * @param lockLsnr           Page lock listener.
+     * @param metaPageId         Metadata page ID.
+     * @param initNew            {@code True} if new metadata should be initialized.
+     * @param pageListCacheLimit Page list cache limit.
+     * @param evictionTracker    Page eviction tracker.
+     * @param statHolder         Statistics holder to track IO operations.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public VersionChainFreeList(
+            int grpId,
+            PageMemory pageMem,
+            PageLockListener lockLsnr,
+            long metaPageId,
+            boolean initNew,
+            @Nullable AtomicLong pageListCacheLimit,
+            PageEvictionTracker evictionTracker,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        super(
+                grpId,
+                "VersionChainFreeList_" + grpId,
+                pageMem,
+                null,
+                lockLsnr,
+                FLAG_AUX,
+                LOG,
+                metaPageId,
+                initNew,
+                pageListCacheLimit,
+                evictionTracker
+        );
+
+        this.statHolder = statHolder;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    protected long allocatePageNoReuse() throws IgniteInternalCheckedException {
+        return pageMem.allocatePage(grpId, INDEX_PARTITION, defaultPageFlag);

Review Comment:
   Same here



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -673,6 +737,18 @@ public DataPagePayload readPayload(final long pageAddr, final int itemId, final
                 nextLink);
     }
 
+    /**
+     * Returns {@code true} iff an item with given ID exists in a page at the given address.
+     *
+     * @param pageAddr address where page data begins in memory
+     * @param itemId   item ID to check
+     * @param pageSize size of the page in bytes
+     * @return {@code true} iff an item with given ID exists in a page at the given address
+     */
+    public boolean itemExists(long pageAddr, int itemId, final int pageSize) {

Review Comment:
   As far as I see, this method doesn't validate the case when you had a direct itemId (that's not the maximal item id on the page) and you delete it. After that method should return "false". But I suspect that the result will be "true"



##########
modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java:
##########
@@ -295,4 +284,334 @@ private List<TestValue> convert(Cursor<BinaryRow> cursor) throws Exception {
                     .collect(toList());
         }
     }
+
+    @Test
+    void readOfUncommittedRowWithCorrespondingTransactionIdReturnsTheRow() {

Review Comment:
   I thought we have most of these new scenarios, but with deserialized objects instead of byte arrays. Why did you add them?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -611,24 +626,73 @@ protected int getDataOffset(long pageAddr, int itemId, int pageSize) {
 
         assert directCnt > 0 : "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(pageAddr, pageSize);
 
+        final int directItemId;

Review Comment:
   We don't usually declare variables as "final". I'm not sure that there's a rule about it, probably not, but code looks less uniform with both final and effectively-final variables in it.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java:
##########
@@ -62,32 +68,77 @@ public void start() {
         this.pageMemory = pageMemory;
 
         try {
-            int grpId = 0;
-
-            long metaPageId = pageMemory.allocatePage(grpId, INDEX_PARTITION, FLAG_AUX);
-
-            this.freeList = new TableFreeList(
-                    grpId,
-                    pageMemory,
-                    PageLockListenerNoOp.INSTANCE,
-                    metaPageId,
-                    true,
-                    null,
-                    PageEvictionTrackerNoOp.INSTANCE,
-                    IoStatisticsHolderNoOp.INSTANCE
-            );
+            this.tableFreeList = createTableFreeList(pageMemory, FREE_LIST_GROUP_ID);

Review Comment:
   Things are a little bit more complicated in reality. Now that you have more than 1 free lists, you should use the same "reuse list" for all of them to avoid leaks.
   (well, there will be no real leaks, just poor memory management).
   For these purposes you can use one of 3 freelists, its reuse bucket will become reuse list for the entire region.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -0,0 +1,515 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+
+import java.nio.ByteBuffer;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.PageMemoryDataRegion;
+import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.NoUncommittedVersionException;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.StorageUtils;
+import org.apache.ignite.internal.storage.TxIdMismatchException;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryDataRegion;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.IgniteCursor;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link MvPartitionStorage} using Page Memory.
+ *
+ * @see MvPartitionStorage
+ */
+public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
+    private static final byte[] TOMBSTONE_PAYLOAD = new byte[0];
+    private static final Predicate<BinaryRow> MATCH_ALL = row -> true;
+
+    private final int partitionId;
+    private final int groupId;
+
+    private final VersionChainFreeList versionChainFreeList;
+    private final VersionChainTree versionChainTree;
+    private final DataPageReader<VersionChain> versionChainDataPageReader;
+    private final RowVersionFreeList rowVersionFreeList;
+    private final DataPageReader<RowVersion> rowVersionDataPageReader;
+
+    /**
+     * Constructor.
+     */
+    public PageMemoryMvPartitionStorage(int partitionId, TableView tableConfig, PageMemoryDataRegion dataRegion) {
+        this.partitionId = partitionId;
+
+        groupId = StorageUtils.groupId(tableConfig);
+
+        versionChainFreeList = ((VolatilePageMemoryDataRegion) dataRegion).versionChainFreeList();
+        rowVersionFreeList = ((VolatilePageMemoryDataRegion) dataRegion).rowVersionFreeList();
+
+        try {
+            versionChainTree = createVersionChainTree(partitionId, tableConfig, dataRegion, versionChainFreeList);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Error occurred while creating the partition storage", e);
+        }
+
+        versionChainDataPageReader = new VersionChainDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
+        rowVersionDataPageReader = new RowVersionDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
+    }
+
+    private VersionChainTree createVersionChainTree(
+            int partitionId,
+            TableView tableConfig,
+            PageMemoryDataRegion dataRegion,
+            VersionChainFreeList versionChainFreeList1
+    ) throws IgniteInternalCheckedException {
+        // TODO: IGNITE-16641 It is necessary to do getting the tree root for the persistent case.
+        long metaPageId = dataRegion.requiredPageMemory().allocatePage(groupId, partitionId, FLAG_AUX);
+
+        // TODO: IGNITE-16641 It is necessary to take into account the persistent case.
+        boolean initNew = true;
+
+        return new VersionChainTree(
+                groupId,
+                tableConfig.name(),
+                dataRegion.pageMemory(),
+                PageLockListenerNoOp.INSTANCE,
+                new AtomicLong(),
+                metaPageId,
+                versionChainFreeList1,
+                partitionId,
+                initNew
+        );
+    }
+
+    @Override
+    public @Nullable BinaryRow read(RowId rowId, UUID txId) throws TxIdMismatchException, StorageException {
+        VersionChain versionChain = findVersionChain(rowId);
+        if (versionChain == null) {
+            return null;
+        }
+
+        return findLatestRowVersion(versionChain, txId, MATCH_ALL);
+    }
+
+    @Override
+    public @Nullable BinaryRow read(RowId rowId, Timestamp timestamp) throws StorageException {
+        VersionChain versionChain = findVersionChain(rowId);
+        if (versionChain == null) {
+            return null;
+        }
+
+        return findRowVersionByTimestamp(versionChain, timestamp);
+    }
+
+    @Nullable
+    private VersionChain findVersionChain(RowId rowId) {
+        try {
+            return versionChainDataPageReader.getRowByLink(versionChainLinkFrom(rowId));
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Version chain lookup failed", e);
+        }
+    }
+
+    private long versionChainLinkFrom(RowId rowId) {
+        if (rowId.partitionId() != partitionId) {
+            throw new IllegalArgumentException("I own partition " + partitionId + " but I was given RowId with partition "
+                    + rowId.partitionId());
+        }
+
+        LinkRowId linkRowId = (LinkRowId) rowId;
+
+        return linkRowId.versionChainLink();
+    }
+
+    @Nullable
+    private ByteBufferRow findLatestRowVersion(VersionChain versionChain, UUID txId, Predicate<BinaryRow> keyFilter) {
+        RowVersion rowVersion = findLatestRowVersion(versionChain);
+        ByteBufferRow row = rowVersionToBinaryRow(rowVersion);
+
+        if (!keyFilter.test(row)) {
+            return null;
+        }
+
+        throwIfChainBelongsToAnotherTx(versionChain, txId);
+
+        return row;
+    }
+
+    private RowVersion findLatestRowVersion(VersionChain versionChain) {
+        return findRowVersion(versionChain.headLink());
+    }
+
+    private RowVersion findRowVersion(long nextRowPartitionlessLink) {
+        long nextLink = PartitionlessLinks.addPartitionIdToPartititionlessLink(nextRowPartitionlessLink, partitionId);
+
+        try {
+            return rowVersionDataPageReader.getRowByLink(nextLink);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Row version lookup failed");
+        }
+    }
+
+    private void throwIfChainBelongsToAnotherTx(VersionChain versionChain, UUID txId) {
+        if (versionChain.transactionId() != null && !txId.equals(versionChain.transactionId())) {
+            throw new TxIdMismatchException();
+        }
+    }
+
+    @Nullable
+    private ByteBufferRow rowVersionToBinaryRow(RowVersion rowVersion) {
+        if (rowVersion.isTombstone()) {
+            return null;
+        }
+
+        return new ByteBufferRow(rowVersion.value());
+    }
+
+    @Nullable
+    private ByteBufferRow findRowVersionInChain(
+            VersionChain versionChain,
+            @Nullable UUID transactionId,
+            @Nullable Timestamp timestamp,
+            Predicate<BinaryRow> keyFilter
+    ) {
+        assert transactionId != null ^ timestamp != null;
+
+        if (transactionId != null) {
+            return findLatestRowVersion(versionChain, transactionId, keyFilter);
+        } else {
+            ByteBufferRow row = findRowVersionByTimestamp(versionChain, timestamp);
+            return keyFilter.test(row) ? row : null;
+        }
+    }
+
+    @Nullable
+    private ByteBufferRow findRowVersionByTimestamp(VersionChain versionChain, Timestamp timestamp) {
+        long nextRowPartitionlessLink = versionChain.headLink();
+
+        while (true) {
+            RowVersion rowVersion = findRowVersion(nextRowPartitionlessLink);
+            Timestamp rowVersionTs = rowVersion.timestamp();
+            if (rowVersionTs != null && rowVersionTs.beforeOrEquals(timestamp)) {
+                return rowVersionToBinaryRow(rowVersion);
+            }
+
+            if (!rowVersion.hasNextLink()) {
+                return null;
+            }
+
+            nextRowPartitionlessLink = rowVersion.nextLink();
+        }
+    }
+
+    @Override
+    public LinkRowId insert(BinaryRow row, UUID txId) throws StorageException {
+        RowVersion rowVersion = insertRowVersion(Objects.requireNonNull(row), RowVersion.NULL_LINK);
+
+        VersionChain versionChain = new VersionChain(partitionId, txId, PartitionlessLinks.removePartitionIdFromLink(rowVersion.link()));
+
+        try {
+            versionChainFreeList.insertDataRow(versionChain);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot store a version chain", e);
+        }
+
+        try {
+            versionChainTree.putx(versionChain);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot put a version chain to the tree", e);
+        }
+
+        return new LinkRowId(versionChain.link());
+    }
+
+    private RowVersion insertRowVersion(@Nullable BinaryRow row, long nextPartitionlessLink) {
+        // TODO IGNITE-16913 Add proper way to write row bytes into array without allocations.
+        byte[] rowBytes = row == null ? TOMBSTONE_PAYLOAD : row.bytes();
+
+        RowVersion rowVersion = new RowVersion(partitionId, nextPartitionlessLink, ByteBuffer.wrap(rowBytes));
+
+        insertRowVersion(rowVersion);
+
+        return rowVersion;
+    }
+
+    private void insertRowVersion(RowVersion rowVersion) {
+        try {
+            rowVersionFreeList.insertDataRow(rowVersion);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot store a row version", e);
+        }
+    }
+
+    @Override
+    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException {
+        VersionChain currentChain = findVersionChainForModification(rowId);
+
+        throwIfChainBelongsToAnotherTx(currentChain, txId);
+
+        RowVersion currentVersion = findLatestRowVersion(currentChain);
+        RowVersion newVersion = insertRowVersion(row, currentVersion.isUncommitted() ? currentVersion.nextLink() : currentChain.headLink());
+
+        if (currentVersion.isUncommitted()) {
+            // as we replace an uncommitted version with new one, we need to remove old uncommitted version
+            removeRowVersion(currentVersion);
+        }
+
+        VersionChain chainReplacement = new VersionChain(
+                partitionId,
+                txId,
+                PartitionlessLinks.removePartitionIdFromLink(newVersion.link())
+        );
+
+        updateVersionChain(currentChain, chainReplacement);
+
+        if (currentVersion.isUncommitted()) {
+            return rowVersionToBinaryRow(currentVersion);
+        } else {
+            return null;
+        }
+    }
+
+    @NotNull
+    private VersionChain findVersionChainForModification(RowId rowId) {
+        VersionChain currentChain = findVersionChain(rowId);
+        if (currentChain == null) {
+            throw new RowIdIsInvalidForModificationsException();
+        }
+        return currentChain;
+    }
+
+    @Override
+    public @Nullable BinaryRow abortWrite(RowId rowId) throws StorageException {
+        VersionChain currentVersionChain = findVersionChainForModification(rowId);
+
+        if (currentVersionChain.transactionId() == null) {
+            throw new NoUncommittedVersionException();
+        }
+
+        RowVersion currentVersion = findLatestRowVersion(currentVersionChain);
+        assert currentVersion.isUncommitted();
+
+        removeRowVersion(currentVersion);
+
+        if (currentVersion.hasNextLink()) {
+            VersionChain versionChainReplacement = VersionChain.withoutTxId(
+                    partitionId,
+                    currentVersionChain.link(),
+                    currentVersion.nextLink()
+            );
+            updateVersionChain(currentVersionChain, versionChainReplacement);
+        } else {
+            // it was the only version, let's remove the chain as well
+            removeVersionChain(currentVersionChain);
+        }
+
+        return rowVersionToBinaryRow(currentVersion);
+    }
+
+    private void removeVersionChain(VersionChain currentVersionChain) {
+        try {
+            versionChainFreeList.removeDataRowByLink(currentVersionChain.link());
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot remove chain version", e);
+        }
+    }
+
+    @Override
+    public void commitWrite(RowId rowId, Timestamp timestamp) throws StorageException {
+        VersionChain currentVersionChain = findVersionChainForModification(rowId);

Review Comment:
   I don't think that "VersionChain" is a good name for what we have. It's more like a descriptor, a head. Chain is what it refers to. Also, it's easy to confuse with "Versioned Chain", the name that's used in many sources about MV storages.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/RowVersion.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.StorableBase;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.storage.pagememory.mv.io.RowVersionDataIo;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents row version inside row version chain.
+ */
+public class RowVersion extends StorableBase {
+    /**
+     * A 'timestamp' representing absense of a timestamp.
+     */
+    public static final Timestamp NULL_TIMESTAMP = new Timestamp(Long.MIN_VALUE, Long.MIN_VALUE);

Review Comment:
   I proposed using MAX_VALUE, what's the reasoning here?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/RowVersion.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.StorableBase;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.storage.pagememory.mv.io.RowVersionDataIo;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents row version inside row version chain.
+ */
+public class RowVersion extends StorableBase {
+    /**
+     * A 'timestamp' representing absense of a timestamp.
+     */
+    public static final Timestamp NULL_TIMESTAMP = new Timestamp(Long.MIN_VALUE, Long.MIN_VALUE);
+    /**
+     * Represents an absent partitionless link.
+     */
+    public static final long NULL_LINK = 0;
+
+    @Nullable
+    private final Timestamp timestamp;
+    private final long nextLink;
+    private final int valueSize;
+    private final ByteBuffer value;
+
+    /**
+     * Constructor.
+     */
+    public RowVersion(int partitionId, long nextLink, ByteBuffer value) {
+        this(partitionId, 0, null, nextLink, value);
+    }
+
+    /**
+     * Constructor.
+     */
+    public RowVersion(int partitionId, long link, @Nullable Timestamp timestamp, long nextLink, ByteBuffer value) {
+        super(partitionId);
+        link(link);
+
+        assert !NULL_TIMESTAMP.equals(timestamp) : "Null timestamp provided";
+
+        this.timestamp = timestamp;
+        this.nextLink = nextLink;
+        this.valueSize = value.limit();
+        this.value = value;
+    }
+
+    @Nullable
+    public Timestamp timestamp() {
+        return timestamp;
+    }
+
+    public Timestamp timestampForStorage() {
+        return timestamp == null ? NULL_TIMESTAMP : timestamp;
+    }
+
+    /**
+     * Returns partitionless link of the next version or {@code 0} if this version is the last in the chain (i.e. it's the oldest version).
+     *
+     * @return partitionless link of the next version or {@code 0} if this version is the last in the chain
+     */
+    public long nextLink() {
+        return nextLink;
+    }
+
+    public int valueSize() {
+        return valueSize;
+    }
+
+    public ByteBuffer value() {
+        return value;
+    }
+
+    public boolean hasNextLink() {
+        return nextLink != NULL_LINK;
+    }
+
+    boolean isTombstone() {
+        return valueSize() == 0;
+    }
+
+    boolean isUncommitted() {
+        return timestamp == null;
+    }
+
+    boolean isCommitted() {
+        return timestamp != null;
+    }
+
+    @Override
+    public int size() {
+        return timestampStoreSize() + nextLinkStoreSize() + valueStoreSize();
+    }
+
+    private int timestampStoreSize() {

Review Comment:
   Can these be constants instead of methods? It's very unusual for Ignite to see code like this.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChain.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import java.util.UUID;
+import org.apache.ignite.internal.pagememory.Storable;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.storage.pagememory.mv.io.VersionChainDataIo;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents row version chain: that is, all versions of the row plus some row-level metadata.
+ */
+public class VersionChain extends VersionChainLink implements Storable {
+    public static long NULL_UUID_COMPONENT = 0;
+
+    private final int partitionId;
+    @Nullable
+    private final UUID transactionId;
+    private final long headLink;
+
+    // TODO: IGNITE-17008 - add nextLink
+
+    /**
+     * Constructs a VersionChain without a transaction ID.
+     */
+    public static VersionChain withoutTxId(int partitionId, long link, long headLink) {
+        return new VersionChain(partitionId, link, null, headLink);
+    }
+
+    /**
+     * Constructor.
+     */
+    public VersionChain(int partitionId, @Nullable UUID transactionId, long headLink) {
+        this.partitionId = partitionId;
+        this.transactionId = transactionId;
+        this.headLink = headLink;
+    }
+
+    /**
+     * Constructor.
+     */
+    public VersionChain(int partitionId, long link, @Nullable UUID transactionId, long headLink) {
+        super(link);
+        this.partitionId = partitionId;
+        this.transactionId = transactionId;
+        this.headLink = headLink;
+    }
+
+    @Nullable
+    public UUID transactionId() {
+        return transactionId;
+    }
+
+    public long headLink() {
+        return headLink;
+    }
+
+    @Override
+    public final int partition() {
+        return partitionId;
+    }
+
+    @Override
+    public int size() {
+        return transactioIdStoreSize() + headLinkStoreSize();
+    }
+
+    private int transactioIdStoreSize() {
+        return 2 * Long.BYTES;
+    }
+
+    private int headLinkStoreSize() {
+        return PartitionlessLinks.PARTITIONLESS_LINK_SIZE_BYTES;
+    }
+
+    @Override
+    public int headerSize() {
+        return 0;

Review Comment:
   This is not a bug, but I'd recommend using meaningful value here. It'll help you later with optimal "read" implementation.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -0,0 +1,515 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+
+import java.nio.ByteBuffer;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.PageMemoryDataRegion;
+import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.NoUncommittedVersionException;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.StorageUtils;
+import org.apache.ignite.internal.storage.TxIdMismatchException;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryDataRegion;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.IgniteCursor;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link MvPartitionStorage} using Page Memory.
+ *
+ * @see MvPartitionStorage
+ */
+public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
+    private static final byte[] TOMBSTONE_PAYLOAD = new byte[0];
+    private static final Predicate<BinaryRow> MATCH_ALL = row -> true;
+
+    private final int partitionId;
+    private final int groupId;
+
+    private final VersionChainFreeList versionChainFreeList;
+    private final VersionChainTree versionChainTree;
+    private final DataPageReader<VersionChain> versionChainDataPageReader;
+    private final RowVersionFreeList rowVersionFreeList;
+    private final DataPageReader<RowVersion> rowVersionDataPageReader;
+
+    /**
+     * Constructor.
+     */
+    public PageMemoryMvPartitionStorage(int partitionId, TableView tableConfig, PageMemoryDataRegion dataRegion) {
+        this.partitionId = partitionId;
+
+        groupId = StorageUtils.groupId(tableConfig);
+
+        versionChainFreeList = ((VolatilePageMemoryDataRegion) dataRegion).versionChainFreeList();
+        rowVersionFreeList = ((VolatilePageMemoryDataRegion) dataRegion).rowVersionFreeList();
+
+        try {
+            versionChainTree = createVersionChainTree(partitionId, tableConfig, dataRegion, versionChainFreeList);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Error occurred while creating the partition storage", e);
+        }
+
+        versionChainDataPageReader = new VersionChainDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
+        rowVersionDataPageReader = new RowVersionDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
+    }
+
+    private VersionChainTree createVersionChainTree(
+            int partitionId,
+            TableView tableConfig,
+            PageMemoryDataRegion dataRegion,
+            VersionChainFreeList versionChainFreeList1
+    ) throws IgniteInternalCheckedException {
+        // TODO: IGNITE-16641 It is necessary to do getting the tree root for the persistent case.
+        long metaPageId = dataRegion.requiredPageMemory().allocatePage(groupId, partitionId, FLAG_AUX);
+
+        // TODO: IGNITE-16641 It is necessary to take into account the persistent case.
+        boolean initNew = true;
+
+        return new VersionChainTree(
+                groupId,
+                tableConfig.name(),
+                dataRegion.pageMemory(),
+                PageLockListenerNoOp.INSTANCE,
+                new AtomicLong(),
+                metaPageId,
+                versionChainFreeList1,
+                partitionId,
+                initNew
+        );
+    }
+
+    @Override
+    public @Nullable BinaryRow read(RowId rowId, UUID txId) throws TxIdMismatchException, StorageException {
+        VersionChain versionChain = findVersionChain(rowId);
+        if (versionChain == null) {
+            return null;
+        }
+
+        return findLatestRowVersion(versionChain, txId, MATCH_ALL);
+    }
+
+    @Override
+    public @Nullable BinaryRow read(RowId rowId, Timestamp timestamp) throws StorageException {
+        VersionChain versionChain = findVersionChain(rowId);
+        if (versionChain == null) {
+            return null;
+        }
+
+        return findRowVersionByTimestamp(versionChain, timestamp);
+    }
+
+    @Nullable
+    private VersionChain findVersionChain(RowId rowId) {
+        try {
+            return versionChainDataPageReader.getRowByLink(versionChainLinkFrom(rowId));
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Version chain lookup failed", e);
+        }
+    }
+
+    private long versionChainLinkFrom(RowId rowId) {
+        if (rowId.partitionId() != partitionId) {
+            throw new IllegalArgumentException("I own partition " + partitionId + " but I was given RowId with partition "
+                    + rowId.partitionId());
+        }
+
+        LinkRowId linkRowId = (LinkRowId) rowId;
+
+        return linkRowId.versionChainLink();
+    }
+
+    @Nullable
+    private ByteBufferRow findLatestRowVersion(VersionChain versionChain, UUID txId, Predicate<BinaryRow> keyFilter) {
+        RowVersion rowVersion = findLatestRowVersion(versionChain);
+        ByteBufferRow row = rowVersionToBinaryRow(rowVersion);
+
+        if (!keyFilter.test(row)) {
+            return null;
+        }
+
+        throwIfChainBelongsToAnotherTx(versionChain, txId);
+
+        return row;
+    }
+
+    private RowVersion findLatestRowVersion(VersionChain versionChain) {
+        return findRowVersion(versionChain.headLink());
+    }
+
+    private RowVersion findRowVersion(long nextRowPartitionlessLink) {
+        long nextLink = PartitionlessLinks.addPartitionIdToPartititionlessLink(nextRowPartitionlessLink, partitionId);
+
+        try {
+            return rowVersionDataPageReader.getRowByLink(nextLink);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Row version lookup failed");
+        }
+    }
+
+    private void throwIfChainBelongsToAnotherTx(VersionChain versionChain, UUID txId) {
+        if (versionChain.transactionId() != null && !txId.equals(versionChain.transactionId())) {
+            throw new TxIdMismatchException();
+        }
+    }
+
+    @Nullable
+    private ByteBufferRow rowVersionToBinaryRow(RowVersion rowVersion) {
+        if (rowVersion.isTombstone()) {
+            return null;
+        }
+
+        return new ByteBufferRow(rowVersion.value());
+    }
+
+    @Nullable
+    private ByteBufferRow findRowVersionInChain(
+            VersionChain versionChain,
+            @Nullable UUID transactionId,
+            @Nullable Timestamp timestamp,
+            Predicate<BinaryRow> keyFilter
+    ) {
+        assert transactionId != null ^ timestamp != null;
+
+        if (transactionId != null) {
+            return findLatestRowVersion(versionChain, transactionId, keyFilter);
+        } else {
+            ByteBufferRow row = findRowVersionByTimestamp(versionChain, timestamp);
+            return keyFilter.test(row) ? row : null;
+        }
+    }
+
+    @Nullable
+    private ByteBufferRow findRowVersionByTimestamp(VersionChain versionChain, Timestamp timestamp) {
+        long nextRowPartitionlessLink = versionChain.headLink();
+
+        while (true) {
+            RowVersion rowVersion = findRowVersion(nextRowPartitionlessLink);
+            Timestamp rowVersionTs = rowVersion.timestamp();
+            if (rowVersionTs != null && rowVersionTs.beforeOrEquals(timestamp)) {

Review Comment:
   I would actually move a comparison method into RowVersion. First reason - it bloats the API of Timestamp. Seconds - it allows you to store timestamp as two longs inside of RowVersion, which is nice. Third - no null-checks and stuff.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -0,0 +1,515 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+
+import java.nio.ByteBuffer;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.PageMemoryDataRegion;
+import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.NoUncommittedVersionException;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.StorageUtils;
+import org.apache.ignite.internal.storage.TxIdMismatchException;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryDataRegion;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.IgniteCursor;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link MvPartitionStorage} using Page Memory.
+ *
+ * @see MvPartitionStorage
+ */
+public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
+    private static final byte[] TOMBSTONE_PAYLOAD = new byte[0];
+    private static final Predicate<BinaryRow> MATCH_ALL = row -> true;
+
+    private final int partitionId;
+    private final int groupId;
+
+    private final VersionChainFreeList versionChainFreeList;
+    private final VersionChainTree versionChainTree;
+    private final DataPageReader<VersionChain> versionChainDataPageReader;
+    private final RowVersionFreeList rowVersionFreeList;
+    private final DataPageReader<RowVersion> rowVersionDataPageReader;
+
+    /**
+     * Constructor.
+     */
+    public PageMemoryMvPartitionStorage(int partitionId, TableView tableConfig, PageMemoryDataRegion dataRegion) {
+        this.partitionId = partitionId;
+
+        groupId = StorageUtils.groupId(tableConfig);
+
+        versionChainFreeList = ((VolatilePageMemoryDataRegion) dataRegion).versionChainFreeList();
+        rowVersionFreeList = ((VolatilePageMemoryDataRegion) dataRegion).rowVersionFreeList();
+
+        try {
+            versionChainTree = createVersionChainTree(partitionId, tableConfig, dataRegion, versionChainFreeList);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Error occurred while creating the partition storage", e);
+        }
+
+        versionChainDataPageReader = new VersionChainDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
+        rowVersionDataPageReader = new RowVersionDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
+    }
+
+    private VersionChainTree createVersionChainTree(
+            int partitionId,
+            TableView tableConfig,
+            PageMemoryDataRegion dataRegion,
+            VersionChainFreeList versionChainFreeList1
+    ) throws IgniteInternalCheckedException {
+        // TODO: IGNITE-16641 It is necessary to do getting the tree root for the persistent case.
+        long metaPageId = dataRegion.requiredPageMemory().allocatePage(groupId, partitionId, FLAG_AUX);
+
+        // TODO: IGNITE-16641 It is necessary to take into account the persistent case.
+        boolean initNew = true;
+
+        return new VersionChainTree(
+                groupId,
+                tableConfig.name(),
+                dataRegion.pageMemory(),
+                PageLockListenerNoOp.INSTANCE,
+                new AtomicLong(),
+                metaPageId,
+                versionChainFreeList1,
+                partitionId,
+                initNew
+        );
+    }
+
+    @Override
+    public @Nullable BinaryRow read(RowId rowId, UUID txId) throws TxIdMismatchException, StorageException {
+        VersionChain versionChain = findVersionChain(rowId);
+        if (versionChain == null) {
+            return null;
+        }
+
+        return findLatestRowVersion(versionChain, txId, MATCH_ALL);
+    }
+
+    @Override
+    public @Nullable BinaryRow read(RowId rowId, Timestamp timestamp) throws StorageException {
+        VersionChain versionChain = findVersionChain(rowId);
+        if (versionChain == null) {
+            return null;
+        }
+
+        return findRowVersionByTimestamp(versionChain, timestamp);
+    }
+
+    @Nullable
+    private VersionChain findVersionChain(RowId rowId) {
+        try {
+            return versionChainDataPageReader.getRowByLink(versionChainLinkFrom(rowId));
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Version chain lookup failed", e);
+        }
+    }
+
+    private long versionChainLinkFrom(RowId rowId) {
+        if (rowId.partitionId() != partitionId) {
+            throw new IllegalArgumentException("I own partition " + partitionId + " but I was given RowId with partition "
+                    + rowId.partitionId());
+        }
+
+        LinkRowId linkRowId = (LinkRowId) rowId;
+
+        return linkRowId.versionChainLink();
+    }
+
+    @Nullable
+    private ByteBufferRow findLatestRowVersion(VersionChain versionChain, UUID txId, Predicate<BinaryRow> keyFilter) {
+        RowVersion rowVersion = findLatestRowVersion(versionChain);
+        ByteBufferRow row = rowVersionToBinaryRow(rowVersion);
+
+        if (!keyFilter.test(row)) {
+            return null;
+        }
+
+        throwIfChainBelongsToAnotherTx(versionChain, txId);
+
+        return row;
+    }
+
+    private RowVersion findLatestRowVersion(VersionChain versionChain) {
+        return findRowVersion(versionChain.headLink());
+    }
+
+    private RowVersion findRowVersion(long nextRowPartitionlessLink) {
+        long nextLink = PartitionlessLinks.addPartitionIdToPartititionlessLink(nextRowPartitionlessLink, partitionId);
+
+        try {
+            return rowVersionDataPageReader.getRowByLink(nextLink);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Row version lookup failed");
+        }
+    }
+
+    private void throwIfChainBelongsToAnotherTx(VersionChain versionChain, UUID txId) {
+        if (versionChain.transactionId() != null && !txId.equals(versionChain.transactionId())) {
+            throw new TxIdMismatchException();
+        }
+    }
+
+    @Nullable
+    private ByteBufferRow rowVersionToBinaryRow(RowVersion rowVersion) {
+        if (rowVersion.isTombstone()) {
+            return null;
+        }
+
+        return new ByteBufferRow(rowVersion.value());
+    }
+
+    @Nullable
+    private ByteBufferRow findRowVersionInChain(
+            VersionChain versionChain,
+            @Nullable UUID transactionId,
+            @Nullable Timestamp timestamp,
+            Predicate<BinaryRow> keyFilter
+    ) {
+        assert transactionId != null ^ timestamp != null;
+
+        if (transactionId != null) {
+            return findLatestRowVersion(versionChain, transactionId, keyFilter);
+        } else {
+            ByteBufferRow row = findRowVersionByTimestamp(versionChain, timestamp);
+            return keyFilter.test(row) ? row : null;
+        }
+    }
+
+    @Nullable
+    private ByteBufferRow findRowVersionByTimestamp(VersionChain versionChain, Timestamp timestamp) {
+        long nextRowPartitionlessLink = versionChain.headLink();
+
+        while (true) {
+            RowVersion rowVersion = findRowVersion(nextRowPartitionlessLink);
+            Timestamp rowVersionTs = rowVersion.timestamp();
+            if (rowVersionTs != null && rowVersionTs.beforeOrEquals(timestamp)) {
+                return rowVersionToBinaryRow(rowVersion);
+            }
+
+            if (!rowVersion.hasNextLink()) {
+                return null;
+            }
+
+            nextRowPartitionlessLink = rowVersion.nextLink();
+        }
+    }
+
+    @Override
+    public LinkRowId insert(BinaryRow row, UUID txId) throws StorageException {
+        RowVersion rowVersion = insertRowVersion(Objects.requireNonNull(row), RowVersion.NULL_LINK);
+
+        VersionChain versionChain = new VersionChain(partitionId, txId, PartitionlessLinks.removePartitionIdFromLink(rowVersion.link()));
+
+        try {
+            versionChainFreeList.insertDataRow(versionChain);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot store a version chain", e);
+        }
+
+        try {
+            versionChainTree.putx(versionChain);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot put a version chain to the tree", e);
+        }
+
+        return new LinkRowId(versionChain.link());
+    }
+
+    private RowVersion insertRowVersion(@Nullable BinaryRow row, long nextPartitionlessLink) {
+        // TODO IGNITE-16913 Add proper way to write row bytes into array without allocations.
+        byte[] rowBytes = row == null ? TOMBSTONE_PAYLOAD : row.bytes();
+
+        RowVersion rowVersion = new RowVersion(partitionId, nextPartitionlessLink, ByteBuffer.wrap(rowBytes));
+
+        insertRowVersion(rowVersion);
+
+        return rowVersion;
+    }
+
+    private void insertRowVersion(RowVersion rowVersion) {
+        try {
+            rowVersionFreeList.insertDataRow(rowVersion);
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot store a row version", e);
+        }
+    }
+
+    @Override
+    public @Nullable BinaryRow addWrite(RowId rowId, @Nullable BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException {

Review Comment:
   As I already mentioned, this method is not effective



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/RowVersion.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.StorableBase;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.storage.pagememory.mv.io.RowVersionDataIo;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents row version inside row version chain.
+ */
+public class RowVersion extends StorableBase {
+    /**
+     * A 'timestamp' representing absense of a timestamp.
+     */
+    public static final Timestamp NULL_TIMESTAMP = new Timestamp(Long.MIN_VALUE, Long.MIN_VALUE);
+    /**
+     * Represents an absent partitionless link.
+     */
+    public static final long NULL_LINK = 0;
+
+    @Nullable
+    private final Timestamp timestamp;
+    private final long nextLink;
+    private final int valueSize;
+    private final ByteBuffer value;
+
+    /**
+     * Constructor.
+     */
+    public RowVersion(int partitionId, long nextLink, ByteBuffer value) {
+        this(partitionId, 0, null, nextLink, value);
+    }
+
+    /**
+     * Constructor.
+     */
+    public RowVersion(int partitionId, long link, @Nullable Timestamp timestamp, long nextLink, ByteBuffer value) {
+        super(partitionId);
+        link(link);
+
+        assert !NULL_TIMESTAMP.equals(timestamp) : "Null timestamp provided";
+
+        this.timestamp = timestamp;
+        this.nextLink = nextLink;
+        this.valueSize = value.limit();
+        this.value = value;
+    }
+
+    @Nullable
+    public Timestamp timestamp() {
+        return timestamp;
+    }
+
+    public Timestamp timestampForStorage() {
+        return timestamp == null ? NULL_TIMESTAMP : timestamp;
+    }
+
+    /**
+     * Returns partitionless link of the next version or {@code 0} if this version is the last in the chain (i.e. it's the oldest version).
+     *
+     * @return partitionless link of the next version or {@code 0} if this version is the last in the chain
+     */
+    public long nextLink() {
+        return nextLink;
+    }
+
+    public int valueSize() {
+        return valueSize;
+    }
+
+    public ByteBuffer value() {
+        return value;
+    }
+
+    public boolean hasNextLink() {
+        return nextLink != NULL_LINK;
+    }
+
+    boolean isTombstone() {
+        return valueSize() == 0;
+    }
+
+    boolean isUncommitted() {
+        return timestamp == null;
+    }
+
+    boolean isCommitted() {
+        return timestamp != null;
+    }
+
+    @Override
+    public int size() {
+        return timestampStoreSize() + nextLinkStoreSize() + valueStoreSize();
+    }
+
+    private int timestampStoreSize() {
+        return 2 * Long.BYTES;
+    }
+
+    private int nextLinkStoreSize() {
+        return PartitionlessLinks.PARTITIONLESS_LINK_SIZE_BYTES;
+    }
+
+    private int valueStoreSize() {
+        return Integer.BYTES + value.limit();
+    }
+
+    @Override
+    public int headerSize() {
+        return 0;

Review Comment:
   I specifically mentioned header sizes in JIRA. Please put "size()" here. This is a bug right now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org