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

[13/43] ignite git commit: ignite-4712 review

ignite-4712 review


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

Branch: refs/heads/ignite-4712
Commit: d376ea9862bbe3be7d9f4a53f2237e55fb92364f
Parents: 5e4e991
Author: sboikov <sb...@gridgain.com>
Authored: Tue Feb 21 17:32:29 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Feb 21 17:32:29 2017 +0300

----------------------------------------------------------------------
 .../cache/database/freelist/PagesList.java      | 19 ++++--
 .../database/IgniteDbAbstractTest.java          | 68 +++++++++-----------
 .../IgniteDbMemoryLeakAbstractTest.java         | 36 +++++++----
 .../IgniteDbMemoryLeakWithExpirationTest.java   |  5 +-
 .../IgniteDbMemoryLeakSqlQueryTest.java         | 17 +++--
 5 files changed, 80 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d376ea98/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
index 5ae3549..7c88041 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
@@ -201,7 +201,8 @@ public abstract class PagesList extends DataStructure {
                     for (int i = 0; i < upd.length; i++) {
                         long tailId = upd[i];
 
-                        try(Page tail = page(tailId)) {
+                        // TODO: need get full bucket size.
+                        try (Page tail = page(tailId)) {
                             long tailAddr = readLock(tail);
 
                             assert tailAddr != 0L;
@@ -847,10 +848,11 @@ public abstract class PagesList extends DataStructure {
 
         while (true) {
             Stripe stripe = tails[cur];
-            if(!stripe.empty)
+
+            if (!stripe.empty)
                 return stripe;
 
-            if((cur = (cur + 1) % len) == init)
+            if ((cur = (cur + 1) % len) == init)
                 return null;
         }
     }
@@ -921,12 +923,16 @@ public abstract class PagesList extends DataStructure {
                     continue;
                 }
 
-                if(!isReuseBucket(bucket) && stripe.empty) {
-                    // Another thread took the last page
+                // TODO: condition !isReuseBucket(bucket) is not correct.
+                if (!isReuseBucket(bucket) && stripe.empty) {
+                    // Another thread took the last page.
                     writeUnlock(tail, tailPageAddr, false);
 
-                    if(bucketsSize[bucket].get() > 0)
+                    if (bucketsSize[bucket].get() > 0) {
+                        lockAttempt = 0;
+
                         continue;
+                    }
                     else
                         return 0L;
                 }
@@ -958,6 +964,7 @@ public abstract class PagesList extends DataStructure {
 
                         boolean empty = io.isEmpty(tailPageAddr);
 
+                        // TODO: add comment, it seems flag is not set to correct value for reuse bucket.
                         stripe.empty = empty;
 
                         // If we got an empty page in non-reuse bucket, move it back to reuse list

http://git-wip-us.apache.org/repos/asf/ignite/blob/d376ea98/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
index 5f4d5e0..46efab2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
@@ -17,10 +17,8 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheRebalanceMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cache.affinity.AffinityFunction;
+import java.io.Serializable;
+import java.util.Arrays;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -34,9 +32,10 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.Random;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 
 /**
  *
@@ -56,7 +55,8 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
     protected abstract boolean indexingEnabled();
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked") @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         MemoryConfiguration dbCfg = new MemoryConfiguration();
@@ -80,9 +80,9 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
         if (indexingEnabled())
             ccfg.setIndexedTypes(Integer.class, DbValue.class);
 
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
         ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
 
         CacheConfiguration ccfg2 = new CacheConfiguration("non-primitive");
@@ -90,9 +90,9 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
         if (indexingEnabled())
             ccfg2.setIndexedTypes(DbKey.class, DbValue.class);
 
-        ccfg2.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg2.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg2.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg2.setAtomicityMode(TRANSACTIONAL);
+        ccfg2.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg2.setRebalanceMode(SYNC);
         ccfg2.setAffinity(new RendezvousAffinityFunction(false, 32));
 
         CacheConfiguration ccfg3 = new CacheConfiguration("large");
@@ -100,32 +100,28 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
         if (indexingEnabled())
             ccfg3.setIndexedTypes(Integer.class, LargeDbValue.class);
 
-        ccfg3.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg3.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg3.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg3.setAtomicityMode(TRANSACTIONAL);
+        ccfg3.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg3.setRebalanceMode(SYNC);
         ccfg3.setAffinity(new RendezvousAffinityFunction(false, 32));
 
         CacheConfiguration ccfg4 = new CacheConfiguration("tiny");
 
-        ccfg4.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg4.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg4.setRebalanceMode(CacheRebalanceMode.SYNC);
-        ccfg4.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg4.setAtomicityMode(TRANSACTIONAL);
+        ccfg4.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg4.setRebalanceMode(SYNC);
+        ccfg4.setAffinity(new RendezvousAffinityFunction(1, null));
 
         CacheConfiguration ccfg5 = new CacheConfiguration("atomic");
 
         if (indexingEnabled())
             ccfg5.setIndexedTypes(DbKey.class, DbValue.class);
 
-        ccfg5.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-        ccfg5.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg5.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg5.setAtomicityMode(ATOMIC);
+        ccfg5.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg5.setRebalanceMode(SYNC);
         ccfg5.setAffinity(new RendezvousAffinityFunction(false, 32));
 
-        final AffinityFunction aff = new RendezvousAffinityFunction(1, null);
-
-        ccfg4.setAffinity(aff);
-
         cfg.setCacheConfiguration(ccfg, ccfg2, ccfg3, ccfg4, ccfg5);
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
@@ -144,25 +140,25 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
      * @param cfg IgniteConfiguration.
      */
     protected void configure(IgniteConfiguration cfg){
-        //NOP
+        // No-op.
     }
 
     /**
      * @param mCfg MemoryConfiguration.
      */
     protected void configure(MemoryConfiguration mCfg){
-        //NOP
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
 
-        long seed = 1464583813940L; // System.currentTimeMillis();
-
-        info("Seed: " + seed + "L");
-
-        BPlusTree.rnd = new Random(seed);
+//        long seed = System.currentTimeMillis();
+//
+//        info("Seed: " + seed + "L");
+//
+//        BPlusTree.rnd = new Random(seed);
 
         startGrids(gridCount());
 
@@ -275,8 +271,6 @@ public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
         @QuerySqlField
         long lVal;
 
-
-
         /**
          * @param iVal Integer value.
          * @param sVal String value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d376ea98/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
index 93306d9..aa32495 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
@@ -30,13 +30,13 @@ import org.apache.ignite.internal.processors.cache.database.DataStructure;
  */
 public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTest {
     /** */
-    @SuppressWarnings("WeakerAccess") protected static final int CONCURRENCY_LEVEL = 8;
+    private static final int CONCURRENCY_LEVEL = 8;
 
     /** */
     private static final int MIN_PAGE_CACHE_SIZE = 1048576 * CONCURRENCY_LEVEL;
 
     /** */
-    private volatile Exception ex = null;
+    private volatile Exception ex;
 
     /** */
     private long warmUpEndTime;
@@ -45,21 +45,24 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
     private long endTime;
 
     /** */
-    private long loadedPages = 0;
+    private long loadedPages;
 
     /** */
-    private long delta = 0;
+    private long delta;
 
     /** */
-    private long probeCnt = 0;
+    private long probeCnt;
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
+
         DataStructure.rnd = null;
 
         long startTime = System.nanoTime();
+
         warmUpEndTime = startTime + TimeUnit.SECONDS.toNanos(warmUp());
+
         endTime = warmUpEndTime + TimeUnit.SECONDS.toNanos(duration());
     }
 
@@ -73,6 +76,7 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
         mCfg.setConcurrencyLevel(CONCURRENCY_LEVEL);
 
         long size = 1024 * pagesMax() * (isLargePage() ? 16 : 1);
+
         mCfg.setPageCacheSize(Math.max(size, MIN_PAGE_CACHE_SIZE));
     }
 
@@ -86,7 +90,8 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
     /**
      * @return Warm up duration in seconds.
      */
-    @SuppressWarnings("WeakerAccess") protected int warmUp() {
+    @SuppressWarnings("WeakerAccess")
+    protected int warmUp() {
         return 300;
     }
 
@@ -102,7 +107,7 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
 
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return (warmUp() + duration() + 1) * 2000; // Two extra seconds to stop all threads.
+        return (warmUp() + duration() + 10) * 1000; // Extra seconds to stop all threads.
     }
 
     /**
@@ -127,17 +132,19 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
      */
     protected void operation(IgniteCache<Object, Object> cache) {
         Object key = key();
-        Object value = value(key);
+        Object val = value(key);
 
         switch (nextInt(3)) {
             case 0:
-                cache.getAndPut(key, value);
+                cache.getAndPut(key, val);
 
                 break;
+
             case 1:
                 cache.get(key);
 
                 break;
+
             case 2:
                 cache.getAndRemove(key);
         }
@@ -173,6 +180,7 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
                     }
                     catch (Exception e) {
                         ex = e;
+
                         break;
                     }
                 }
@@ -223,18 +231,19 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
         long pagesMax = pagesMax();
 
         assertTrue(
-            "Maximal allowed pages number is exceeded. [allowed=" + pagesMax + "; actual= " + pagesActual + "]",
+            "Maximal allowed pages number is exceeded [allowed=" + pagesMax + ", actual= " + pagesActual + "]",
             pagesActual <= pagesMax);
 
         if (loadedPages > 0) {
             delta += pagesActual - loadedPages;
+
             int allowedDelta = pagesDelta();
 
-            if(probeCnt++ > 12) { // we need some statistic first. Minimal statistic is taken for a minute.
+            if (probeCnt++ > 12) { // We need some statistic first. Minimal statistic is taken for a minute.
                 long actualDelta = delta / probeCnt;
 
                 assertTrue(
-                    "Average growth pages in the number is more than expected. [allowed=" + allowedDelta + "; actual=" + actualDelta + "]",
+                    "Average growth pages in the number is more than expected [allowed=" + allowedDelta + ", actual=" + actualDelta + "]",
                     actualDelta <= allowedDelta);
             }
         }
@@ -250,7 +259,8 @@ public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTes
     /**
      * @return Expected average number of pages, on which their total number can grow per 5 seconds.
      */
-    @SuppressWarnings("WeakerAccess") protected int pagesDelta() {
+    @SuppressWarnings("WeakerAccess")
+    protected int pagesDelta() {
         return 3;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d376ea98/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
index 19502b0..6e0abaf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
@@ -17,12 +17,11 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.internal.IgniteEx;
-
 import javax.cache.expiry.CreatedExpiryPolicy;
 import javax.cache.expiry.Duration;
 import javax.cache.expiry.ExpiryPolicy;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.IgniteEx;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d376ea98/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakSqlQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakSqlQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakSqlQueryTest.java
index b65cac0..57f9fb5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakSqlQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakSqlQueryTest.java
@@ -38,21 +38,24 @@ public class IgniteDbMemoryLeakSqlQueryTest extends IgniteDbMemoryLeakTest {
     /** {@inheritDoc} */
     @Override protected void operation(IgniteCache<Object, Object> cache) {
         Object key = key();
-        Object value = value(key);
+        Object val = value(key);
 
         switch (nextInt(4)) {
             case 0:
-                cache.getAndPut(key, value);
+                cache.getAndPut(key, val);
 
                 break;
+
             case 1:
                 cache.get(key);
 
                 break;
+
             case 2:
                 cache.getAndRemove(key);
 
                 break;
+
             case 3:
                 cache.query(sqlQuery(cache)).getAll();
         }
@@ -63,9 +66,11 @@ public class IgniteDbMemoryLeakSqlQueryTest extends IgniteDbMemoryLeakTest {
      * @return SqlFieldsQuery.
      */
     @NotNull private SqlFieldsQuery sqlQuery(IgniteCache<Object, Object> cache) {
-        String query = String.format("select _key from \"%s\".DbValue where iVal=?", cache.getName());
-        SqlFieldsQuery sqlQuery = new SqlFieldsQuery(query);
-        sqlQuery.setArgs(nextInt(200_000));
-        return sqlQuery;
+        String qry = String.format("select _key from \"%s\".DbValue where iVal=?", cache.getName());
+
+        SqlFieldsQuery sqlQry = new SqlFieldsQuery(qry);
+        sqlQry.setArgs(nextInt(200_000));
+
+        return sqlQry;
     }
 }