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/08/05 09:19:24 UTC

[GitHub] [ignite-3] sashapolo commented on a diff in pull request #976: IGNITE-17085 Support persistent case for page-memory-based MV storage

sashapolo commented on code in PR #976:
URL: https://github.com/apache/ignite-3/pull/976#discussion_r938592991


##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaTest.java:
##########
@@ -80,32 +80,89 @@ void testPageCount() {
         assertEquals(2, meta.pageCount());
     }
 
+    @Test
+    void testVersionChainTreeRootPageId() {
+        PartitionMeta meta = new PartitionMeta(null, 0, 0, 0, 0, 0, 0);
+
+        assertEquals(0, meta.versionChainTreeRootPageId());
+
+        assertDoesNotThrow(() -> meta.versionChainTreeRootPageId(null, 100));

Review Comment:
   I think `assertDoesNotThrow` is pretty useless, but you can leave it if you want



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -164,8 +170,113 @@ public void destroy() throws StorageException {
 
     /** {@inheritDoc} */
     @Override
-    public PageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
-        throw new UnsupportedOperationException("Not supported yet");
+    public PersistentPageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
+        TableView tableView = tableCfg.value();
+
+        FilePageStore filePageStore = ensurePartitionFilePageStore(tableView, partitionId);
+
+        CheckpointManager checkpointManager = dataRegion.checkpointManager();
+
+        CheckpointTimeoutLock checkpointTimeoutLock = checkpointManager.checkpointTimeoutLock();

Review Comment:
   Not related to this PR, but I think that `CheckpointTimeoutLock` is a very misleading name



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java:
##########
@@ -104,60 +103,43 @@ public void start() {
     }
 
     private TableFreeList createTableFreeList(PageMemory pageMemory) throws IgniteInternalCheckedException {
-        long metaPageId = pageMemory.allocatePage(FREE_LIST_GROUP_ID, FREE_LIST_PARTITION_ID, FLAG_AUX);
-
         return new TableFreeList(
                 FREE_LIST_GROUP_ID,
                 FREE_LIST_PARTITION_ID,
                 pageMemory,
                 PageLockListenerNoOp.INSTANCE,
-                metaPageId,
+                pageMemory.allocatePage(FREE_LIST_GROUP_ID, FREE_LIST_PARTITION_ID, FLAG_AUX),

Review Comment:
   Because it was more clear that this allocated page is a meta page



##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java:
##########
@@ -259,9 +210,9 @@ void scanByTimestampWorksCorrectlyAfterCommitAndAbortFollowedByUncommittedWrite(
 
     @Test
     void readByTimestampWorksCorrectlyIfNoUncommittedValueExists() {
-        LinkRowId rowId = storage.insert(binaryRow, txId);
+        LinkRowId rowId = (LinkRowId) insert(binaryRow, txId);

Review Comment:
   what is this cast for?



##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java:
##########
@@ -21,114 +21,55 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
-import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.nio.file.Path;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.IntStream;
-import org.apache.ignite.configuration.schemas.store.UnknownDataStorageConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.ConstantValueDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.FunctionCallDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.HashIndexConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.NullValueDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.TableConfiguration;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
-import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
-import org.apache.ignite.internal.pagememory.configuration.schema.UnsafeMemoryAllocatorConfigurationSchema;
 import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.storage.AbstractMvPartitionStorageTest;
 import org.apache.ignite.internal.storage.RowId;
-import org.apache.ignite.internal.storage.pagememory.AbstractPageMemoryTableStorage;
-import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryStorageEngine;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageConfigurationSchema;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageView;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfiguration;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfigurationSchema;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+/**
+ * Base test for MV partition storages based on PageMemory.
+ */
 @ExtendWith(ConfigurationExtension.class)
 @ExtendWith(WorkDirectoryExtension.class)
-class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<PageMemoryMvPartitionStorage> {
-    private final PageIoRegistry ioRegistry = new PageIoRegistry();
+abstract class AbstractPageMemoryMvPartitionStorageTest<T extends AbstractPageMemoryMvPartitionStorage> extends
+        AbstractMvPartitionStorageTest<T> {
+    protected final PageIoRegistry ioRegistry = new PageIoRegistry();
 
-    private final BinaryRow binaryRow3 = binaryRow(key, new TestValue(22, "bar3"));
+    protected final BinaryRow binaryRow3 = binaryRow(key, new TestValue(22, "bar3"));

Review Comment:
   can still be private



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -226,21 +336,92 @@ TableFreeList createTableFreeList(
         }
     }
 
+    /**
+     * Returns new {@link VersionChainFreeList} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param rootPageId Root page ID.
+     * @param initNew {@code True} if new metadata should be initialized.
+     * @throws StorageException If failed.
+     */
+    VersionChainFreeList createVersionChainFreeList(

Review Comment:
   also, all of them can be `private`



##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java:
##########
@@ -21,114 +21,55 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
-import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.nio.file.Path;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.IntStream;
-import org.apache.ignite.configuration.schemas.store.UnknownDataStorageConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.ConstantValueDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.FunctionCallDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.HashIndexConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.NullValueDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.TableConfiguration;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
-import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
-import org.apache.ignite.internal.pagememory.configuration.schema.UnsafeMemoryAllocatorConfigurationSchema;
 import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.storage.AbstractMvPartitionStorageTest;
 import org.apache.ignite.internal.storage.RowId;
-import org.apache.ignite.internal.storage.pagememory.AbstractPageMemoryTableStorage;
-import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryStorageEngine;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageConfigurationSchema;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageView;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfiguration;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfigurationSchema;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+/**
+ * Base test for MV partition storages based on PageMemory.
+ */
 @ExtendWith(ConfigurationExtension.class)
 @ExtendWith(WorkDirectoryExtension.class)
-class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<PageMemoryMvPartitionStorage> {
-    private final PageIoRegistry ioRegistry = new PageIoRegistry();
+abstract class AbstractPageMemoryMvPartitionStorageTest<T extends AbstractPageMemoryMvPartitionStorage> extends

Review Comment:
   Why does `AbstractMvPartitionStorageTest` have a generic parameter? I was able to remove it from this class and from all other inheritors without any implications



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaTest.java:
##########
@@ -80,32 +80,89 @@ void testPageCount() {
         assertEquals(2, meta.pageCount());
     }
 
+    @Test
+    void testVersionChainTreeRootPageId() {

Review Comment:
   What are these tests for? Just to test setters and getters?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -226,21 +336,92 @@ TableFreeList createTableFreeList(
         }
     }
 
+    /**
+     * Returns new {@link VersionChainFreeList} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param rootPageId Root page ID.
+     * @param initNew {@code True} if new metadata should be initialized.
+     * @throws StorageException If failed.
+     */
+    VersionChainFreeList createVersionChainFreeList(

Review Comment:
   Maybe all these *create* methods could be moved to a factory? This will also reduce the number of parameters, since some of them will be stored in the factory itself



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