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/12/03 08:45:49 UTC

[GitHub] [ignite-3] rpuch opened a new pull request, #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

rpuch opened a new pull request, #1405:
URL: https://github.com/apache/ignite-3/pull/1405

   https://issues.apache.org/jira/browse/IGNITE-18118


-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1044187146


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -215,7 +242,25 @@ public RaftGroupConfiguration committedGroupConfiguration() {
         }
 
         try {
-            return groupConfigFromBytes(meta.lastGroupConfig());
+            groupConfigReadWriteLock.readLock().lock();

Review Comment:
   We already have busy lock taken, in line 240. Isn't this enough?



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048111722


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/freelist/io/IndexColumnsDataIo.java:
##########
@@ -48,7 +48,7 @@ protected IndexColumnsDataIo(int ver) {
     protected void writeRowData(long pageAddr, int dataOff, int payloadSize, IndexColumns row, boolean newRow) {
         assertPageType(pageAddr);
 
-        putShort(pageAddr, dataOff, (short) payloadSize);
+        putShort(pageAddr, dataOff, narrowIntToShort(payloadSize));

Review Comment:
   I already reverted the change



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1049308112


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/BlobFragmentIo.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.io;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putLong;
+
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.util.PageUtils;
+import org.apache.ignite.lang.IgniteStringBuilder;
+
+/**
+ * Pages IO for blob fragments.
+ *
+ * <p>First, ID of the next page in the chain is stored (0 if current page is the last one in the chain).
+ * Then, if the page is the first in the chain, total blob length is stored as 4 bytes.
+ * Finally, bytes of a blob fragment are stored.
+ */
+public class BlobFragmentIo extends PageIo {
+    /** Page IO type. */
+    public static final short T_BLOB_FRAGMENT_IO = 13;
+
+    private static final int NEXT_PAGE_ID_OFF = PageIo.COMMON_HEADER_END;
+
+    private static final int FRAGMENT_BYTES_OFF = NEXT_PAGE_ID_OFF + Long.BYTES;

Review Comment:
   Storing page IDs as 6 bytes will make the code more complex, but the gain is too little (2 bytes per page), so it does not seem worth the complexity.
   
   I made the way we calculate the offset more clear and clarified the page layout even more in the javadoc.



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048112793


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   In general, I agree that commas are not needed, you can change the general style in a technical debt ticket



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048119606


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   I added a ticket https://issues.apache.org/jira/browse/IGNITE-18407
   
   Here, it seems better to leave it as is (just newlines, no commas) as we know consider 'commas+newlines' a tach debt.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   I added a ticket https://issues.apache.org/jira/browse/IGNITE-18407
   
   Here, it seems better to leave it as is (just newlines, no commas) as we know consider 'commas+newlines' a tech debt.



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047123609


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   Removed the additional check. But this should not be forgotten later.



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047465619


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -36,15 +35,19 @@ public class PartitionMetaIo extends PageIo {
 
     private static final int LAST_APPLIED_TERM_OFF = LAST_APPLIED_INDEX_OFF + Long.BYTES;
 
-    private static final int ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;
+    private static final int LAST_RAFT_GROUP_CONFIG_LINK_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;
+
+    private static final int ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF = LAST_RAFT_GROUP_CONFIG_LINK_OFF + Long.BYTES;
 
     private static final int INDEX_COLUMNS_FREE_LIST_ROOT_PAGE_ID_OFF = ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF + Long.BYTES;
 
     private static final int VERSION_CHAIN_TREE_ROOT_PAGE_ID_OFF = INDEX_COLUMNS_FREE_LIST_ROOT_PAGE_ID_OFF + Long.BYTES;
 
     public static final int INDEX_TREE_META_PAGE_ID_OFF = VERSION_CHAIN_TREE_ROOT_PAGE_ID_OFF + Long.BYTES;
 
-    private static final int PAGE_COUNT_OFF = INDEX_TREE_META_PAGE_ID_OFF + Long.BYTES;
+    private static final int BLOB_FREE_LIST_ROOT_PAGE_ID_OFF = INDEX_TREE_META_PAGE_ID_OFF + Long.BYTES;

Review Comment:
   I think it should be removed.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -36,15 +35,19 @@ public class PartitionMetaIo extends PageIo {
 
     private static final int LAST_APPLIED_TERM_OFF = LAST_APPLIED_INDEX_OFF + Long.BYTES;
 
-    private static final int ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;
+    private static final int LAST_RAFT_GROUP_CONFIG_LINK_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;

Review Comment:
   ```suggestion
       private static final int LAST_RAFT_GROUP_CONFIG_FIRST_PAGE_ID_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {

Review Comment:
   What about the fact that we always free old pages and create new ones at the beginning? seems like a simpler option.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java:
##########
@@ -152,7 +152,7 @@ protected static TestValue value(BinaryRow binaryRow) {
         return new IgniteBiTuple<>(key(binaryRow), value(binaryRow));
     }
 
-    protected static List<IgniteBiTuple<TestKey, TestValue>> toList(Cursor<ReadResult> cursor) throws Exception {
+    protected static List<IgniteBiTuple<TestKey, TestValue>> drainToList(Cursor<ReadResult> cursor) {

Review Comment:
   I think that in tests where it is used, you can remove the Exception.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId, true);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMemory,
+                    groupId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId, boolean firstPage) throws IgniteInternalCheckedException {
+        long pageId;
+
+        if (maybePageId == NO_PAGE_ID) {
+            pageId = allocatePage();
+
+            PageHandler.initPage(pageMemory, groupId, pageId, latestBlobIo(firstPage), PageLockListenerNoOp.INSTANCE, statisticsHolder);
+        } else {
+            pageId = maybePageId;
+        }
+
+        return pageId;
+    }
+
+    private long allocatePage() throws IgniteInternalCheckedException {
+        long pageId = reuseList.takeRecycledPage();
+
+        if (pageId != 0) {
+            pageId = reuseList.initRecycledPage(pageId, PageIdAllocator.FLAG_AUX, null);
+        }
+
+        if (pageId == 0) {
+            pageId = pageMemory.allocatePage(groupId, partitionId, PageIdAllocator.FLAG_AUX);
+        }
+
+        return pageId;
+    }
+
+    private static BlobIo latestBlobIo(boolean firstPage) {
+        return firstPage ? BlobFirstIo.VERSIONS.latest() : BlobDataIo.VERSIONS.latest();
+    }
+
+    private void freePagesStartingWith(long pageId) throws IgniteInternalCheckedException {
+        if (pageId != NO_PAGE_ID) {
+            reuseList.addForRecycle(recycleAndCollectPagesStartingWith(pageId));
+        }
+    }
+
+    private ReuseBag recycleAndCollectPagesStartingWith(long startingPageId) throws IgniteInternalCheckedException {
+        ReuseBag reuseBag = new LongListReuseBag();
+
+        long pageId = startingPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Long nextPageId = PageHandler.writePage(pageMemory, groupId, pageId, PageLockListenerNoOp.INSTANCE,
+                    recycleAndAddToReuseBag, null, reuseBag, 0, pageId, IoStatisticsHolderNoOp.INSTANCE);
+
+            assert nextPageId != pageId : pageId;
+
+            pageId = nextPageId;
+        }
+
+        return reuseBag;
+    }
+
+    /**
+     * State of a read operation.
+     */
+    private static class ReadState {
+        private byte @Nullable [] bytes;
+
+        private int bytesOffset;
+
+        private long nextPageId = NO_PAGE_ID;
+    }
+
+    /**
+     * Reads a fragment stored in a page.
+     */
+    private static class ReadFragment implements PageHandler<ReadState, Boolean> {
+        @Override
+        public Boolean run(int groupId, long pageId, long page, long pageAddr, PageIo io, ReadState state, int unused,
+                IoStatisticsHolder statHolder) throws IgniteInternalCheckedException {
+            BlobIo blobIo = (BlobIo) io;
+
+            if (state.bytes == null) {
+                assert state.bytesOffset == 0;
+
+                state.bytes = new byte[blobIo.getTotalLength(pageAddr)];
+            }
+
+            int fragmentLength = blobIo.getFragmentLength(pageAddr);
+            blobIo.getFragmentBytes(pageAddr, state.bytes, state.bytesOffset, fragmentLength);
+
+            int newBytesOffset = state.bytesOffset + fragmentLength;
+
+            if (newBytesOffset < state.bytes.length) {
+                long nextPageId = blobIo.getNextPageId(pageAddr);
+
+                assert nextPageId != NO_PAGE_ID;
+
+                state.nextPageId = nextPageId;
+            } else {
+                state.nextPageId = NO_PAGE_ID;

Review Comment:
   I think it's worth making sure that there is no nextPageId in BlobIo.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   ```suggestion
                   .app(", lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
                   .app(", lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/MvPageIoModule.java:
##########
@@ -38,7 +40,9 @@ public Collection<IoVersions<?>> ioVersions() {
                 VersionChainMetaIo.VERSIONS,
                 VersionChainInnerIo.VERSIONS,
                 VersionChainLeafIo.VERSIONS,
-                RowVersionDataIo.VERSIONS
+                RowVersionDataIo.VERSIONS,
+                BlobFirstIo.VERSIONS,

Review Comment:
   I think that we could get by with one page, for example `BlobFragmentIo`, which will contain: the length of the fragment, the fragment itself (byte[]) and the pageId of the next fragment. 
   
   I don't think we'll be reading and writing the config often, so we can copy arrays when we read.
   
   As an optimization, we can write the length of the full blob length in the first fragment, since the current fragment will be completely filled.



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048067898


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   Why do we need commas if we already add newlines?



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048128030


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -226,22 +262,37 @@ public void committedGroupConfiguration(RaftGroupConfiguration config) {
         assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
 
         CheckpointProgress lastCheckpoint = checkpointManager.lastCheckpointProgress();
-
         UUID lastCheckpointId = lastCheckpoint == null ? null : lastCheckpoint.id();
 
-        meta.lastGroupConfig(lastCheckpointId, groupConfigToBytes(config));
+        byte[] raftGroupConfigBytes = raftGroupConfigToBytes(config);
+
+        raftGroupConfigReadWriteLock.writeLock().lock();
+
+        try {
+            if (meta.lastRaftGroupConfigFirstPageId() == BlobStorage.NO_PAGE_ID) {
+                long configPageId = blobStorage.addBlob(raftGroupConfigBytes);
+
+                meta.lastRaftGroupConfigFirstPageId(lastCheckpointId, configPageId);
+            } else {
+                blobStorage.updateBlob(meta.lastRaftGroupConfigFirstPageId(), raftGroupConfigBytes);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot save committed group configuration, groupId=" + groupId + ", partitionId=" + groupId, e);
+        } finally {
+            raftGroupConfigReadWriteLock.writeLock().unlock();
+        }
     }
 
     @Nullable
-    private static RaftGroupConfiguration groupConfigFromBytes(byte @Nullable [] bytes) {
+    private static RaftGroupConfiguration raftGroupConfigFromBytes(byte @Nullable [] bytes) {

Review Comment:
   Also added https://issues.apache.org/jira/browse/IGNITE-18408 about storing bytes, not config object



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1044189846


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java:
##########
@@ -152,7 +152,7 @@ protected static TestValue value(BinaryRow binaryRow) {
         return new IgniteBiTuple<>(key(binaryRow), value(binaryRow));
     }
 
-    protected static List<IgniteBiTuple<TestKey, TestValue>> toList(Cursor<ReadResult> cursor) throws Exception {
+    protected static List<IgniteBiTuple<TestKey, TestValue>> drainToList(Cursor<ReadResult> cursor) throws Exception {

Review Comment:
   Removed the `throws` declaration.
   
   I renamed the method so that we can statically import `Collectors.toList`.
   
   There is `BlockingQueue.drainTo()` that drains the queue to a collection. `drainToList` follows the same logic.



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048111252


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {

Review Comment:
   We'll update RAFT config on each configuration change, this should happen rarely. But it won't hurt to make the mechanism a bit more generic (it doesn't cost too much in terms of complexity).



-- 
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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047121590


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {

Review Comment:
   This is interesting after all, why didn't you extend the DataStructure class? It has a bunch of conveniences



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {

Review Comment:
   I agree that we should reuse old pages, it's simpler actually. And I believe that for the most part, the amount of pages will stay the same.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -226,22 +262,37 @@ public void committedGroupConfiguration(RaftGroupConfiguration config) {
         assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
 
         CheckpointProgress lastCheckpoint = checkpointManager.lastCheckpointProgress();
-
         UUID lastCheckpointId = lastCheckpoint == null ? null : lastCheckpoint.id();
 
-        meta.lastGroupConfig(lastCheckpointId, groupConfigToBytes(config));
+        byte[] raftGroupConfigBytes = raftGroupConfigToBytes(config);
+
+        raftGroupConfigReadWriteLock.writeLock().lock();
+
+        try {
+            if (meta.lastRaftGroupConfigFirstPageId() == BlobStorage.NO_PAGE_ID) {
+                long configPageId = blobStorage.addBlob(raftGroupConfigBytes);
+
+                meta.lastRaftGroupConfigFirstPageId(lastCheckpointId, configPageId);
+            } else {
+                blobStorage.updateBlob(meta.lastRaftGroupConfigFirstPageId(), raftGroupConfigBytes);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot save committed group configuration, groupId=" + groupId + ", partitionId=" + groupId, e);
+        } finally {
+            raftGroupConfigReadWriteLock.writeLock().unlock();
+        }
     }
 
     @Nullable
-    private static RaftGroupConfiguration groupConfigFromBytes(byte @Nullable [] bytes) {
+    private static RaftGroupConfiguration raftGroupConfigFromBytes(byte @Nullable [] bytes) {

Review Comment:
   By the way, why do we have to mention RAFT here? There's a chance of us having more replication protocol in the future, can we try and abstract ourselves a little bit? At least stop renaming old methods into a new that mention RAFT even more.
   
   It would be a good idea to rename RaftGroupConfiguration as well. Or even better. I don't understand why it's a storages' task to convert an object into a byte array. Looks like a wrong design decision. 



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/MvPageIoModule.java:
##########
@@ -38,7 +40,9 @@ public Collection<IoVersions<?>> ioVersions() {
                 VersionChainMetaIo.VERSIONS,
                 VersionChainInnerIo.VERSIONS,
                 VersionChainLeafIo.VERSIONS,
-                RowVersionDataIo.VERSIONS
+                RowVersionDataIo.VERSIONS,
+                BlobFirstIo.VERSIONS,

Review Comment:
   I can agree on this one, a single page type could be enough. We could store the total length before the payload, for example, that's not a big deal and the code complexity won't change too much



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/freelist/io/IndexColumnsDataIo.java:
##########
@@ -48,7 +48,7 @@ protected IndexColumnsDataIo(int ver) {
     protected void writeRowData(long pageAddr, int dataOff, int payloadSize, IndexColumns row, boolean newRow) {
         assertPageType(pageAddr);
 
-        putShort(pageAddr, dataOff, (short) payloadSize);
+        putShort(pageAddr, dataOff, narrowIntToShort(payloadSize));

Review Comment:
   Ok, I also think that this method is integrated too early. Here we should _assert_ that payloadSize fits the page, not even the short range.
   And, I believe, such assertion must already exist somewhere before calling the `writeRowData`



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   There's a `java.lang.Math#toIntExact` in Java. Maybe we should borrow the naming convention and the exception type? Just saying



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1049315988


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMem,
+                    grpId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId) throws IgniteInternalCheckedException {

Review Comment:
   Inlined the method



-- 
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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047108906


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   Current limit is 32k I believe
   EDIT: it's 16k actually



-- 
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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047108906


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   Current limit is 32k I believe



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048079424


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {

Review Comment:
   This looks like a suggestion and not a fact :)
   
   I wanted to avoid unneeded reallocation.



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048088097


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {

Review Comment:
   How often will we update the configuration?



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1044177958


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   If I'm not mistaken, the check in `AbstractDataPageIo#addRow()` is made with `assert` keyword, so it might absent in a production environment, hence it cannot be relied upon. I'd prefer to either leave the check that I added or change the check in `addRow()` from `assert` to throwing an exception.
   
   WDYT?



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1044566239


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   I think the assertion should be enough and check all this in tests. 
   In production code, this check is not needed.



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1049283859


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMem,
+                    grpId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId) throws IgniteInternalCheckedException {

Review Comment:
   Ok, `allocatePage()` I do not think that it is needed, it is used in one place.



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048104512


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   Is this general style justified by something? If I'm not mistaken, then using this 'commas and NLs' style would produce output like the following:
   
   ```
   field1=val1
   ,field2=val2
   ,field3=val3
   ```
   
   while 'just NLs' style will produce
   
   ```
   field1=val1
   field2=val2
   field3=val3
   ```
   
   To me, second result seems more readable (there is less noise caused by commas, and this noise is not needed).
   
   Why would we prefer first result to the second one?



-- 
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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048262539


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {

Review Comment:
   Methods for allocating pages, that you pretty much copy-pasted unwillingly. I think they could be reused as well



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048086838


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -251,6 +253,7 @@ protected void printPage(long addr, int pageSize, IgniteStringBuilder sb) {
         sb.app("TablePartitionMeta [").nl()
                 .app("lastAppliedIndex=").app(getLastAppliedIndex(addr)).nl()
                 .app("lastAppliedTerm=").app(getLastAppliedTerm(addr)).nl()
+                .app("lastRaftGroupConfigFirstPageId=").app(getLastRaftGroupConfigFirstPageId(addr)).nl()

Review Comment:
   Just a general style for pages
   org.apache.ignite.internal.pagememory.tree.io.BplusIo#printPage
   org.apache.ignite.internal.pagememory.tree.io.BplusMetaIo#printPage
   etc



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048113376


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {

Review Comment:
   The main reason is that I didn't know that I would benefit from extending `DataStructure`. I reused page recycling from there, what other conveniences would be useful here?



-- 
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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048263068


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -226,22 +262,37 @@ public void committedGroupConfiguration(RaftGroupConfiguration config) {
         assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
 
         CheckpointProgress lastCheckpoint = checkpointManager.lastCheckpointProgress();
-
         UUID lastCheckpointId = lastCheckpoint == null ? null : lastCheckpoint.id();
 
-        meta.lastGroupConfig(lastCheckpointId, groupConfigToBytes(config));
+        byte[] raftGroupConfigBytes = raftGroupConfigToBytes(config);
+
+        raftGroupConfigReadWriteLock.writeLock().lock();
+
+        try {
+            if (meta.lastRaftGroupConfigFirstPageId() == BlobStorage.NO_PAGE_ID) {
+                long configPageId = blobStorage.addBlob(raftGroupConfigBytes);
+
+                meta.lastRaftGroupConfigFirstPageId(lastCheckpointId, configPageId);
+            } else {
+                blobStorage.updateBlob(meta.lastRaftGroupConfigFirstPageId(), raftGroupConfigBytes);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot save committed group configuration, groupId=" + groupId + ", partitionId=" + groupId, e);
+        } finally {
+            raftGroupConfigReadWriteLock.writeLock().unlock();
+        }
     }
 
     @Nullable
-    private static RaftGroupConfiguration groupConfigFromBytes(byte @Nullable [] bytes) {
+    private static RaftGroupConfiguration raftGroupConfigFromBytes(byte @Nullable [] bytes) {

Review Comment:
   Thank you!



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048688041


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMem,
+                    grpId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId) throws IgniteInternalCheckedException {
+        long pageId;
+
+        if (maybePageId == NO_PAGE_ID) {
+            pageId = allocatePage();
+
+            init(pageId, BlobFragmentIo.VERSIONS.latest());
+        } else {
+            pageId = maybePageId;
+        }
+
+        return pageId;
+    }
+
+    private long allocatePage() throws IgniteInternalCheckedException {
+        return allocatePage(null);
+    }
+
+    private void freePagesStartingWith(long pageId) throws IgniteInternalCheckedException {
+        if (pageId != NO_PAGE_ID) {
+            reuseList.addForRecycle(recycleAndCollectPagesStartingWith(pageId));
+        }
+    }
+
+    private ReuseBag recycleAndCollectPagesStartingWith(long startingPageId) throws IgniteInternalCheckedException {
+        ReuseBag reuseBag = new LongListReuseBag();
+
+        long pageId = startingPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Long nextPageId = PageHandler.writePage(pageMem, grpId, pageId, PageLockListenerNoOp.INSTANCE,
+                    recycleAndAddToReuseBag, null, reuseBag, 0, pageId, IoStatisticsHolderNoOp.INSTANCE);
+
+            assert nextPageId != pageId : pageId;
+
+            pageId = nextPageId;
+        }
+
+        return reuseBag;
+    }
+
+    /**
+     * State of a read operation.
+     */
+    private static class ReadState {
+        private byte @Nullable [] bytes;
+
+        private int bytesOffset;
+
+        private long nextPageId = NO_PAGE_ID;
+
+        private int totalLength;

Review Comment:
   Can we get this information from the length of the array?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/BlobFragmentIo.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.io;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putLong;
+
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.util.PageUtils;
+import org.apache.ignite.lang.IgniteStringBuilder;
+
+/**
+ * Pages IO for blob fragments.
+ *
+ * <p>First, ID of the next page in the chain is stored (0 if current page is the last one in the chain).
+ * Then, if the page is the first in the chain, total blob length is stored as 4 bytes.
+ * Finally, bytes of a blob fragment are stored.
+ */
+public class BlobFragmentIo extends PageIo {
+    /** Page IO type. */
+    public static final short T_BLOB_FRAGMENT_IO = 13;
+
+    private static final int NEXT_PAGE_ID_OFF = PageIo.COMMON_HEADER_END;
+
+    private static final int FRAGMENT_BYTES_OFF = NEXT_PAGE_ID_OFF + Long.BYTES;

Review Comment:
   I think it's better to use `org.apache.ignite.internal.pagememory.util.PartitionlessLinks`. 
   I mean not 8 bytes but 6.
   
   It confuses me a little that this is a constant, let's make it calculated, and in the documentation we will present the layout of the page, for example, as in `org.apache.ignite.internal.storage.pagememory.index.sorted.io.SortedIndexTreeIo`.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMem,
+                    grpId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId) throws IgniteInternalCheckedException {
+        long pageId;
+
+        if (maybePageId == NO_PAGE_ID) {
+            pageId = allocatePage();
+
+            init(pageId, BlobFragmentIo.VERSIONS.latest());
+        } else {
+            pageId = maybePageId;
+        }
+
+        return pageId;
+    }
+
+    private long allocatePage() throws IgniteInternalCheckedException {
+        return allocatePage(null);
+    }
+
+    private void freePagesStartingWith(long pageId) throws IgniteInternalCheckedException {
+        if (pageId != NO_PAGE_ID) {
+            reuseList.addForRecycle(recycleAndCollectPagesStartingWith(pageId));
+        }
+    }
+
+    private ReuseBag recycleAndCollectPagesStartingWith(long startingPageId) throws IgniteInternalCheckedException {
+        ReuseBag reuseBag = new LongListReuseBag();
+
+        long pageId = startingPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Long nextPageId = PageHandler.writePage(pageMem, grpId, pageId, PageLockListenerNoOp.INSTANCE,
+                    recycleAndAddToReuseBag, null, reuseBag, 0, pageId, IoStatisticsHolderNoOp.INSTANCE);
+
+            assert nextPageId != pageId : pageId;
+
+            pageId = nextPageId;
+        }
+
+        return reuseBag;
+    }
+
+    /**
+     * State of a read operation.
+     */
+    private static class ReadState {
+        private byte @Nullable [] bytes;
+
+        private int bytesOffset;
+
+        private long nextPageId = NO_PAGE_ID;
+
+        private int totalLength;
+
+        private boolean isFirstPage() {
+            return bytesOffset == 0;
+        }
+    }
+
+    /**
+     * Reads a fragment stored in a page.
+     */
+    private class ReadFragment implements PageHandler<ReadState, Boolean> {
+        @Override
+        public Boolean run(int groupId, long pageId, long page, long pageAddr, PageIo io, ReadState state, int unused,
+                IoStatisticsHolder statHolder) throws IgniteInternalCheckedException {
+            BlobFragmentIo blobIo = (BlobFragmentIo) io;
+
+            if (state.bytes == null) {
+                assert state.isFirstPage();
+
+                state.bytes = new byte[blobIo.getTotalLength(pageAddr)];
+                state.totalLength = blobIo.getTotalLength(pageAddr);
+            }
+
+            int capacityForBytes = blobIo.getCapacityForFragmentBytes(pageSize(), state.isFirstPage());
+            int fragmentLength = Math.min(capacityForBytes, state.totalLength - state.bytesOffset);
+
+            blobIo.getFragmentBytes(pageAddr, state.isFirstPage(), state.bytes, state.bytesOffset, fragmentLength);
+
+            long nextPageId = blobIo.getNextPageId(pageAddr);
+
+            int newBytesOffset = state.bytesOffset + fragmentLength;
+
+            if (newBytesOffset < state.bytes.length) {
+                assert nextPageId != NO_PAGE_ID;
+
+                state.nextPageId = nextPageId;
+            } else {
+                assert nextPageId == NO_PAGE_ID;
+
+                state.nextPageId = NO_PAGE_ID;
+            }
+
+            state.bytesOffset = newBytesOffset;
+
+            return true;
+        }
+    }
+
+    /**
+     * State of a write operation.
+     */
+    private static class WriteState {
+        private final byte[] bytes;
+        private int bytesOffset;
+
+        private long pageId;
+
+        private boolean stop;
+        private long firstPageToFreeId = NO_PAGE_ID;
+
+        private WriteState(byte[] bytes) {
+            this.bytes = bytes;
+        }
+
+        private boolean isFirstPage() {
+            return bytesOffset == 0;
+        }
+    }
+
+    /**
+     * Writes a fragment to a page.
+     */
+    private class WriteFragment implements PageHandler<WriteState, Boolean> {
+        @Override
+        public Boolean run(int groupId, long pageId, long page, long pageAddr, PageIo io, WriteState state, int unused,
+                IoStatisticsHolder statHolder) throws IgniteInternalCheckedException {
+            BlobFragmentIo blobIo = (BlobFragmentIo) io;
+
+            int capacityForBytes = blobIo.getCapacityForFragmentBytes(pageSize(), state.isFirstPage());
+
+            int fragmentLength = Math.min(capacityForBytes, state.bytes.length - state.bytesOffset);
+
+            if (state.isFirstPage()) {
+                blobIo.setTotalLength(pageAddr, state.bytes.length);
+            }
+            blobIo.setFragmentBytes(pageAddr, state.isFirstPage(), state.bytes, state.bytesOffset, fragmentLength);

Review Comment:
   ```suggestion
               
               blobIo.setFragmentBytes(pageAddr, state.isFirstPage(), state.bytes, state.bytesOffset, fragmentLength);
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/io/BlobFragmentIo.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.io;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putLong;
+
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.util.PageUtils;
+import org.apache.ignite.lang.IgniteStringBuilder;
+
+/**
+ * Pages IO for blob fragments.

Review Comment:
   I think you should add a description something like:
   `Pages IO for storing a blob as a chain of its fragments.`



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");

Review Comment:
   Maybe not needed?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMem,
+                    grpId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId) throws IgniteInternalCheckedException {

Review Comment:
   I think you can use `org.apache.ignite.internal.pagememory.datastructure.DataStructure#allocatePage(org.apache.ignite.internal.pagememory.reuse.ReuseBag)` with `null`



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1044568539


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -215,7 +242,25 @@ public RaftGroupConfiguration committedGroupConfiguration() {
         }
 
         try {
-            return groupConfigFromBytes(meta.lastGroupConfig());
+            groupConfigReadWriteLock.readLock().lock();

Review Comment:
   my fault



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1049277307


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");
+
+        long firstPageId = allocatePageIfNeeded(maybeFirstPageId);
+
+        WriteState state = new WriteState(bytes);
+        state.pageId = firstPageId;
+
+        do {
+            Boolean ok = PageHandler.writePage(
+                    pageMem,
+                    grpId,
+                    state.pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    writeFragment,
+                    null,
+                    state,
+                    0,
+                    false,
+                    statisticsHolder
+            );
+
+            assert ok : state.pageId;
+        } while (!state.stop);
+
+        freePagesStartingWith(state.firstPageToFreeId);
+
+        return firstPageId;
+    }
+
+    private long allocatePageIfNeeded(long maybePageId) throws IgniteInternalCheckedException {

Review Comment:
   This is what I do



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1046793045


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   Can a user configure a really large page, like 1Mb? If yes, then free space in a page might be around 1Mb, which is way more than 64kb. If the user does not have `-ea` (because they do this in production), they will get strange behavior instead of a proper exception.
   
   Assertions should be used only for catching programmer errors (i.e. our errors). They should not be used to protect against a possible misconfiguration. Also, tests would not help in such scenario.



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047622862


##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java:
##########
@@ -105,7 +121,94 @@ void testReadAfterRestart() throws Exception {
         tearDown();
 
         setUp();
+    }
 
-        assertRowMatches(binaryRow, read(rowId, HybridTimestamp.MAX_VALUE));
+    @Test
+    void groupConfigIsPersisted() throws Exception {
+        RaftGroupConfiguration originalConfig = new RaftGroupConfiguration(
+                List.of("peer1", "peer2"),
+                List.of("old-peer1", "old-peer2"),
+                List.of("learner1", "learner2"),
+                List.of("old-learner1", "old-learner2")
+        );
+
+        storage.runConsistently(() -> {
+            storage.committedGroupConfiguration(originalConfig);
+
+            return null;
+        });
+
+        restartStorage();
+
+        RaftGroupConfiguration readConfig = storage.committedGroupConfiguration();
+
+        assertThat(readConfig, is(equalTo(originalConfig)));
+    }
+
+    @Test
+    void groupConfigWhichDoesNotFitInOnePageIsPersisted() throws Exception {
+        List<String> oneMbOfPeers = IntStream.range(0, 100_000)
+                .mapToObj(n -> String.format("peer%06d", n))
+                .collect(toList());
+
+        RaftGroupConfiguration originalConfig = new RaftGroupConfiguration(
+                oneMbOfPeers,
+                List.of("old-peer1", "old-peer2"),
+                List.of("learner1", "learner2"),
+                List.of("old-learner1", "old-learner2")
+        );
+
+        storage.runConsistently(() -> {
+            storage.committedGroupConfiguration(originalConfig);
+
+            return null;
+        });
+
+        restartStorage();
+
+        RaftGroupConfiguration readConfig = storage.committedGroupConfiguration();
+
+        assertThat(readConfig, is(equalTo(originalConfig)));
+    }
+
+    @Test
+    void groupConfigPagesAreRecycled() throws Exception {
+        List<String> oneMbOfPeers = IntStream.range(0, 100_000)

Review Comment:
   I think it's worth making sure the size of these `Strings` is greater than at least 2 pages.



##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java:
##########
@@ -105,7 +121,94 @@ void testReadAfterRestart() throws Exception {
         tearDown();
 
         setUp();
+    }
 
-        assertRowMatches(binaryRow, read(rowId, HybridTimestamp.MAX_VALUE));
+    @Test
+    void groupConfigIsPersisted() throws Exception {
+        RaftGroupConfiguration originalConfig = new RaftGroupConfiguration(
+                List.of("peer1", "peer2"),
+                List.of("old-peer1", "old-peer2"),
+                List.of("learner1", "learner2"),
+                List.of("old-learner1", "old-learner2")
+        );
+
+        storage.runConsistently(() -> {
+            storage.committedGroupConfiguration(originalConfig);
+
+            return null;
+        });
+
+        restartStorage();
+
+        RaftGroupConfiguration readConfig = storage.committedGroupConfiguration();
+
+        assertThat(readConfig, is(equalTo(originalConfig)));
+    }
+
+    @Test
+    void groupConfigWhichDoesNotFitInOnePageIsPersisted() throws Exception {
+        List<String> oneMbOfPeers = IntStream.range(0, 100_000)
+                .mapToObj(n -> String.format("peer%06d", n))
+                .collect(toList());
+
+        RaftGroupConfiguration originalConfig = new RaftGroupConfiguration(
+                oneMbOfPeers,
+                List.of("old-peer1", "old-peer2"),
+                List.of("learner1", "learner2"),
+                List.of("old-learner1", "old-learner2")
+        );
+
+        storage.runConsistently(() -> {
+            storage.committedGroupConfiguration(originalConfig);
+
+            return null;
+        });
+
+        restartStorage();
+
+        RaftGroupConfiguration readConfig = storage.committedGroupConfiguration();
+
+        assertThat(readConfig, is(equalTo(originalConfig)));
+    }
+
+    @Test
+    void groupConfigPagesAreRecycled() throws Exception {
+        List<String> oneMbOfPeers = IntStream.range(0, 100_000)
+                .mapToObj(n -> String.format("peer%06d", n))
+                .collect(toList());
+
+        RaftGroupConfiguration originalConfigOfMoreThanOnePage = new RaftGroupConfiguration(
+                oneMbOfPeers,
+                List.of("old-peer1", "old-peer2"),
+                List.of("learner1", "learner2"),
+                List.of("old-learner1", "old-learner2")
+        );
+
+        storage.runConsistently(() -> {
+            storage.committedGroupConfiguration(originalConfigOfMoreThanOnePage);
+
+            return null;
+        });
+
+        long freeSpaceBeforeTrimming = pageMemStorage.rowVersionFreeList.freeSpace();

Review Comment:
   unused



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048327497


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -226,22 +262,37 @@ public void committedGroupConfiguration(RaftGroupConfiguration config) {
         assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
 
         CheckpointProgress lastCheckpoint = checkpointManager.lastCheckpointProgress();
-
         UUID lastCheckpointId = lastCheckpoint == null ? null : lastCheckpoint.id();
 
-        meta.lastGroupConfig(lastCheckpointId, groupConfigToBytes(config));
+        byte[] raftGroupConfigBytes = raftGroupConfigToBytes(config);
+
+        raftGroupConfigReadWriteLock.writeLock().lock();
+
+        try {
+            if (meta.lastRaftGroupConfigFirstPageId() == BlobStorage.NO_PAGE_ID) {
+                long configPageId = blobStorage.addBlob(raftGroupConfigBytes);
+
+                meta.lastRaftGroupConfigFirstPageId(lastCheckpointId, configPageId);
+            } else {
+                blobStorage.updateBlob(meta.lastRaftGroupConfigFirstPageId(), raftGroupConfigBytes);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot save committed group configuration, groupId=" + groupId + ", partitionId=" + groupId, e);
+        } finally {
+            raftGroupConfigReadWriteLock.writeLock().unlock();
+        }
     }
 
     @Nullable
-    private static RaftGroupConfiguration groupConfigFromBytes(byte @Nullable [] bytes) {
+    private static RaftGroupConfiguration raftGroupConfigFromBytes(byte @Nullable [] bytes) {

Review Comment:
   Renamed internal fields/constants/methods (not relating to the API)



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1044566723


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java:
##########
@@ -152,7 +152,7 @@ protected static TestValue value(BinaryRow binaryRow) {
         return new IgniteBiTuple<>(key(binaryRow), value(binaryRow));
     }
 
-    protected static List<IgniteBiTuple<TestKey, TestValue>> toList(Cursor<ReadResult> cursor) throws Exception {
+    protected static List<IgniteBiTuple<TestKey, TestValue>> drainToList(Cursor<ReadResult> cursor) throws Exception {

Review Comment:
   Ok



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1042427805


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   Similar check is already in `org.apache.ignite.internal.pagememory.io.AbstractDataPageIo#addRow`, we can expand the existing one if necessary.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMeta.java:
##########
@@ -50,7 +50,7 @@ public class PartitionMeta {
 
     private volatile long lastAppliedTerm;
 
-    private volatile byte @Nullable [] lastGroupConfig;
+    private volatile long lastGroupConfigLink;

Review Comment:
   Please indicate that this is a raft group configuration.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java:
##########
@@ -152,7 +152,7 @@ protected static TestValue value(BinaryRow binaryRow) {
         return new IgniteBiTuple<>(key(binaryRow), value(binaryRow));
     }
 
-    protected static List<IgniteBiTuple<TestKey, TestValue>> toList(Cursor<ReadResult> cursor) throws Exception {
+    protected static List<IgniteBiTuple<TestKey, TestValue>> drainToList(Cursor<ReadResult> cursor) throws Exception {

Review Comment:
   Why "drain"? method does not throw `Exception`.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -438,6 +442,52 @@ private IndexMetaTree createIndexMetaTree(
         }
     }
 
+    /**
+     * Returns new {@link BlobFreeList} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param pageMemory Persistent page memory instance.
+     * @param meta Partition metadata.
+     * @throws StorageException If failed.
+     */
+    private BlobFreeList createBlobFreeList(
+            TableView tableView,
+            int partId,
+            PersistentPageMemory pageMemory,
+            PartitionMeta meta
+    ) throws StorageException {
+        try {
+            boolean initNew = false;
+
+            if (meta.blobFreeListRootPageId() == 0) {
+                long rootPageId = pageMemory.allocatePage(tableView.tableId(), partId, FLAG_AUX);
+
+                meta.blobFreeListRootPageId(lastCheckpointId(), rootPageId);
+
+                initNew = true;
+            }
+
+            return new BlobFreeList(
+                    tableView.tableId(),
+                    partId,
+                    dataRegion.pageMemory(),
+                    null,

Review Comment:
   Why is the `reuseList` empty?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -36,7 +35,9 @@ public class PartitionMetaIo extends PageIo {
 
     private static final int LAST_APPLIED_TERM_OFF = LAST_APPLIED_INDEX_OFF + Long.BYTES;
 
-    private static final int ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;
+    private static final int LAST_GROUP_CONFIG_LINK_OFF = LAST_APPLIED_TERM_OFF + Long.BYTES;
+
+    private static final int ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF = LAST_GROUP_CONFIG_LINK_OFF + Long.BYTES;

Review Comment:
   `blobFreeListRootPageId` skipped?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -215,7 +242,25 @@ public RaftGroupConfiguration committedGroupConfiguration() {
         }
 
         try {
-            return groupConfigFromBytes(meta.lastGroupConfig());
+            groupConfigReadWriteLock.readLock().lock();

Review Comment:
   Forgot
   ```
   if (!closeBusyLock.enterBusy()) {
               throwStorageClosedException();
           }
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -438,6 +442,52 @@ private IndexMetaTree createIndexMetaTree(
         }
     }
 
+    /**
+     * Returns new {@link BlobFreeList} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param pageMemory Persistent page memory instance.
+     * @param meta Partition metadata.
+     * @throws StorageException If failed.
+     */
+    private BlobFreeList createBlobFreeList(

Review Comment:
   Technically, we can use `RowVersionFreeList` without creating an additional `FreeList`.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMeta.java:
##########
@@ -327,7 +348,7 @@ private PartitionMetaSnapshot(@Nullable UUID checkpointId, PartitionMeta partiti
             this.checkpointId = checkpointId;
             lastAppliedIndex = partitionMeta.lastAppliedIndex;
             lastAppliedTerm = partitionMeta.lastAppliedTerm;
-            lastGroupConfig = partitionMeta.lastGroupConfig;
+            lastGroupConfigLink = partitionMeta.lastGroupConfigLink;

Review Comment:
   `blobFreeListRootPageId` skipped?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java:
##########
@@ -101,15 +102,15 @@ public void setLastAppliedTerm(long pageAddr, long lastAppliedTerm) {
     }
 
     /**
-     * Sets last group config.
+     * Sets link to blob representing last group config.
      *
      * @param pageAddr Page address.
-     * @param lastGroupConfig Byte representation of last group config..
+     * @param lastGroupConfigLink Link to blob representing last group config..
      */
-    public void setLastGroupConfig(long pageAddr, byte @Nullable [] lastGroupConfig) {
+    public void setLastGroupConfig(long pageAddr, long lastGroupConfigLink) {

Review Comment:
   Maybe `setLastGroupConfigLink` ?
   Please add into toString



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -62,6 +68,15 @@ public class PersistentPageMemoryMvPartitionStorage extends AbstractPageMemoryMv
     /** Checkpoint listener. */
     private final CheckpointListener checkpointListener;
 
+    /** FreeList for blobs. */
+    private final BlobFreeList blobFreeList;
+
+    /** Used to read data from pagememory pages. */
+    private final DataPageReader pageReader;
+
+    /** Lock that protects group config read/write. */

Review Comment:
   Please indicate that this is a raft group configuration.



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1049274291


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFragmentIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage extends DataStructure {
+    static final long NO_PAGE_ID = 0;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        super("BlobStorage", groupId, null, partitionId, pageMemory, PageLockListenerNoOp.INSTANCE, PageIdAllocator.FLAG_AUX);
+
+        super.reuseList = reuseList;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMem,
+                    grpId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        Objects.requireNonNull(bytes, "bytes is null");

Review Comment:
   I'd like to check this precondition. A meaningful error message is a good thing.



-- 
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


[GitHub] [ignite-3] tkalkirill merged pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill merged PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405


-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048113437


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/BlobStorage.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.Objects;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.datastructure.DataStructure;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.reuse.LongListReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseBag;
+import org.apache.ignite.internal.pagememory.reuse.ReuseList;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobDataIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobFirstIo;
+import org.apache.ignite.internal.storage.pagememory.mv.io.BlobIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Used to store a limited number of blobs (just byte arrays) per partition. Each blob is stored in a sequence
+ * of pages forming a linked list (a previous page links to the next one).
+ *
+ * <p>If a lot of blobs (comparable with the number of rows) needs to be stored in a partition, another mechanism
+ * (probably using a {@link org.apache.ignite.internal.pagememory.freelist.FreeList}) should be used.
+ */
+public class BlobStorage {
+    static final long NO_PAGE_ID = 0;
+
+    private final ReuseList reuseList;
+    private final PageMemory pageMemory;
+
+    private final int groupId;
+    private final int partitionId;
+
+    private final IoStatisticsHolder statisticsHolder;
+
+    private final RecycleAndAddToReuseBag recycleAndAddToReuseBag = new RecycleAndAddToReuseBag();
+
+    private final ReadFragment readFragment = new ReadFragment();
+
+    private final WriteFragment writeFragment = new WriteFragment();
+
+    /**
+     * Creates a new instance.
+     */
+    public BlobStorage(ReuseList reuseList, PageMemory pageMemory, int groupId, int partitionId, IoStatisticsHolder statisticsHolder) {
+        this.reuseList = reuseList;
+        this.pageMemory = pageMemory;
+        this.groupId = groupId;
+        this.partitionId = partitionId;
+        this.statisticsHolder = statisticsHolder;
+    }
+
+    /**
+     * Reads a blob stored starting at a page with the given ID.
+     *
+     * @param firstPageId ID of first page.
+     * @return Byte array for the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public byte[] readBlob(long firstPageId) throws IgniteInternalCheckedException {
+        ReadState readState = new ReadState();
+
+        long pageId = firstPageId;
+
+        while (pageId != NO_PAGE_ID) {
+            Boolean ok = PageHandler.readPage(
+                    pageMemory,
+                    groupId,
+                    pageId,
+                    PageLockListenerNoOp.INSTANCE,
+                    readFragment,
+                    readState,
+                    0,
+                    false,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+
+            assert ok : pageId;
+
+            pageId = readState.nextPageId;
+        }
+
+        assert readState.bytes != null;
+
+        return readState.bytes;
+    }
+
+    /**
+     * Adds a new blob to the storage.
+     *
+     * @param bytes Blob bytes.
+     * @return ID of the page starting the chain representing the blob.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public long addBlob(byte[] bytes) throws IgniteInternalCheckedException {
+        return doStore(NO_PAGE_ID, bytes);
+    }
+
+    /**
+     * Updates the blob content.
+     *
+     * @param firstPageId ID of the first page in the chain storing the blob.
+     * @param bytes New blob content.
+     * @throws IgniteInternalCheckedException If something goes wrong.
+     */
+    public void updateBlob(long firstPageId, byte[] bytes) throws IgniteInternalCheckedException {
+        doStore(firstPageId, bytes);
+    }
+
+    private long doStore(long maybeFirstPageId, byte[] bytes) throws IgniteInternalCheckedException {

Review Comment:
   Ok



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048104428


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -226,22 +262,37 @@ public void committedGroupConfiguration(RaftGroupConfiguration config) {
         assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
 
         CheckpointProgress lastCheckpoint = checkpointManager.lastCheckpointProgress();
-
         UUID lastCheckpointId = lastCheckpoint == null ? null : lastCheckpoint.id();
 
-        meta.lastGroupConfig(lastCheckpointId, groupConfigToBytes(config));
+        byte[] raftGroupConfigBytes = raftGroupConfigToBytes(config);
+
+        raftGroupConfigReadWriteLock.writeLock().lock();
+
+        try {
+            if (meta.lastRaftGroupConfigFirstPageId() == BlobStorage.NO_PAGE_ID) {
+                long configPageId = blobStorage.addBlob(raftGroupConfigBytes);
+
+                meta.lastRaftGroupConfigFirstPageId(lastCheckpointId, configPageId);
+            } else {
+                blobStorage.updateBlob(meta.lastRaftGroupConfigFirstPageId(), raftGroupConfigBytes);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot save committed group configuration, groupId=" + groupId + ", partitionId=" + groupId, e);
+        } finally {
+            raftGroupConfigReadWriteLock.writeLock().unlock();
+        }
     }
 
     @Nullable
-    private static RaftGroupConfiguration groupConfigFromBytes(byte @Nullable [] bytes) {
+    private static RaftGroupConfiguration raftGroupConfigFromBytes(byte @Nullable [] bytes) {

Review Comment:
   What is known about the new protocols?
   I would be a little embarrassed that we have a configuration framework and a separate group configuration, so it's clearer.



-- 
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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
rpuch commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1048116194


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java:
##########
@@ -226,22 +262,37 @@ public void committedGroupConfiguration(RaftGroupConfiguration config) {
         assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
 
         CheckpointProgress lastCheckpoint = checkpointManager.lastCheckpointProgress();
-
         UUID lastCheckpointId = lastCheckpoint == null ? null : lastCheckpoint.id();
 
-        meta.lastGroupConfig(lastCheckpointId, groupConfigToBytes(config));
+        byte[] raftGroupConfigBytes = raftGroupConfigToBytes(config);
+
+        raftGroupConfigReadWriteLock.writeLock().lock();
+
+        try {
+            if (meta.lastRaftGroupConfigFirstPageId() == BlobStorage.NO_PAGE_ID) {
+                long configPageId = blobStorage.addBlob(raftGroupConfigBytes);
+
+                meta.lastRaftGroupConfigFirstPageId(lastCheckpointId, configPageId);
+            } else {
+                blobStorage.updateBlob(meta.lastRaftGroupConfigFirstPageId(), raftGroupConfigBytes);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Cannot save committed group configuration, groupId=" + groupId + ", partitionId=" + groupId, e);
+        } finally {
+            raftGroupConfigReadWriteLock.writeLock().unlock();
+        }
     }
 
     @Nullable
-    private static RaftGroupConfiguration groupConfigFromBytes(byte @Nullable [] bytes) {
+    private static RaftGroupConfiguration raftGroupConfigFromBytes(byte @Nullable [] bytes) {

Review Comment:
   I added a ticket https://issues.apache.org/jira/browse/IGNITE-18405 because renaming this class is out of scope of this PR.



-- 
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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1405: IGNITE-18118 Support group configuration in persistent pagememory-based MvPartitionStorage

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #1405:
URL: https://github.com/apache/ignite-3/pull/1405#discussion_r1047109712


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1438,6 +1438,22 @@ protected abstract void writeRowData(
             boolean newRow
     ) throws IgniteInternalCheckedException;
 
+
+    /**
+     * Narrows an {@code int} down to {@code short} throwing an exception if the value cannot be exactly represented as a {@code short}.
+     *
+     * @param intValue Value to narrow down.
+     * @return Resulting short value.
+     * @throws IllegalArgumentException If the provided value does not fit the {@code short} range.
+     */
+    protected static short narrowIntToShort(int intValue) {

Review Comment:
   I do not think that users will use our product immediately in production without test runs on test environments with -ea.



-- 
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