SammyVimes commented on code in PR #845:
URL: https://github.com/apache/ignite-3/pull/845#discussion_r889912238


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PageMemoryStorageEngine.java:
##########
@@ -43,29 +55,88 @@ public class PageMemoryStorageEngine implements 
StorageEngine {
     /** Engine name. */
     public static final String ENGINE_NAME = "pagememory";
 
+    private final String igniteInstanceName;
+
     private final PageMemoryStorageEngineConfiguration engineConfig;
 
     private final PageIoRegistry ioRegistry;
 
-    private final Map<String, VolatilePageMemoryDataRegion> regions = new 
ConcurrentHashMap<>();
+    private final Path storagePath;
+
+    @Nullable
+    private final LongJvmPauseDetector longJvmPauseDetector;
+
+    private final Map<String, AbstractPageMemoryDataRegion> regions = new 
ConcurrentHashMap<>();
+
+    @Nullable
+    private volatile FilePageStoreManager filePageStoreManager;
+
+    @Nullable
+    private volatile CheckpointManager checkpointManager;

Review Comment:
   Seems like `filePageStoreManager` and `checkpointManager` can be made 
non-volatile. Not sure though, but I think our start process is single-threaded



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/io/PartitionMetaIo.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.storage.pagememory.io;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+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.lang.IgniteStringBuilder;
+
+/**
+ * Io for partition metadata pages.
+ */
+public class PartitionMetaIo extends PageIo {
+    private static final int TREE_ROOT_PAGE_ID_OFF = COMMON_HEADER_END;
+
+    private static final int REUSE_LIST_ROOT_PAGE_ID_OFF = 
TREE_ROOT_PAGE_ID_OFF + 8;

Review Comment:
   ```suggestion
       private static final int REUSE_LIST_ROOT_PAGE_ID_OFF = 
TREE_ROOT_PAGE_ID_OFF + Long.BYTES;
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/io/PartitionMetaIo.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.storage.pagememory.io;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+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.lang.IgniteStringBuilder;
+
+/**
+ * Io for partition metadata pages.
+ */
+public class PartitionMetaIo extends PageIo {
+    private static final int TREE_ROOT_PAGE_ID_OFF = COMMON_HEADER_END;
+
+    private static final int REUSE_LIST_ROOT_PAGE_ID_OFF = 
TREE_ROOT_PAGE_ID_OFF + 8;
+
+    /** Page IO type. */
+    public static final short T_TABLE_PARTITION_META_IO = 7;

Review Comment:
   I think all of the Io types should be stored as constants in a separate 
class. This way making a mistake of having same types for different IOs will be 
less probable



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryDataRegion.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.util.Constants.GiB;
+import static org.apache.ignite.internal.util.Constants.MiB;
+
+import java.util.Arrays;
+import 
org.apache.ignite.internal.pagememory.configuration.schema.PageMemoryDataRegionConfiguration;
+import 
org.apache.ignite.internal.pagememory.configuration.schema.PageMemoryDataRegionView;
+import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
+import org.apache.ignite.internal.pagememory.persistence.PageMemoryImpl;
+import 
org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointManager;
+import 
org.apache.ignite.internal.pagememory.persistence.store.FilePageStoreManager;
+
+/**
+ * Implementation of {@link AbstractPageMemoryDataRegion} for persistent case.
+ */
+class PersistentPageMemoryDataRegion extends AbstractPageMemoryDataRegion {
+    private final FilePageStoreManager filePageStoreManager;
+
+    private final CheckpointManager checkpointManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     * @param ioRegistry IO registry.
+     * @param filePageStoreManager File page store manager.
+     * @param checkpointManager Checkpoint manager.
+     * @param pageSize Page size in bytes.
+     */
+    public PersistentPageMemoryDataRegion(
+            PageMemoryDataRegionConfiguration cfg,
+            PageIoRegistry ioRegistry,
+            FilePageStoreManager filePageStoreManager,
+            CheckpointManager checkpointManager,
+            int pageSize
+    ) {
+        super(cfg, ioRegistry, pageSize);
+
+        this.filePageStoreManager = filePageStoreManager;
+        this.checkpointManager = checkpointManager;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void start() {
+        PageMemoryDataRegionView dataRegionConfigView = cfg.value();
+
+        assert persistent() : dataRegionConfigView.name();
+
+        PageMemoryImpl pageMemoryImpl = new PageMemoryImpl(
+                cfg,
+                ioRegistry,
+                calculateSegmentSizes(dataRegionConfigView, 
Runtime.getRuntime().availableProcessors()),
+                calculateCheckpointBufferSize(dataRegionConfigView),
+                filePageStoreManager,
+                null,
+                (fullPageId, buf, tag) -> {
+                    // Write page to disk.
+                    filePageStoreManager.write(fullPageId.groupId(), 
fullPageId.pageId(), buf, tag, true);
+                },
+                checkpointManager.checkpointTimeoutLock(),
+                pageSize
+        );
+
+        pageMemoryImpl.start();
+
+        pageMemory = pageMemoryImpl;
+    }
+
+    /**
+     * Returns file page store manager.
+     */
+    public FilePageStoreManager filePageStoreManager() {
+        return filePageStoreManager;
+    }
+
+    /**
+     * Returns checkpoint manager.
+     */
+    public CheckpointManager checkpointManager() {
+        return checkpointManager;
+    }
+
+    /**
+     * Calculates the size of segments in bytes.
+     *
+     * @param dataRegionConfigView Data region configuration.
+     * @param concurrencyLevel Number of concurrent segments in Ignite 
internal page mapping tables, must be greater than 0.
+     */
+    static long[] calculateSegmentSizes(PageMemoryDataRegionView 
dataRegionConfigView, int concurrencyLevel) {
+        assert concurrencyLevel > 0 : concurrencyLevel;
+
+        long maxSize = dataRegionConfigView.maxSize();
+
+        long fragmentSize = Math.max(maxSize / concurrencyLevel, MiB);
+
+        long[] sizes = new long[concurrencyLevel];
+
+        Arrays.fill(sizes, fragmentSize);
+
+        return sizes;
+    }
+
+    /**
+     * Calculates the size of the checkpoint buffer in bytes.

Review Comment:
   More info on what's happening here won't hurt, because in other way the 
numbers look odd



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.storage.StorageUtils.groupId;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.FullPageId;
+import org.apache.ignite.internal.pagememory.evict.PageEvictionTrackerNoOp;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.persistence.PageMemoryImpl;
+import 
org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.pagememory.persistence.store.FilePageStore;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.pagememory.io.PartitionMetaIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Implementation of {@link AbstractPageMemoryTableStorage} for persistent 
case.
+ */
+class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
+    /**
+     * Constructor.
+     *
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     */
+    public PersistentPageMemoryTableStorage(
+            TableConfiguration tableCfg,
+            PersistentPageMemoryDataRegion dataRegion
+    ) {
+        super(tableCfg, dataRegion);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void start() throws StorageException {
+        super.start();
+
+        TableView tableView = tableCfg.value();
+
+        try {
+            // TODO: IGNITE-16665 Directory name needs to be corrected to 
support table renaming
+            ((PersistentPageMemoryDataRegion) dataRegion)
+                    .filePageStoreManager()
+                    .initialize(tableView.name(), groupId(tableView), 
tableView.partitions());
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Error initializing file page stores 
for table: " + tableView.name(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected VolatilePageMemoryPartitionStorage createPartitionStorage(int 
partId) throws StorageException {

Review Comment:
   Shouldn't it be PersistentPageMemoryPartitionStorage?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
+import static org.apache.ignite.internal.storage.StorageUtils.groupId;
+
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.FullPageId;
+import org.apache.ignite.internal.pagememory.evict.PageEvictionTrackerNoOp;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.pagememory.persistence.PageMemoryImpl;
+import 
org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.pagememory.persistence.store.FilePageStore;
+import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.pagememory.io.PartitionMetaIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Implementation of {@link AbstractPageMemoryTableStorage} for persistent 
case.
+ */
+class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
+    /**
+     * Constructor.
+     *
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     */
+    public PersistentPageMemoryTableStorage(
+            TableConfiguration tableCfg,
+            PersistentPageMemoryDataRegion dataRegion
+    ) {
+        super(tableCfg, dataRegion);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void start() throws StorageException {
+        super.start();
+
+        TableView tableView = tableCfg.value();
+
+        try {
+            // TODO: IGNITE-16665 Directory name needs to be corrected to 
support table renaming
+            ((PersistentPageMemoryDataRegion) dataRegion)
+                    .filePageStoreManager()
+                    .initialize(tableView.name(), groupId(tableView), 
tableView.partitions());
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException("Error initializing file page stores 
for table: " + tableView.name(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected VolatilePageMemoryPartitionStorage createPartitionStorage(int 
partId) throws StorageException {
+        TableView tableView = tableCfg.value();
+
+        FilePageStore partitionFilePageStore = 
ensurePartitionFilePageStore(tableView, partId);
+
+        CheckpointTimeoutLock checkpointTimeoutLock = 
((PersistentPageMemoryDataRegion) dataRegion)
+                .checkpointManager()
+                .checkpointTimeoutLock();
+
+        checkpointTimeoutLock.checkpointReadLock();
+
+        try {
+            PartitionMeta partitionMeta = 
getOrCreateTableTreePartitionMetas(tableView, partId, partitionFilePageStore);
+
+            TableFreeList tableFreeList = createTableFreeList(tableView, 
partId, partitionMeta);
+
+            autoCloseables.add(tableFreeList::close);
+
+            TableTree tableTree = createTableTree(tableView, partId, 
tableFreeList, partitionMeta);
+
+            return new PersistentPageMemoryPartitionStorage(partId, 
tableFreeList, tableTree, checkpointTimeoutLock);
+        } finally {
+            checkpointTimeoutLock.checkpointReadUnlock();
+        }
+    }
+
+    /**
+     * Initializes the partition file page store if it hasn't already.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @return Partition file page store.
+     * @throws StorageException If failed.
+     */
+    FilePageStore ensurePartitionFilePageStore(TableView tableView, int 
partId) throws StorageException {
+        try {
+            FilePageStore filePageStore = ((PersistentPageMemoryDataRegion) 
dataRegion)
+                    .filePageStoreManager()
+                    .getStore(groupId(tableView), partId);
+
+            filePageStore.ensure();
+
+            return filePageStore;
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException(
+                    String.format("Error initializing file page store 
[tableName=%s, partitionId=%s]", tableView.name(), partId),
+                    e
+            );
+        }
+    }
+
+    /**
+     * Returns the read and created new partition {@link PartitionMeta}.

Review Comment:
   the read?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to