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/07 16:59:44 UTC

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

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