You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/10/13 17:43:23 UTC

[38/50] [abbrv] ignite git commit: IGNITE-6385 Offheap page eviction doesn't work if data is loaded without data streamer - Fixes #2666.

IGNITE-6385 Offheap page eviction doesn't work if data is loaded without data streamer - Fixes #2666.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>

(cherry picked from commit 64b4abe)

(cherry picked from commit 7ca611f)


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/7daefd27
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/7daefd27
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/7daefd27

Branch: refs/heads/ignite-2.1.5-p1
Commit: 7daefd2701c7a566d759c2b7de4912ea3f5d37a8
Parents: 1faa8db
Author: Ivan Rakov <iv...@gmail.com>
Authored: Mon Sep 18 11:35:17 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Fri Sep 22 15:28:54 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/freelist/PagesList.java   | 35 ++++++-----
 .../paged/PageEvictionDataStreamerTest.java     | 65 ++++++++++++++++++++
 .../paged/PageEvictionMultinodeTest.java        |  6 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |  2 +
 4 files changed, 91 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
index 6c355f6..8a540a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -154,9 +155,8 @@ public abstract class PagesList extends DataStructure {
      */
     protected final void init(long metaPageId, boolean initNew) throws IgniteCheckedException {
         if (metaPageId != 0L) {
-            if (initNew) {
+            if (initNew)
                 init(metaPageId, PagesListMetaIO.VERSIONS.latest());
-            }
             else {
                 Map<Integer, GridLongList> bucketsData = new HashMap<>();
 
@@ -509,10 +509,14 @@ public abstract class PagesList extends DataStructure {
      */
     private Stripe getPageForPut(int bucket) throws IgniteCheckedException {
         // Striped pool optimization.
-        int stripeIdx; IgniteThread igniteThread = IgniteThread.current();
+        IgniteThread igniteThread = IgniteThread.current();
 
-        if (igniteThread != null && (stripeIdx = igniteThread.stripe()) != -1) {
-            Stripe[] tails = getBucket(bucket);
+        Stripe[] tails = getBucket(bucket);
+
+        if (igniteThread != null && igniteThread.policy() == GridIoPolicy.DATA_STREAMER_POOL) {
+            int stripeIdx = igniteThread.stripe();
+
+            assert stripeIdx != -1 : igniteThread;
 
             while (tails == null || stripeIdx >= tails.length) {
                 addStripe(bucket, true);
@@ -523,8 +527,6 @@ public abstract class PagesList extends DataStructure {
             return tails[stripeIdx];
         }
 
-        Stripe[] tails = getBucket(bucket);
-
         if (tails == null)
             return addStripe(bucket, true);
 
@@ -854,9 +856,8 @@ public abstract class PagesList extends DataStructure {
 
                         assert nextPageAddr != 0L;
 
-                        if (locked == null) {
+                        if (locked == null)
                             locked = new GridLongList(6);
-                        }
 
                         locked.add(nextId);
                         locked.add(nextPage);
@@ -911,9 +912,8 @@ public abstract class PagesList extends DataStructure {
                 updateTail(bucket, pageId, prevId);
 
                 // Release write.
-                for (int i = 0; i < locked.size(); i+=3) {
+                for (int i = 0; i < locked.size(); i += 3)
                     writeUnlock(locked.get(i), locked.get(i + 1), locked.get(i + 2), FALSE, true);
-                }
             }
         }
 
@@ -933,9 +933,13 @@ public abstract class PagesList extends DataStructure {
         int len = tails.length;
 
         // Striped pool optimization.
-        int stripeIdx; IgniteThread igniteThread = IgniteThread.current();
+        IgniteThread igniteThread = IgniteThread.current();
+
+        if (igniteThread != null && igniteThread.policy() == GridIoPolicy.DATA_STREAMER_POOL) {
+            int stripeIdx = igniteThread.stripe();
+
+            assert stripeIdx != -1 : igniteThread;
 
-        if (igniteThread != null && (stripeIdx = igniteThread.stripe()) != -1) {
             if (stripeIdx >= len)
                 return null;
 
@@ -971,8 +975,11 @@ public abstract class PagesList extends DataStructure {
         // Striped pool optimization.
         IgniteThread igniteThread = IgniteThread.current();
 
-        if (igniteThread != null && igniteThread.stripe() != -1)
+        if (igniteThread != null && igniteThread.policy() == GridIoPolicy.DATA_STREAMER_POOL) {
+            assert igniteThread.stripe() != -1 : igniteThread;
+
             return writeLock(pageId, page);
+        }
 
         long pageAddr = tryWriteLock(pageId, page);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java
new file mode 100644
index 0000000..6b6d687
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionDataStreamerTest.java
@@ -0,0 +1,65 @@
+/*
+* 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.processors.cache.eviction.paged;
+
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class PageEvictionDataStreamerTest extends PageEvictionMultinodeTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_LRU, super.getConfiguration(gridName));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
+        IgniteCache<Object, Object> cache = clientGrid.getOrCreateCache(cfg);
+
+        try (IgniteDataStreamer<Object, Object> ldr = clientGrid.dataStreamer(cfg.getName())) {
+            ldr.allowOverwrite(true);
+
+            for (int i = 1; i <= ENTRIES; i++) {
+                ThreadLocalRandom r = ThreadLocalRandom.current();
+
+                if (r.nextInt() % 5 == 0)
+                    ldr.addData(i, new TestObject(PAGE_SIZE / 4 - 50 + r.nextInt(5000))); // Fragmented object.
+                else
+                    ldr.addData(i, new TestObject(r.nextInt(PAGE_SIZE / 4 - 50))); // Fits in one page.
+
+                if (i % (ENTRIES / 10) == 0)
+                    System.out.println(">>> Entries put: " + i);
+            }
+        }
+
+        int resultingSize = cache.size(CachePeekMode.PRIMARY);
+
+        System.out.println(">>> Resulting size: " + resultingSize);
+
+        // Eviction started, no OutOfMemory occurred, success.
+        assertTrue(resultingSize < ENTRIES);
+
+        clientGrid.destroyCache(cfg.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
index c2c0775..e4587aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
@@ -42,7 +42,7 @@ public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest
         CacheWriteSynchronizationMode.FULL_SYNC, CacheWriteSynchronizationMode.FULL_ASYNC};
 
     /** Client grid. */
-    private Ignite clientGrid;
+    Ignite clientGrid;
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
@@ -94,7 +94,7 @@ public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest
      * @param cfg Config.
      * @throws Exception If failed.
      */
-    private void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
+    protected void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
         IgniteCache<Object, Object> cache = clientGrid.getOrCreateCache(cfg);
 
         for (int i = 1; i <= ENTRIES; i++) {
@@ -121,7 +121,7 @@ public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest
         System.out.println(">>> Resulting size: " + resultingSize);
 
         // Eviction started, no OutOfMemory occurred, success.
-        assertTrue(resultingSize < ENTRIES);
+        assertTrue(resultingSize < ENTRIES * 10 / 11);
 
         clientGrid.destroyCache(cfg.getName());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7daefd27/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
index 9f03c60..7985e2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.eviction.fifo.FifoEvictionPol
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearOnlyNearEvictionPolicySelfTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionDataStreamerTest;
 import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionReadThroughTest;
 import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionTouchOrderTest;
 import org.apache.ignite.internal.processors.cache.eviction.paged.Random2LruNearEnabledPageEvictionMultinodeTest;
@@ -79,6 +80,7 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(Random2LruPageEvictionWithRebalanceTest.class));
         suite.addTest(new TestSuite(PageEvictionTouchOrderTest.class));
         suite.addTest(new TestSuite(PageEvictionReadThroughTest.class));
+        suite.addTest(new TestSuite(PageEvictionDataStreamerTest.class));
 
         return suite;
     }