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 2021/08/10 21:40:30 UTC

[GitHub] [ignite] Mmuzaf opened a new pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Mmuzaf opened a new pull request #9317:
URL: https://github.com/apache/ignite/pull/9317


   
   
   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


-- 
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] Mmuzaf commented on a change in pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9317:
URL: https://github.com/apache/ignite/pull/9317#discussion_r692418572



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -271,6 +273,12 @@ public IndexStorage getIndexStorage() {
 
     /** {@inheritDoc} */
     @Override public void beforeCheckpointBegin(Context ctx) throws IgniteCheckedException {
+        List<CacheDataStore> destroyedStores = StreamSupport.stream(cacheDataStores().spliterator(), false)

Review comment:
       I've added there additional guarantees - destroyed cache data stores must not be visible to listeners. I've fixed the assertion as you suggested.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -398,33 +373,46 @@ public CacheDataStore dataStore(int part) {
 
     /** {@inheritDoc} */
     @Override public long cacheEntriesCount(int cacheId, int part) {
-        CacheDataStore store = partitionData(part);
+        CacheDataStore store = dataStore(part, true);
 
         return store == null ? 0 : store.cacheSize(cacheId);
     }
 
+    /** {@inheritDoc} */
+    @Override public Iterable<CacheDataStore> cacheDataStores() {
+        return cacheDataStores(grp -> new AlwaysTruePredicate<>());

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -398,33 +373,46 @@ public CacheDataStore dataStore(int part) {
 
     /** {@inheritDoc} */
     @Override public long cacheEntriesCount(int cacheId, int part) {
-        CacheDataStore store = partitionData(part);
+        CacheDataStore store = dataStore(part, true);
 
         return store == null ? 0 : store.cacheSize(cacheId);
     }
 
+    /** {@inheritDoc} */
+    @Override public Iterable<CacheDataStore> cacheDataStores() {
+        return cacheDataStores(grp -> new AlwaysTruePredicate<>());
+    }
+
+    /**
+     * @param factory Factory which will produce filtering predicate.

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -398,33 +373,46 @@ public CacheDataStore dataStore(int part) {
 
     /** {@inheritDoc} */
     @Override public long cacheEntriesCount(int cacheId, int part) {
-        CacheDataStore store = partitionData(part);
+        CacheDataStore store = dataStore(part, true);
 
         return store == null ? 0 : store.cacheSize(cacheId);
     }
 
+    /** {@inheritDoc} */
+    @Override public Iterable<CacheDataStore> cacheDataStores() {
+        return cacheDataStores(grp -> new AlwaysTruePredicate<>());
+    }
+
+    /**
+     * @param factory Factory which will produce filtering predicate.
+     * @return Iterable over all existing cache data stores except which one is marked as <tt>destroyed</tt>.
+     */
+    private Iterable<CacheDataStore> cacheDataStores(
+        Function<CacheGroupContext, IgnitePredicate<GridDhtLocalPartition>> factory
+    ) {
+        return grp.isLocal() ? Collections.singletonList(locCacheDataStore) :
+            F.iterator(grp.topology().currentLocalPartitions(), GridDhtLocalPartition::dataStore, true,
+                factory.apply(grp), p -> !p.dataStore().destroyed());
+    }
+
     /**
      * @param primary Primary data flag.
-     * @param backup Primary data flag.
+     * @param backup Backup data flag.
      * @param topVer Topology version.
      * @return Data stores iterator.
      */
     private Iterator<CacheDataStore> cacheData(boolean primary, boolean backup, AffinityTopologyVersion topVer) {
         assert primary || backup;
 
-        if (grp.isLocal())
-            return singletonIterator(locCacheDataStore);
-        else {
-            Iterator<GridDhtLocalPartition> it = grp.topology().currentLocalPartitions().iterator();
-
+        return cacheDataStores(grp -> {
             if (primary && backup)
-                return F.iterator(it, GridDhtLocalPartition::dataStore, true);
+                return new AlwaysTruePredicate<>();

Review comment:
       Fixed.




-- 
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] Mmuzaf commented on a change in pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9317:
URL: https://github.com/apache/ignite/pull/9317#discussion_r692419854



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -1546,6 +1498,11 @@ public CacheDataStoreImpl(
             mvccUpdateMarker = new MvccMarkUpdatedHandler(grp);
             mvccUpdateTxStateHint = new MvccUpdateTxStateHintHandler(grp);
             mvccApplyChanges = new MvccApplyChangesHandler(grp);
+
+            if (cleaner == null)

Review comment:
       This is done for lazy cache data store initaliztion. Currently, if the page store exists it will trigger the initialization of the cache data store, however, it's not required on partition creation.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -1280,22 +1239,21 @@ private long allocateForTree() throws IgniteCheckedException {
 
     /** {@inheritDoc} */
     @Override public final CacheDataStore createCacheDataStore(int p) throws IgniteCheckedException {
-        CacheDataStore dataStore;
-
         partStoreLock.lock(p);
 
         try {
-            assert !partDataStores.containsKey(p);
+            CacheDataStore store = createCacheDataStore0(p);
 
-            dataStore = createCacheDataStore0(p);
+            // Inject row cache cleaner on store creation.
+            // Used in case the cache with enabled SqlOnheapCache is single cache at the cache group.
+            if (ctx.kernalContext().query().moduleEnabled() && !grp.isLocal())
+                store.setRowCacheCleaner(ctx.kernalContext().indexProcessor().rowCacheCleaner(grp.groupId()));

Review comment:
       Fixed.




-- 
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] Mmuzaf commented on a change in pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9317:
URL: https://github.com/apache/ignite/pull/9317#discussion_r693083590



##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
##########
@@ -914,7 +914,7 @@ public void testCacheIdleVerifyDumpForCorruptedDataOnSystemCache() throws Except
             U.log(log, dumpWithConflicts);
 
             // Non-persistent caches do not have counter conflicts
-            assertContains(log, dumpWithConflicts, "found 3 conflict partitions: [counterConflicts=1, " +
+            assertContains(log, dumpWithConflicts, "found 4 conflict partitions: [counterConflicts=2, " +

Review comment:
       I've created a new issue to fix this.
   https://issues.apache.org/jira/browse/IGNITE-15355




-- 
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] alex-plekhanov commented on a change in pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9317:
URL: https://github.com/apache/ignite/pull/9317#discussion_r691970123



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -271,6 +273,12 @@ public IndexStorage getIndexStorage() {
 
     /** {@inheritDoc} */
     @Override public void beforeCheckpointBegin(Context ctx) throws IgniteCheckedException {
+        List<CacheDataStore> destroyedStores = StreamSupport.stream(cacheDataStores().spliterator(), false)

Review comment:
       1. If you do some actions only to check assertions, it's better to wrap it to `if (U.assertioEnabled()) ...`
   2. In this case `assert F.size(cacheDataStores().iterator(), CacheDataStore::destroyed) == 0;` looks simpler to understand and do not produce extra objects.
   3. Why do we need this check at all, if `cacheDataStores()` doesn't return destroyed partitions?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -398,33 +373,46 @@ public CacheDataStore dataStore(int part) {
 
     /** {@inheritDoc} */
     @Override public long cacheEntriesCount(int cacheId, int part) {
-        CacheDataStore store = partitionData(part);
+        CacheDataStore store = dataStore(part, true);
 
         return store == null ? 0 : store.cacheSize(cacheId);
     }
 
+    /** {@inheritDoc} */
+    @Override public Iterable<CacheDataStore> cacheDataStores() {
+        return cacheDataStores(grp -> new AlwaysTruePredicate<>());
+    }
+
+    /**
+     * @param factory Factory which will produce filtering predicate.

Review comment:
       Why do we need a factory of predicates here? Function argument `grp` is the same cache group as stored in this offheap manager. We can pass only predicate with the same behavior.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -398,33 +373,46 @@ public CacheDataStore dataStore(int part) {
 
     /** {@inheritDoc} */
     @Override public long cacheEntriesCount(int cacheId, int part) {
-        CacheDataStore store = partitionData(part);
+        CacheDataStore store = dataStore(part, true);
 
         return store == null ? 0 : store.cacheSize(cacheId);
     }
 
+    /** {@inheritDoc} */
+    @Override public Iterable<CacheDataStore> cacheDataStores() {
+        return cacheDataStores(grp -> new AlwaysTruePredicate<>());

Review comment:
       ` new AlwaysTruePredicate<>()` -> `F.alwaysTrue()`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -2699,10 +2699,9 @@ private RestoreLogicalState applyLogicalUpdates(
                         CacheGroupContext ctx = cctx.cache().cacheGroup(rbRec.groupId());
 
                         if (ctx != null && !ctx.isLocal()) {
-                            ctx.topology().forceCreatePartition(rbRec.partitionId());
+                            GridDhtLocalPartition part = ctx.topology().forceCreatePartition(rbRec.partitionId());
 
-                            ctx.offheap().onPartitionInitialCounterUpdated(rbRec.partitionId(), rbRec.start(),
-                                rbRec.range());
+                            ctx.offheap().dataStore(part).updateCounter(rbRec.start(), rbRec.range());

Review comment:
       `onPartitionInitialCounterUpdated` was equivalent for `updateInitialCounter`, but why `updateCounter` here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -1280,22 +1239,21 @@ private long allocateForTree() throws IgniteCheckedException {
 
     /** {@inheritDoc} */
     @Override public final CacheDataStore createCacheDataStore(int p) throws IgniteCheckedException {
-        CacheDataStore dataStore;
-
         partStoreLock.lock(p);
 
         try {
-            assert !partDataStores.containsKey(p);
+            CacheDataStore store = createCacheDataStore0(p);
 
-            dataStore = createCacheDataStore0(p);
+            // Inject row cache cleaner on store creation.
+            // Used in case the cache with enabled SqlOnheapCache is single cache at the cache group.
+            if (ctx.kernalContext().query().moduleEnabled() && !grp.isLocal())
+                store.setRowCacheCleaner(ctx.kernalContext().indexProcessor().rowCacheCleaner(grp.groupId()));

Review comment:
       Why this part was moved from `GridDhtLocalPartition`? I see the only difference: now row cache cleaner is set under the lock, but looks like this lock doesn't protect anything related to row cache cleaner.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
##########
@@ -100,7 +100,7 @@ public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cac
      * @param tag Partition tag (growing 1-based partition file version).
      * @throws IgniteCheckedException If failed to handle partition destroy callback.
      */
-    public void onPartitionDestroyed(int grpId, int partId, int tag) throws IgniteCheckedException;
+    public void truncate(int grpId, int partId, int tag) throws IgniteCheckedException;

Review comment:
       What's wrong with the old method name? It's consistent with `onPartitionCreated`, can we keep it?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -1290,18 +1271,18 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException {
     }
 
     /** {@inheritDoc} */
-    @Override public void preloadPartition(int part) throws IgniteCheckedException {
+    @Override public void preloadPartition(int partId) throws IgniteCheckedException {
         if (grp.isLocal()) {
-            dataStore(part).preload();
+            dataStore(null).preload();
 
             return;
         }
 
-        GridDhtLocalPartition locPart = grp.topology().localPartition(part, AffinityTopologyVersion.NONE, false, false);
+        GridDhtLocalPartition locPart = grp.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false);
 
         assert locPart != null && locPart.reservations() > 0;
 
-        locPart.dataStore().preload();
+        dataStore(locPart).preload();

Review comment:
       I think this change is redundant (it's an additional check in `dataStore()` for local caches, but we already checked it)

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/CacheScanQueryFailoverTest.java
##########
@@ -172,7 +173,7 @@ public void testScanQueryOnEvictedPartition() throws Exception {
         // Force checkpoint to destroy evicted partitions store.
         forceCheckpoint(grid0);
 
-        GridTestUtils.assertThrowsAnyCause(log, iter1::next, IgniteException.class, "Failed to get next data row");
+        GridTestUtils.assertThrows(log, iter1::next, NoSuchElementException.class, null);

Review comment:
       `assertFalse(iter1.hasNext());`

##########
File path: modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerClusterByClassTest.java
##########
@@ -914,7 +914,7 @@ public void testCacheIdleVerifyDumpForCorruptedDataOnSystemCache() throws Except
             U.log(log, dumpWithConflicts);
 
             // Non-persistent caches do not have counter conflicts
-            assertContains(log, dumpWithConflicts, "found 3 conflict partitions: [counterConflicts=1, " +
+            assertContains(log, dumpWithConflicts, "found 4 conflict partitions: [counterConflicts=2, " +

Review comment:
       Why count of conflicts changed? I didn't get how it related to changes made.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -398,33 +373,46 @@ public CacheDataStore dataStore(int part) {
 
     /** {@inheritDoc} */
     @Override public long cacheEntriesCount(int cacheId, int part) {
-        CacheDataStore store = partitionData(part);
+        CacheDataStore store = dataStore(part, true);
 
         return store == null ? 0 : store.cacheSize(cacheId);
     }
 
+    /** {@inheritDoc} */
+    @Override public Iterable<CacheDataStore> cacheDataStores() {
+        return cacheDataStores(grp -> new AlwaysTruePredicate<>());
+    }
+
+    /**
+     * @param factory Factory which will produce filtering predicate.
+     * @return Iterable over all existing cache data stores except which one is marked as <tt>destroyed</tt>.
+     */
+    private Iterable<CacheDataStore> cacheDataStores(
+        Function<CacheGroupContext, IgnitePredicate<GridDhtLocalPartition>> factory
+    ) {
+        return grp.isLocal() ? Collections.singletonList(locCacheDataStore) :
+            F.iterator(grp.topology().currentLocalPartitions(), GridDhtLocalPartition::dataStore, true,
+                factory.apply(grp), p -> !p.dataStore().destroyed());
+    }
+
     /**
      * @param primary Primary data flag.
-     * @param backup Primary data flag.
+     * @param backup Backup data flag.
      * @param topVer Topology version.
      * @return Data stores iterator.
      */
     private Iterator<CacheDataStore> cacheData(boolean primary, boolean backup, AffinityTopologyVersion topVer) {
         assert primary || backup;
 
-        if (grp.isLocal())
-            return singletonIterator(locCacheDataStore);
-        else {
-            Iterator<GridDhtLocalPartition> it = grp.topology().currentLocalPartitions().iterator();
-
+        return cacheDataStores(grp -> {
             if (primary && backup)
-                return F.iterator(it, GridDhtLocalPartition::dataStore, true);
+                return new AlwaysTruePredicate<>();

Review comment:
       `F.alwaysTrue()`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -624,7 +612,7 @@ public CacheDataStore dataStore(int part) {
     /** {@inheritDoc} */
     @Nullable @Override public CacheDataRow read(GridCacheContext cctx, KeyCacheObject key)
         throws IgniteCheckedException {
-        CacheDataStore dataStore = dataStore(cctx, key);
+        CacheDataStore dataStore = dataStore(cctx.affinity().partition(key), false);

Review comment:
       Are you sure that the call of `cctx.affinity().partition(key)` for local cache doesn't throw an exception (for example, if affinity assignment cache is not set for local caches)? I think it's better to keep the old method (with local cache check).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -310,26 +308,28 @@ private void removeCacheData(int cacheId) {
         }
     }
 
-    /**
-     * @param part Partition.
-     * @return Data store for given entry.
-     */
-    @Override public CacheDataStore dataStore(GridDhtLocalPartition part) {
+    /** {@inheritDoc} */
+    @Override public CacheDataStore dataStore(@Nullable GridDhtLocalPartition part) {
         if (grp.isLocal())
             return locCacheDataStore;
-        else {
-            assert part != null;
 
-            return part.dataStore();
-        }
+        assert part != null;
+
+        return part.dataStore();
     }
 
     /**
-     * @param part Partition.
-     * @return Data store for given entry.
+     * @param partId Partition id.
+     * @param renting {@code true} if renting partitions must also be shown.

Review comment:
       I think a name like `includeRenting` is more appropriate.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -1546,6 +1498,11 @@ public CacheDataStoreImpl(
             mvccUpdateMarker = new MvccMarkUpdatedHandler(grp);
             mvccUpdateTxStateHint = new MvccUpdateTxStateHintHandler(grp);
             mvccApplyChanges = new MvccApplyChangesHandler(grp);
+
+            if (cleaner == null)

Review comment:
       What is the purpose of change row cache cleaner to a supplier? The code complexity is about the same, the amount of code is about the same, the behavior is about the same. 




-- 
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] Mmuzaf merged pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
Mmuzaf merged pull request #9317:
URL: https://github.com/apache/ignite/pull/9317


   


-- 
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] Mmuzaf commented on a change in pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9317:
URL: https://github.com/apache/ignite/pull/9317#discussion_r692418848



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
##########
@@ -2699,10 +2699,9 @@ private RestoreLogicalState applyLogicalUpdates(
                         CacheGroupContext ctx = cctx.cache().cacheGroup(rbRec.groupId());
 
                         if (ctx != null && !ctx.isLocal()) {
-                            ctx.topology().forceCreatePartition(rbRec.partitionId());
+                            GridDhtLocalPartition part = ctx.topology().forceCreatePartition(rbRec.partitionId());
 
-                            ctx.offheap().onPartitionInitialCounterUpdated(rbRec.partitionId(), rbRec.start(),
-                                rbRec.range());
+                            ctx.offheap().dataStore(part).updateCounter(rbRec.start(), rbRec.range());

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -310,26 +308,28 @@ private void removeCacheData(int cacheId) {
         }
     }
 
-    /**
-     * @param part Partition.
-     * @return Data store for given entry.
-     */
-    @Override public CacheDataStore dataStore(GridDhtLocalPartition part) {
+    /** {@inheritDoc} */
+    @Override public CacheDataStore dataStore(@Nullable GridDhtLocalPartition part) {
         if (grp.isLocal())
             return locCacheDataStore;
-        else {
-            assert part != null;
 
-            return part.dataStore();
-        }
+        assert part != null;
+
+        return part.dataStore();
     }
 
     /**
-     * @param part Partition.
-     * @return Data store for given entry.
+     * @param partId Partition id.
+     * @param renting {@code true} if renting partitions must also be shown.

Review comment:
       Fixed.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/CacheScanQueryFailoverTest.java
##########
@@ -172,7 +173,7 @@ public void testScanQueryOnEvictedPartition() throws Exception {
         // Force checkpoint to destroy evicted partitions store.
         forceCheckpoint(grid0);
 
-        GridTestUtils.assertThrowsAnyCause(log, iter1::next, IgniteException.class, "Failed to get next data row");
+        GridTestUtils.assertThrows(log, iter1::next, NoSuchElementException.class, null);

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
##########
@@ -624,7 +612,7 @@ public CacheDataStore dataStore(int part) {
     /** {@inheritDoc} */
     @Nullable @Override public CacheDataRow read(GridCacheContext cctx, KeyCacheObject key)
         throws IgniteCheckedException {
-        CacheDataStore dataStore = dataStore(cctx, key);
+        CacheDataStore dataStore = dataStore(cctx.affinity().partition(key), false);

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
##########
@@ -1290,18 +1271,18 @@ private Metas getOrAllocateCacheMetas() throws IgniteCheckedException {
     }
 
     /** {@inheritDoc} */
-    @Override public void preloadPartition(int part) throws IgniteCheckedException {
+    @Override public void preloadPartition(int partId) throws IgniteCheckedException {
         if (grp.isLocal()) {
-            dataStore(part).preload();
+            dataStore(null).preload();
 
             return;
         }
 
-        GridDhtLocalPartition locPart = grp.topology().localPartition(part, AffinityTopologyVersion.NONE, false, false);
+        GridDhtLocalPartition locPart = grp.topology().localPartition(partId, AffinityTopologyVersion.NONE, false, false);
 
         assert locPart != null && locPart.reservations() > 0;
 
-        locPart.dataStore().preload();
+        dataStore(locPart).preload();

Review comment:
       Fixed.




-- 
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] Mmuzaf commented on a change in pull request #9317: IGNITE-15283 Remove duplicated managing of CacheDataStore in offheap manager

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9317:
URL: https://github.com/apache/ignite/pull/9317#discussion_r692417895



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
##########
@@ -100,7 +100,7 @@ public void initializeForCache(CacheGroupDescriptor grpDesc, StoredCacheData cac
      * @param tag Partition tag (growing 1-based partition file version).
      * @throws IgniteCheckedException If failed to handle partition destroy callback.
      */
-    public void onPartitionDestroyed(int grpId, int partId, int tag) throws IgniteCheckedException;
+    public void truncate(int grpId, int partId, int tag) throws IgniteCheckedException;

Review comment:
       I suggest keeping this `as is`. Currently, the method naming has a too broader scope, so it's hard identify what's exactly happening under the hood without reading the method implementation. 
   The `onPartitionCreated ` is not used at all, I've removed them too.




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