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/06/28 08:13:08 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request, #902: IGNITE-17246 Get rid of the index partition

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

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


-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -312,25 +298,22 @@ private Path ensureGroupWorkDir(String grpName) throws IgniteInternalCheckedExce
         return groupWorkDir;
     }
 
-    private GroupPageStoreHolder<FilePageStore> createGroupFilePageStoreHolder(
+    private List<FilePageStore> createGroupFilePageStoreHolder(
             String grpName,
             int partitions
     ) throws IgniteInternalCheckedException {
         Path groupWorkDir = ensureGroupWorkDir(grpName);
 
         FilePageStoreFactory filePageStoreFactory = new FilePageStoreFactory(filePageStoreFileIoFactory, pageSize);
 
-        FilePageStore idxFilePageStore = filePageStoreFactory.createPageStore(TYPE_IDX, groupWorkDir.resolve(INDEX_FILE_NAME));
-
         FilePageStore[] partitionFilePageStores = new FilePageStore[partitions];

Review Comment:
   Fix it



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   Fact that groups have page stores.



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   do you mean cache groups?



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {

Review Comment:
   Creating files can take some more time than just putting an object into memory, from the documentation for **ConcurrentHashMap#computeIfAbsent**, it is recommended to make the function simple and fast so as not to block other threads.



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PersistentPageMemory.java:
##########
@@ -1476,11 +1476,6 @@ private void resetDirtyPages() {
         public boolean tryToRemovePage(FullPageId fullPageId, long absPtr) throws IgniteInternalCheckedException {
             assert writeLock().isHeldByCurrentThread();
 
-            // Do not evict group meta pages.

Review Comment:
   Because it was stored in the index partition, and now it will not be. It is not entirely clear whether this page will be needed in the future, in 2.0 information for snapshots was stored there before metastorage, now I think this information will be located in 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] tkalkirill commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   What about:
   - `List<FilePageStore> holder` -> `List<FilePageStore> partitionFilePageStores`
   - **createGroupFilePageStoreHolder** -> **createPartitionFilePageStores**
   - **groupPageStoreHolders** -> **groupPageStores**



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {

Review Comment:
   can we use `groupPageStoreHolders.computeIfAbsent` instead?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -113,12 +113,11 @@ private VersionChainTree createVersionChainTree(
         return new VersionChainTree(
                 groupId,
                 tableConfig.name(),
-                dataRegion.pageMemory(),
+                partitionId, dataRegion.pageMemory(),

Review Comment:
   can we rename it to `partId` for conistency?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -251,10 +237,10 @@ public FilePageStore getStore(int grpId, int partId) throws IgniteInternalChecke
      * @param cleanFiles Delete files.
      */
     void stopAllGroupFilePageStores(boolean cleanFiles) {
-        List<GroupPageStoreHolder<FilePageStore>> holders = new ArrayList<>(groupPageStoreHolders.size());
+        List<List<FilePageStore>> holders = new ArrayList<>(groupPageStoreHolders.size());
 
-        for (Iterator<GroupPageStoreHolder<FilePageStore>> it = groupPageStoreHolders.values().iterator(); it.hasNext(); ) {
-            GroupPageStoreHolder<FilePageStore> holder = it.next();
+        for (Iterator<List<FilePageStore>> it = groupPageStoreHolders.values().iterator(); it.hasNext(); ) {

Review Comment:
   Is this method only called on node stop? If yes, can we simply copy the values of `groupPageStoreHolders` and then call `clear`?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -281,7 +267,7 @@ void stopAllGroupFilePageStores(boolean cleanFiles) {
     }
 
     private static void stopGroupFilePageStores(
-            Collection<GroupPageStoreHolder<FilePageStore>> groupFilePageStoreHolders,
+            Collection<List<FilePageStore>> groupFilePageStoreHolders,

Review Comment:
   I can see that we flatten this list below. Can we do it in the `stopAllGroupFilePageStores`? The type will be more readable then



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   this variable and the corresponding method should be renamed



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PersistentPageMemory.java:
##########
@@ -1476,11 +1476,6 @@ private void resetDirtyPages() {
         public boolean tryToRemovePage(FullPageId fullPageId, long absPtr) throws IgniteInternalCheckedException {
             assert writeLock().isHeldByCurrentThread();
 
-            // Do not evict group meta pages.

Review Comment:
   Why did we get rid of "group meta pages"?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoreHolderMap.java:
##########
@@ -17,20 +17,21 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
- * Proxy class for map (grpId -> page store holder) that wraps data adding and replacing operations to disallow concurrent execution
+ * Proxy class for map (grpId -> page store list) that wraps data adding and replacing operations to disallow concurrent execution
  * simultaneously with cleanup of file page storage.
  *
  * <p>Wrapping of data removing operations is not needed.
  *
  * @param <T> Type of {@link PageStore}.
  */
-class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, GroupPageStoreHolder<T>> {
+class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, List<T>> {

Review Comment:
   Not related to this PR, but I think it is a bad idea to inherit from `ConcurrentHashMap` directly, because, for example `putAll` could delegate to `put` (I know that's it's not the case in the real implementation) which will call `afterAsyncCompletion` multiple times. I would suggest to use composition instead.



##########
modules/page-memory/src/integrationTest/java/org/apache/ignite/internal/pagememory/tree/AbstractBplusTreePageMemoryTest.java:
##########
@@ -217,12 +217,13 @@ protected long getTestTimeout() {
      * Returns reuse list.
      *
      * @param grpId Cache ID.
+     * @param partId Partition ID.
      * @param pageMem Page memory.
      * @param rootId Root page ID.
      * @param initNew Init new flag.
      * @throws Exception If failed.
      */
-    protected ReuseList createReuseList(int grpId, PageMemory pageMem, long rootId, boolean initNew) throws Exception {
+    protected ReuseList createReuseList(int grpId, int partId, PageMemory pageMem, long rootId, boolean initNew) throws Exception {

Review Comment:
   shouldn't this method be `abstract`?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainTree.java:
##########
@@ -37,61 +35,51 @@
  * {@link BplusTree} implementation for storing version chains.
  */
 public class VersionChainTree extends BplusTree<VersionChainLink, VersionChain> {
-    private final int partitionId;
-
     private final VersionChainDataPageReader dataPageReader;
 
     /**
      * Constructor.
      *
      * @param grpId Group ID.
      * @param grpName Group name.
+     * @param partitionId Partition id.
      * @param pageMem Page memory.
      * @param lockLsnr Page lock listener.
      * @param globalRmvId Global remove ID.
      * @param metaPageId Meta page ID.
      * @param reuseList Reuse list.
-     * @param partitionId Partition id.
      * @param initNew {@code True} if new tree should be created.
      */
     public VersionChainTree(
             int grpId,
             String grpName,
+            int partitionId,

Review Comment:
   same here



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java:
##########
@@ -41,6 +40,8 @@
 class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
     private static final int FREE_LIST_GROUP_ID = 0;
 
+    private static final int FREE_LIST_PARTITION_ID = 0;

Review Comment:
   So, is partition 0 still reserved for some technical purposes?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -312,25 +298,22 @@ private Path ensureGroupWorkDir(String grpName) throws IgniteInternalCheckedExce
         return groupWorkDir;
     }
 
-    private GroupPageStoreHolder<FilePageStore> createGroupFilePageStoreHolder(
+    private List<FilePageStore> createGroupFilePageStoreHolder(
             String grpName,
             int partitions
     ) throws IgniteInternalCheckedException {
         Path groupWorkDir = ensureGroupWorkDir(grpName);
 
         FilePageStoreFactory filePageStoreFactory = new FilePageStoreFactory(filePageStoreFileIoFactory, pageSize);
 
-        FilePageStore idxFilePageStore = filePageStoreFactory.createPageStore(TYPE_IDX, groupWorkDir.resolve(INDEX_FILE_NAME));
-
         FilePageStore[] partitionFilePageStores = new FilePageStore[partitions];

Review Comment:
   Why do you need an array here if you copy it into a List? Can we simply use an `ArrayList` here?



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -241,7 +226,8 @@ void testStopAllGroupFilePageStores() throws Exception {
             assertTrue(waitForCondition(
                     () -> workDir.resolve("db/group-test1").toFile().listFiles().length == 0,
                     10,
-                    100
+                    // Because deleting files happens in a new thread.

Review Comment:
   I can see that we wait for all async operations to complete when calling `manager1.stop()`. Can we simply check this condition after the manager has been stopped?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoreHolderMap.java:
##########
@@ -17,20 +17,21 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
- * Proxy class for map (grpId -> page store holder) that wraps data adding and replacing operations to disallow concurrent execution
+ * Proxy class for map (grpId -> page store list) that wraps data adding and replacing operations to disallow concurrent execution
  * simultaneously with cleanup of file page storage.
  *
  * <p>Wrapping of data removing operations is not needed.
  *
  * @param <T> Type of {@link PageStore}.
  */
-class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, GroupPageStoreHolder<T>> {
+class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, List<T>> {

Review Comment:
   This class should be renamed



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {

Review Comment:
   sounds reasonable



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   There are no caches in 3.0 only tables



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -241,7 +226,8 @@ void testStopAllGroupFilePageStores() throws Exception {
             assertTrue(waitForCondition(
                     () -> workDir.resolve("db/group-test1").toFile().listFiles().length == 0,
                     10,
-                    100
+                    // Because deleting files happens in a new thread.

Review Comment:
   Fix it



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   what is a group then?



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -281,7 +267,7 @@ void stopAllGroupFilePageStores(boolean cleanFiles) {
     }
 
     private static void stopGroupFilePageStores(
-            Collection<GroupPageStoreHolder<FilePageStore>> groupFilePageStoreHolders,
+            Collection<List<FilePageStore>> groupFilePageStoreHolders,

Review Comment:
   Fix it



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainTree.java:
##########
@@ -37,61 +35,51 @@
  * {@link BplusTree} implementation for storing version chains.
  */
 public class VersionChainTree extends BplusTree<VersionChainLink, VersionChain> {
-    private final int partitionId;
-
     private final VersionChainDataPageReader dataPageReader;
 
     /**
      * Constructor.
      *
      * @param grpId Group ID.
      * @param grpName Group name.
+     * @param partitionId Partition id.
      * @param pageMem Page memory.
      * @param lockLsnr Page lock listener.
      * @param globalRmvId Global remove ID.
      * @param metaPageId Meta page ID.
      * @param reuseList Reuse list.
-     * @param partitionId Partition id.
      * @param initNew {@code True} if new tree should be created.
      */
     public VersionChainTree(
             int grpId,
             String grpName,
+            int partitionId,

Review Comment:
   fix it



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -237,19 +237,16 @@ public FilePageStore getStore(int grpId, int partId) throws IgniteInternalChecke
      * @param cleanFiles Delete files.
      */
     void stopAllGroupFilePageStores(boolean cleanFiles) {
-        List<List<FilePageStore>> holders = new ArrayList<>(groupPageStoreHolders.size());
+        List<List<FilePageStore>> holders = List.copyOf(groupPageStoreHolders.allPageStores());
 
-        for (Iterator<List<FilePageStore>> it = groupPageStoreHolders.values().iterator(); it.hasNext(); ) {
-            List<FilePageStore> holder = it.next();
-
-            it.remove();
-
-            holders.add(holder);
-        }
+        groupPageStoreHolders.clear();
 
         Runnable stopPageStores = () -> {
             try {
-                stopGroupFilePageStores(holders, cleanFiles);
+                stopGroupFilePageStores(
+                        holders.stream().flatMap(Collection::stream).collect(toList()),

Review Comment:
   I actually meant that you could flatten this list when copying it on line 240



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -223,12 +220,10 @@ void testStopAllGroupFilePageStores() throws Exception {
 
             manager1.stopAllGroupFilePageStores(true);
 
-            assertTrue(waitForCondition(
-                    () -> workDir.resolve("db/group-test1").toFile().listFiles().length == 0,
-                    10,
-                    // Because deleting files happens in a new thread.
-                    1_000
-            ));
+            // Waits for all asynchronous operations to complete.
+            manager1.stop();
+
+            assertThat(workDir.resolve("db/group-test1").toFile().listFiles(), emptyArray());

Review Comment:
   same here



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -196,15 +196,12 @@ void testStopAllGroupFilePageStores() throws Exception {
 
             manager0.stopAllGroupFilePageStores(false);
 
-            assertFalse(waitForCondition(
-                    () -> workDir.resolve("db/group-test0").toFile().listFiles().length == 0,
-                    10,
-                    100
-            ));
+            // Waits for all asynchronous operations to complete.
+            manager0.stop();
 
             assertThat(

Review Comment:
   you now stop the manager twice, I think this assertion should simply be moved further below



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   Agree with the first two, but what do you mean by `groupPageStores`? Do we have "group pages"?



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoreHolderMap.java:
##########
@@ -17,20 +17,21 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
- * Proxy class for map (grpId -> page store holder) that wraps data adding and replacing operations to disallow concurrent execution
+ * Proxy class for map (grpId -> page store list) that wraps data adding and replacing operations to disallow concurrent execution
  * simultaneously with cleanup of file page storage.
  *
  * <p>Wrapping of data removing operations is not needed.
  *
  * @param <T> Type of {@link PageStore}.
  */
-class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, GroupPageStoreHolder<T>> {
+class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, List<T>> {

Review Comment:
   Discussed personally, we apply my proposals for renaming.



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -237,19 +237,16 @@ public FilePageStore getStore(int grpId, int partId) throws IgniteInternalChecke
      * @param cleanFiles Delete files.
      */
     void stopAllGroupFilePageStores(boolean cleanFiles) {
-        List<List<FilePageStore>> holders = new ArrayList<>(groupPageStoreHolders.size());
+        List<List<FilePageStore>> holders = List.copyOf(groupPageStoreHolders.allPageStores());
 
-        for (Iterator<List<FilePageStore>> it = groupPageStoreHolders.values().iterator(); it.hasNext(); ) {
-            List<FilePageStore> holder = it.next();
-
-            it.remove();
-
-            holders.add(holder);
-        }
+        groupPageStoreHolders.clear();
 
         Runnable stopPageStores = () -> {
             try {
-                stopGroupFilePageStores(holders, cleanFiles);
+                stopGroupFilePageStores(
+                        holders.stream().flatMap(Collection::stream).collect(toList()),

Review Comment:
   Fix it



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/integrationTest/java/org/apache/ignite/internal/pagememory/tree/AbstractBplusTreePageMemoryTest.java:
##########
@@ -217,12 +217,13 @@ protected long getTestTimeout() {
      * Returns reuse list.
      *
      * @param grpId Cache ID.
+     * @param partId Partition ID.
      * @param pageMem Page memory.
      * @param rootId Root page ID.
      * @param initNew Init new flag.
      * @throws Exception If failed.
      */
-    protected ReuseList createReuseList(int grpId, PageMemory pageMem, long rootId, boolean initNew) throws Exception {
+    protected ReuseList createReuseList(int grpId, int partId, PageMemory pageMem, long rootId, boolean initNew) throws Exception {

Review Comment:
   Fix it.



-- 
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] vveider merged pull request #902: IGNITE-17246 Get rid of the index partition

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


-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoreHolderMap.java:
##########
@@ -17,20 +17,21 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
- * Proxy class for map (grpId -> page store holder) that wraps data adding and replacing operations to disallow concurrent execution
+ * Proxy class for map (grpId -> page store list) that wraps data adding and replacing operations to disallow concurrent execution
  * simultaneously with cleanup of file page storage.
  *
  * <p>Wrapping of data removing operations is not needed.
  *
  * @param <T> Type of {@link PageStore}.
  */
-class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, GroupPageStoreHolder<T>> {
+class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, List<T>> {

Review Comment:
   Same here, what is a group page store?



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PersistentPageMemory.java:
##########
@@ -1476,11 +1476,6 @@ private void resetDirtyPages() {
         public boolean tryToRemovePage(FullPageId fullPageId, long absPtr) throws IgniteInternalCheckedException {
             assert writeLock().isHeldByCurrentThread();
 
-            // Do not evict group meta pages.

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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -251,10 +237,10 @@ public FilePageStore getStore(int grpId, int partId) throws IgniteInternalChecke
      * @param cleanFiles Delete files.
      */
     void stopAllGroupFilePageStores(boolean cleanFiles) {
-        List<GroupPageStoreHolder<FilePageStore>> holders = new ArrayList<>(groupPageStoreHolders.size());
+        List<List<FilePageStore>> holders = new ArrayList<>(groupPageStoreHolders.size());
 
-        for (Iterator<GroupPageStoreHolder<FilePageStore>> it = groupPageStoreHolders.values().iterator(); it.hasNext(); ) {
-            GroupPageStoreHolder<FilePageStore> holder = it.next();
+        for (Iterator<List<FilePageStore>> it = groupPageStoreHolders.values().iterator(); it.hasNext(); ) {

Review Comment:
   fix it



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoreHolderMap.java:
##########
@@ -17,20 +17,21 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
- * Proxy class for map (grpId -> page store holder) that wraps data adding and replacing operations to disallow concurrent execution
+ * Proxy class for map (grpId -> page store list) that wraps data adding and replacing operations to disallow concurrent execution
  * simultaneously with cleanup of file page storage.
  *
  * <p>Wrapping of data removing operations is not needed.
  *
  * @param <T> Type of {@link PageStore}.
  */
-class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, GroupPageStoreHolder<T>> {
+class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, List<T>> {

Review Comment:
   What about **GroupPageStoreHolderMap** -> **GroupPageStoresMap** ?



-- 
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] sashapolo commented on a diff in pull request #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java:
##########
@@ -41,6 +40,8 @@
 class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
     private static final int FREE_LIST_GROUP_ID = 0;
 
+    private static final int FREE_LIST_PARTITION_ID = 0;

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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -177,19 +168,19 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
      *
      * @param grpName Group name.
      * @param grpId Group ID.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID} + 1.
+     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
      * @throws IgniteInternalCheckedException If failed.
      */
     public void initialize(String grpName, int grpId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID + 1 : partitions;
+        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
 
         initGroupDirLock.lock(grpId);
 
         try {
             if (!groupPageStoreHolders.containsKey(grpId)) {
-                GroupPageStoreHolder<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);
+                List<FilePageStore> holder = createGroupFilePageStoreHolder(grpName, partitions);

Review Comment:
   Discussed personally, we apply my proposals for renaming.



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoreHolderMap.java:
##########
@@ -17,20 +17,21 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.BiFunction;
 import java.util.function.Function;
 
 /**
- * Proxy class for map (grpId -> page store holder) that wraps data adding and replacing operations to disallow concurrent execution
+ * Proxy class for map (grpId -> page store list) that wraps data adding and replacing operations to disallow concurrent execution
  * simultaneously with cleanup of file page storage.
  *
  * <p>Wrapping of data removing operations is not needed.
  *
  * @param <T> Type of {@link PageStore}.
  */
-class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, GroupPageStoreHolder<T>> {
+class GroupPageStoreHolderMap<T extends PageStore> extends ConcurrentHashMap<Integer, List<T>> {

Review Comment:
   Made a composition.



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -196,15 +196,12 @@ void testStopAllGroupFilePageStores() throws Exception {
 
             manager0.stopAllGroupFilePageStores(false);
 
-            assertFalse(waitForCondition(
-                    () -> workDir.resolve("db/group-test0").toFile().listFiles().length == 0,
-                    10,
-                    100
-            ));
+            // Waits for all asynchronous operations to complete.
+            manager0.stop();
 
             assertThat(

Review Comment:
   Fix it



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -223,12 +220,10 @@ void testStopAllGroupFilePageStores() throws Exception {
 
             manager1.stopAllGroupFilePageStores(true);
 
-            assertTrue(waitForCondition(
-                    () -> workDir.resolve("db/group-test1").toFile().listFiles().length == 0,
-                    10,
-                    // Because deleting files happens in a new thread.
-                    1_000
-            ));
+            // Waits for all asynchronous operations to complete.
+            manager1.stop();
+
+            assertThat(workDir.resolve("db/group-test1").toFile().listFiles(), emptyArray());

Review Comment:
   Fix it



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java:
##########
@@ -41,6 +40,8 @@
 class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
     private static final int FREE_LIST_GROUP_ID = 0;
 
+    private static final int FREE_LIST_PARTITION_ID = 0;

Review Comment:
   Not reserved, but selected for a memory case, this is not necessary for a persistent case.
   Maybe we'll change that in the future.



-- 
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 #902: IGNITE-17246 Get rid of the index partition

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java:
##########
@@ -113,12 +113,11 @@ private VersionChainTree createVersionChainTree(
         return new VersionChainTree(
                 groupId,
                 tableConfig.name(),
-                dataRegion.pageMemory(),
+                partitionId, dataRegion.pageMemory(),

Review Comment:
   Fix it



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