You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/05/22 17:44:24 UTC

[01/15] ignite git commit: ignite-gg-12163 fix null cache name, fix eviction test (init memory size)

Repository: ignite
Updated Branches:
  refs/heads/ignite-5267 [created] e31f37171


ignite-gg-12163 fix null cache name, fix eviction test (init memory size)


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

Branch: refs/heads/ignite-5267
Commit: 3f92485680647353f2cf9e774f6c4b891072e1ee
Parents: e0bf4e0
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 11:09:23 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 11:09:23 2017 +0300

----------------------------------------------------------------------
 .../database/db/IgniteDbPageEvictionSelfTest.java     | 14 +++++++-------
 .../database/db/file/PageStoreEvictionSelfTest.java   |  2 +-
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3f924856/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbPageEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbPageEvictionSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbPageEvictionSelfTest.java
index 1d1ca46..ed68b64 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbPageEvictionSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbPageEvictionSelfTest.java
@@ -57,11 +57,11 @@ public class IgniteDbPageEvictionSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        MemoryConfiguration dbCfg = new MemoryConfiguration();
+        MemoryConfiguration memCfg = new MemoryConfiguration();
 
-        dbCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
+        memCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
 
-        dbCfg.setPageSize(1024);
+        memCfg.setPageSize(1024);
 
         MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
 
@@ -69,12 +69,12 @@ public class IgniteDbPageEvictionSelfTest extends GridCommonAbstractTest {
         memPlcCfg.setInitialSize(50 * 1024 * 1024);
         memPlcCfg.setMaxSize(50 * 1024 * 1024);
 
-        dbCfg.setMemoryPolicies(memPlcCfg);
-        dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+        memCfg.setMemoryPolicies(memPlcCfg);
+        memCfg.setDefaultMemoryPolicyName("dfltMemPlc");
 
-        cfg.setMemoryConfiguration(dbCfg);
+        cfg.setMemoryConfiguration(memCfg);
 
-        CacheConfiguration<DbKey, DbValue> ccfg = new CacheConfiguration<>();
+        CacheConfiguration<DbKey, DbValue> ccfg = new CacheConfiguration<>(CACHE_NAME);
 
         ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         ccfg.setRebalanceMode(CacheRebalanceMode.NONE);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f924856/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreEvictionSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreEvictionSelfTest.java
index 1c43582..cc43613 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreEvictionSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/PageStoreEvictionSelfTest.java
@@ -54,7 +54,7 @@ public class PageStoreEvictionSelfTest extends GridCommonAbstractTest {
     private static final long CHUNK_SIZE = 1024 * 1024;
 
     /** */
-    private static final long MEMORY_LIMIT = 5 * CHUNK_SIZE;
+    private static final long MEMORY_LIMIT = 10 * CHUNK_SIZE;
 
     /** */
     private static final int PAGES_NUM = 128_000;


[02/15] ignite git commit: Fixed whole cluster restart test

Posted by ag...@apache.org.
Fixed whole cluster restart test


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

Branch: refs/heads/ignite-5267
Commit: 05c6c4ebfba2f820eb1bdfec57cd4fe353d83b49
Parents: 3f92485
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon May 22 11:18:29 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon May 22 11:18:51 2017 +0300

----------------------------------------------------------------------
 .../db/IgniteDbWholeClusterRestartSelfTest.java | 28 ++++++++++++++------
 1 file changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/05c6c4eb/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbWholeClusterRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbWholeClusterRestartSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbWholeClusterRestartSelfTest.java
index cce3b6e..fa988f2 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbWholeClusterRestartSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/IgniteDbWholeClusterRestartSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.configuration.PersistenceConfiguration;
 import org.apache.ignite.internal.processors.cache.database.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.checkpoint.noop.NoopCheckpointSpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -76,6 +77,10 @@ public class IgniteDbWholeClusterRestartSelfTest extends GridCommonAbstractTest
         ccfg1.setBackups(2);
 
         cfg.setLateAffinityAssignment(false);
+        cfg.setActiveOnStart(false);
+
+        // To avoid hostname lookup on start.
+        cfg.setCheckpointSpi(new NoopCheckpointSpi());
 
         cfg.setCacheConfiguration(ccfg1);
 
@@ -122,6 +127,8 @@ public class IgniteDbWholeClusterRestartSelfTest extends GridCommonAbstractTest
     public void testRestarts() throws Exception {
         startGrids(GRID_CNT);
 
+        ignite(0).active(true);
+
         awaitPartitionMapExchange();
 
         try (IgniteDataStreamer<Object, Object> ds = ignite(0).dataStreamer(CACHE_NAME)) {
@@ -144,16 +151,21 @@ public class IgniteDbWholeClusterRestartSelfTest extends GridCommonAbstractTest
             for (Integer idx : idxs)
                 startGrid(idx);
 
-            for (int g = 0; g < GRID_CNT; g++) {
-                Ignite ig = ignite(g);
+            try {
+                ignite(0).active(true);
 
-                for (int k = 0; k < ENTRIES_COUNT; k++)
-                    assertEquals("Failed to read [g=" + g + ", part=" + ig.affinity(CACHE_NAME).partition(k) +
-                        ", nodes=" + ig.affinity(CACHE_NAME).mapKeyToPrimaryAndBackups(k) + ']',
-                        k, ig.cache(CACHE_NAME).get(k));
-            }
+                for (int g = 0; g < GRID_CNT; g++) {
+                    Ignite ig = ignite(g);
 
-            stopAllGrids();
+                    for (int k = 0; k < ENTRIES_COUNT; k++)
+                        assertEquals("Failed to read [g=" + g + ", part=" + ig.affinity(CACHE_NAME).partition(k) +
+                            ", nodes=" + ig.affinity(CACHE_NAME).mapKeyToPrimaryAndBackups(k) + ']',
+                            k, ig.cache(CACHE_NAME).get(k));
+                }
+            }
+            finally {
+                stopAllGrids();
+            }
         }
     }
 }


[10/15] ignite git commit: ignite-gg-12163 IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE for avoid OOM

Posted by ag...@apache.org.
ignite-gg-12163 IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE for avoid OOM


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

Branch: refs/heads/ignite-5267
Commit: f3bb4740c3da5a32df03fd8ef2f08ace6ed9ffdc
Parents: f41c4fb
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 15:46:41 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 15:46:41 2017 +0300

----------------------------------------------------------------------
 .../cache/database/wal/FileWriteAheadLogManager.java    | 12 +++++++++++-
 1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f3bb4740/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
index 30b56d7..1ef9384 100644
--- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
+++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
@@ -109,6 +109,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** */
     public static final String IGNITE_PDS_WAL_FSYNC_DELAY = "IGNITE_PDS_WAL_FSYNC_DELAY"; // TODO may be move to config
 
+    /** Ignite pds wal record iterator buffer size. */
+    public static final String IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE = "IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE";
+
     /** */
     public static final String IGNITE_PDS_WAL_ALWAYS_WRITE_FULL_PAGES = "IGNITE_PDS_WAL_ALWAYS_WRITE_FULL_PAGES";
 
@@ -1933,6 +1936,10 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** */
         private static final long serialVersionUID = 0L;
 
+        /** Buffer size. */
+        private final int buffSize = IgniteSystemProperties.getInteger(
+            IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE, 64 * 1024 * 1024);
+
         /** */
         private final File walWorkDir;
 
@@ -2005,8 +2012,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             this.end = end;
             this.log = log;
 
+            int tlbSize0 = 16 * tlbSize;
+            int buffSize0 = tlbSize0 < buffSize ? tlbSize0 : buffSize;
+
             // Do not allocate direct buffer for iterator.
-            buf = ByteBuffer.allocate(16 * tlbSize);
+            buf = ByteBuffer.allocate(buffSize0);
             buf.order(ByteOrder.nativeOrder());
 
             init();


[03/15] ignite git commit: ignite-gg-12163 WAL_REBALANCE_THRESHOLD property restored in test

Posted by ag...@apache.org.
ignite-gg-12163 WAL_REBALANCE_THRESHOLD property restored in test


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

Branch: refs/heads/ignite-5267
Commit: 4c575f0f433659760b2f17ea80cbc55d24109551
Parents: e0bf4e0
Author: Sergey Chugunov <se...@gmail.com>
Authored: Mon May 22 12:13:43 2017 +0300
Committer: Sergey Chugunov <se...@gmail.com>
Committed: Mon May 22 12:14:37 2017 +0300

----------------------------------------------------------------------
 .../IgniteWalHistoryReservationsSelfTest.java     | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4c575f0f/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
index a92a29f..d4d8de5 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
@@ -39,6 +39,8 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_REBALANCE_THRESHOLD;
+
 /**
  *
  */
@@ -71,7 +73,13 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
 
         cfg.setCacheConfiguration(ccfg1);
 
-        cfg.setPersistenceConfiguration(new PersistenceConfiguration());
+        PersistenceConfiguration pstCfg = new PersistenceConfiguration();
+
+        pstCfg.setPersistenceStorePath("/ramdisk/ignite/db/database");
+        pstCfg.setWalStorePath("/ramdisk/ignite/db/wal");
+        pstCfg.setWalArchivePath("/ramdisk/ignite/db/wal_archive");
+
+        cfg.setPersistenceConfiguration(pstCfg);
 
         return cfg;
     }
@@ -85,6 +93,8 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
+        System.clearProperty(IGNITE_WAL_REBALANCE_THRESHOLD);
+
         client = false;
 
         stopAllGrids();
@@ -96,6 +106,8 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testReservedOnExchange() throws Exception {
+        System.setProperty(IGNITE_WAL_REBALANCE_THRESHOLD, "0");
+
         final int entryCnt = 10_000;
         final int initGridCnt = 4;
 
@@ -179,6 +191,8 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testRemovesArePreloadedIfHistoryIsAvailable() throws Exception {
+        System.setProperty(IGNITE_WAL_REBALANCE_THRESHOLD, "0");
+
         int entryCnt = 10_000;
 
         Ignite ig0 = startGrids(2);
@@ -268,6 +282,8 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testNodeLeftDuringExchange() throws Exception {
+        System.setProperty(IGNITE_WAL_REBALANCE_THRESHOLD, "0");
+
         final int entryCnt = 10_000;
         final int initGridCnt = 4;
 


[07/15] ignite git commit: Fixed assertion error - page eviction should be before locking entry

Posted by ag...@apache.org.
Fixed assertion error - page eviction should be before locking entry

(cherry picked from commit 711bbb7)


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

Branch: refs/heads/ignite-5267
Commit: cd17ee0fee9c3c5fba357f1190e5ffbe9a7e3681
Parents: 90717b4
Author: Ivan Rakov <iv...@gmail.com>
Authored: Mon May 22 14:33:29 2017 +0300
Committer: Ivan Rakov <iv...@gmail.com>
Committed: Mon May 22 14:36:27 2017 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/atomic/GridDhtAtomicCache.java         | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cd17ee0f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index b8cb56f..0dafa2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1765,6 +1765,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         ctx.shared().database().checkpointReadLock();
 
         try {
+            ctx.shared().database().ensureFreeSpace(ctx.memoryPolicy());
+
             // If batch store update is enabled, we need to lock all entries.
             // First, need to acquire locks on cache entries, then check filter.
             List<GridDhtCacheEntry> locked = lockEntries(req, req.topologyVersion());;
@@ -1788,8 +1790,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     // Do not check topology version if topology was locked on near node by
                     // external transaction or explicit lock.
                     if (req.topologyLocked() || !needRemap(req.topologyVersion(), top.topologyVersion())) {
-                        ctx.shared().database().ensureFreeSpace(ctx.memoryPolicy());
-
                         boolean hasNear = ctx.discovery().cacheNearNode(node, name());
 
                         // Assign next version for update inside entries lock.


[12/15] ignite git commit: ignite-gg-12163 IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE cosmetic update

Posted by ag...@apache.org.
ignite-gg-12163 IGNITE_PDS_WAL_RECORD_ITERATOR_BUFFER_SIZE cosmetic update


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

Branch: refs/heads/ignite-5267
Commit: e78441c03b2cb1e4d5d6f26462e98eb88b2720c9
Parents: f3bb474
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 16:15:11 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 16:15:11 2017 +0300

----------------------------------------------------------------------
 .../snapshot/StartSnapshotOperationDiscoveryMessage.java      | 4 +++-
 .../cache/distributed/dht/GridDhtLocalPartition.java          | 7 +++++--
 .../cache/database/wal/FileWriteAheadLogManager.java          | 3 +--
 3 files changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e78441c0/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
index 700f404..18bca36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/snapshot/StartSnapshotOperationDiscoveryMessage.java
@@ -115,7 +115,9 @@ public class StartSnapshotOperationDiscoveryMessage implements DiscoveryCustomMe
         return id;
     }
 
-    /** {@inheritDoc} */
+    /**
+     *
+     */
     public IgniteUuid operationId() {
         return operationId;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e78441c0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 1d910a3..a35c168 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -141,8 +141,11 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
      * @param id Partition ID.
      * @param entryFactory Entry factory.
      */
-    @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor") GridDhtLocalPartition(GridCacheContext cctx,
-        int id, GridCacheMapEntryFactory entryFactory) {
+    @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor") GridDhtLocalPartition(
+        GridCacheContext cctx,
+        int id,
+        GridCacheMapEntryFactory entryFactory
+    ) {
         super(cctx, entryFactory, Math.max(10, GridCacheAdapter.DFLT_START_CACHE_SIZE / cctx.affinity().partitions()));
 
         this.id = id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e78441c0/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
index 1ef9384..2346f7e 100644
--- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
+++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/FileWriteAheadLogManager.java
@@ -2012,8 +2012,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             this.end = end;
             this.log = log;
 
-            int tlbSize0 = 16 * tlbSize;
-            int buffSize0 = tlbSize0 < buffSize ? tlbSize0 : buffSize;
+            int buffSize0 = Math.min(16 * tlbSize, buffSize);
 
             // Do not allocate direct buffer for iterator.
             buf = ByteBuffer.allocate(buffSize0);


[08/15] ignite git commit: ignite-gg-12163 fix for WAL history reservation was ported, test was fixed

Posted by ag...@apache.org.
ignite-gg-12163 fix for WAL history reservation was ported, test was fixed


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

Branch: refs/heads/ignite-5267
Commit: 2625dd8903b6b5ecee2b314ca6916310e65ace5e
Parents: 89d0cd2
Author: Sergey Chugunov <se...@gmail.com>
Authored: Mon May 22 14:40:25 2017 +0300
Committer: Sergey Chugunov <se...@gmail.com>
Committed: Mon May 22 14:45:17 2017 +0300

----------------------------------------------------------------------
 .../IgniteDhtPartitionHistorySuppliersMap.java  |  3 +
 .../GridCacheDatabaseSharedManager.java         | 79 +++++++++++++-------
 .../IgniteWalHistoryReservationsSelfTest.java   | 13 +++-
 3 files changed, 66 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2625dd89/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java
index 333eb97..15294c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/IgniteDhtPartitionHistorySuppliersMap.java
@@ -80,6 +80,9 @@ public class IgniteDhtPartitionHistorySuppliersMap implements Serializable {
      * @param cntr Partition counter.
      */
     public synchronized void put(UUID nodeId, int cacheId, int partId, long cntr) {
+        if (map == null)
+            map = new HashMap<>();
+
         Map<T2<Integer, Integer>, Long> nodeMap = map.get(nodeId);
 
         if (nodeMap == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/2625dd89/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
index 08857ee..0ea8b68 100755
--- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
+++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
@@ -883,32 +883,23 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 if (part.state() != GridDhtPartitionState.OWNING || part.dataStore().size() <= ggWalRebalanceThreshold)
                     continue;
 
-                for (Long cpTs : checkpointHist.checkpoints()) {
-                    try {
-                        CheckpointEntry entry = checkpointHist.entry(cpTs);
-
-                        if (!entry.cacheStates.containsKey(cacheCtx.cacheId()) ||
-                            !entry.cacheStates.get(cacheCtx.cacheId()).partitions().containsKey(part.id()))
-                            continue;
-
-                        WALPointer ptr = searchPartitionCounter(cacheCtx, part.id(), entry.checkpointTimestamp());
-
-                        if (ptr != null && cctx.wal().reserve(ptr)) {
-                            Map<Integer, T2<Long, WALPointer>> cacheMap = reservedForExchange.get(cacheCtx.cacheId());
+                CheckpointEntry cpEntry = searchCheckpointEntry(cacheCtx, part.id(), null);
 
-                            if (cacheMap == null) {
-                                cacheMap = new HashMap<>();
+                try {
+                    if (cpEntry != null && cctx.wal().reserve(cpEntry.cpMark)) {
+                        Map<Integer, T2<Long, WALPointer>> cacheMap = reservedForExchange.get(cacheCtx.cacheId());
 
-                                reservedForExchange.put(cacheCtx.cacheId(), cacheMap);
-                            }
+                        if (cacheMap == null) {
+                            cacheMap = new HashMap<>();
 
-                            cacheMap.put(part.id(), new T2<>(entry.partitionCounter(cacheCtx.cacheId(), part.id()), ptr));
+                            reservedForExchange.put(cacheCtx.cacheId(), cacheMap);
                         }
-                    }
-                    catch (IgniteCheckedException ex) {
-                        U.error(log, "Error while trying to reserve history", ex);
-                    }
 
+                        cacheMap.put(part.id(), new T2<>(cpEntry.partitionCounter(cacheCtx.cacheId(), part.id()), cpEntry.cpMark));
+                    }
+                }
+                catch (IgniteCheckedException ex) {
+                    U.error(log, "Error while trying to reserve history", ex);
                 }
             }
         }
@@ -948,7 +939,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /** {@inheritDoc} */
     @Override public boolean reserveHistoryForPreloading(int cacheId, int partId, long cntr) {
-        WALPointer ptr = searchPartitionCounter(cctx.cacheContext(cacheId), partId, cntr);
+        CheckpointEntry cpEntry = searchCheckpointEntry(cctx.cacheContext(cacheId), partId, cntr);
+
+        if (cpEntry == null)
+            return false;
+
+        WALPointer ptr = cpEntry.cpMark;
 
         if (ptr == null)
             return false;
@@ -1063,12 +1059,29 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      *
      * @param cacheCtx Cache context.
      * @param part Partition ID.
-     * @param partCntrSince Partition counter.
-     * @return WAL pointer or {@code null} if failed to search.
+     * @param partCntrSince Partition counter or {@code null} to search for minimal counter.
+     * @return Checkpoint entry or {@code null} if failed to search.
      */
-    public WALPointer searchPartitionCounter(GridCacheContext cacheCtx, int part, Long partCntrSince) {
+    @Nullable public WALPointer searchPartitionCounter(GridCacheContext cacheCtx, int part, @Nullable Long partCntrSince) {
+        CheckpointEntry entry = searchCheckpointEntry(cacheCtx, part, partCntrSince);
+
+        if (entry == null)
+            return null;
+
+        return entry.cpMark;
+    }
+
+    /**
+     * Tries to search for a WAL pointer for the given partition counter start.
+     *
+     * @param cacheCtx Cache context.
+     * @param part Partition ID.
+     * @param partCntrSince Partition counter or {@code null} to search for minimal counter.
+     * @return Checkpoint entry or {@code null} if failed to search.
+     */
+    @Nullable private CheckpointEntry searchCheckpointEntry(GridCacheContext cacheCtx, int part, @Nullable Long partCntrSince) {
         boolean hasGap = false;
-        WALPointer first = null;
+        CheckpointEntry first = null;
 
         for (Long cpTs : checkpointHist.checkpoints()) {
             try {
@@ -1077,8 +1090,18 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 Long foundCntr = entry.partitionCounter(cacheCtx.cacheId(), part);
 
                 if (foundCntr != null) {
-                    if (foundCntr <= partCntrSince) {
-                        first = entry.cpMark;
+                    if (partCntrSince == null) {
+                        if (hasGap) {
+                            first = entry;
+
+                            hasGap = false;
+                        }
+
+                        if (first == null)
+                            first = entry;
+                    }
+                    else if (foundCntr <= partCntrSince) {
+                        first = entry;
 
                         hasGap = false;
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2625dd89/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
index 82a6ac1..e241085 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
@@ -114,6 +114,17 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
 
         forceCheckpoint();
 
+        for (int k = 0; k < entryCnt; k++)
+            cache.put(k, k * 2);
+
+        forceCheckpoint();
+
+        for (int k = 0; k < entryCnt; k++)
+            cache.put(k, k);
+
+        forceCheckpoint();
+
+
         Lock lock = cache.lock(0);
 
         lock.lock();
@@ -329,7 +340,7 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
 
             assert reserved;
 
-            stopGrid(initGridCnt - 1);
+            stopGrid(Integer.toString(initGridCnt - 1), true, false);
         }
         finally {
             lock.unlock();


[11/15] ignite git commit: ignite-gg-12163 constant renamed for name consistency reasons

Posted by ag...@apache.org.
ignite-gg-12163 constant renamed for name consistency reasons


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

Branch: refs/heads/ignite-5267
Commit: a760e6e72a716f59a0110954c1efb665215cfcaf
Parents: f3bb474
Author: Sergey Chugunov <se...@gmail.com>
Authored: Mon May 22 15:56:58 2017 +0300
Committer: Sergey Chugunov <se...@gmail.com>
Committed: Mon May 22 15:56:58 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteSystemProperties.java    |  2 +-
 .../cache/database/GridCacheDatabaseSharedManager.java    |  4 ++--
 .../db/file/IgniteWalHistoryReservationsSelfTest.java     | 10 +++++-----
 3 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a760e6e7/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 3842d9b..9d368a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -613,7 +613,7 @@ public final class IgniteSystemProperties {
     /**
      * WAL rebalance threshold.
      */
-     public static final String IGNITE_WAL_REBALANCE_THRESHOLD = "IGNITE_WAL_REBALANCE_THRESHOLD";
+     public static final String IGNITE_PDS_WAL_REBALANCE_THRESHOLD = "IGNITE_PDS_WAL_REBALANCE_THRESHOLD";
 
     /** Returns true for system properties only avoiding sending sensitive information. */
     private static final IgnitePredicate<Map.Entry<String, String>> PROPS_FILTER = new IgnitePredicate<Map.Entry<String, String>>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a760e6e7/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
index 0ea8b68..c1fed9f 100755
--- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
+++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/GridCacheDatabaseSharedManager.java
@@ -129,7 +129,7 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_PARTITION_DESTROY_CHECKPOINT_DELAY;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_REBALANCE_THRESHOLD;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 
 /**
  *
@@ -151,7 +151,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /** */
     private final int ggWalRebalanceThreshold = IgniteSystemProperties.getInteger(
-        IGNITE_WAL_REBALANCE_THRESHOLD, 500_000);
+        IGNITE_PDS_WAL_REBALANCE_THRESHOLD, 500_000);
 
     /** Checkpoint lock hold count. */
     private static final ThreadLocal<Integer> CHECKPOINT_LOCK_HOLD_COUNT = new ThreadLocal<Integer>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a760e6e7/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
index e241085..73d6feb 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
@@ -39,7 +39,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_REBALANCE_THRESHOLD;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
 
 /**
  *
@@ -87,7 +87,7 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        System.clearProperty(IGNITE_WAL_REBALANCE_THRESHOLD);
+        System.clearProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD);
 
         client = false;
 
@@ -100,7 +100,7 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testReservedOnExchange() throws Exception {
-        System.setProperty(IGNITE_WAL_REBALANCE_THRESHOLD, "0");
+        System.setProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "0");
 
         final int entryCnt = 10_000;
         final int initGridCnt = 4;
@@ -196,7 +196,7 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testRemovesArePreloadedIfHistoryIsAvailable() throws Exception {
-        System.setProperty(IGNITE_WAL_REBALANCE_THRESHOLD, "0");
+        System.setProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "0");
 
         int entryCnt = 10_000;
 
@@ -287,7 +287,7 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testNodeLeftDuringExchange() throws Exception {
-        System.setProperty(IGNITE_WAL_REBALANCE_THRESHOLD, "0");
+        System.setProperty(IGNITE_PDS_WAL_REBALANCE_THRESHOLD, "0");
 
         final int entryCnt = 10_000;
         final int initGridCnt = 4;


[14/15] ignite git commit: ignite-gg-12163 fix testPageRecoveryAfterFileCorruption

Posted by ag...@apache.org.
ignite-gg-12163 fix testPageRecoveryAfterFileCorruption


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

Branch: refs/heads/ignite-5267
Commit: c95734eba2f0bba4ff8111c778267f404683eee0
Parents: a403ca8
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 17:50:11 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 17:50:11 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  5 +-
 ...entStoreRecoveryAfterFileCorruptionTest.java | 59 +++++++++++++-------
 2 files changed, 43 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c95734eb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 3f26871..76ea847 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1167,7 +1167,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheCtx.onStarted();
 
         if (log.isInfoEnabled())
-            log.info("Started cache [name=" + U.maskName(cfg.getName()) + ", memoryPolicyName=" + cfg.getMemoryPolicyName() + ", mode=" + cfg.getCacheMode() + ']');
+            log.info("Started cache [name=" + U.maskName(cfg.getName()) +
+                ", memoryPolicyName=" + cfg.getMemoryPolicyName() +
+                ", mode=" + cfg.getCacheMode() + ']'
+            );
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c95734eb/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreRecoveryAfterFileCorruptionTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreRecoveryAfterFileCorruptionTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreRecoveryAfterFileCorruptionTest.java
index fbed6ee..fd77996 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreRecoveryAfterFileCorruptionTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreRecoveryAfterFileCorruptionTest.java
@@ -22,8 +22,10 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.nio.channels.FileChannel;
 import java.util.Collection;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
@@ -48,20 +50,35 @@ import org.apache.ignite.internal.processors.cache.database.pagemem.PageMemoryIm
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+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 static org.apache.ignite.internal.processors.cache.database.wal.FileWriteAheadLogManager.IGNITE_PDS_WAL_ALWAYS_WRITE_FULL_PAGES;
+
 /**
  *
  */
 public class IgnitePersistentStoreRecoveryAfterFileCorruptionTest extends GridCommonAbstractTest {
+    /** Ip finder. */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /** Total pages. */
     private static final int totalPages = 1024;
 
+    /** Cache name. */
+    private final String cacheName = "cache";
+
+    /** Policy name. */
+    private final String policyName = "dfltMemPlc";
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        CacheConfiguration ccfg = new CacheConfiguration("partitioned");
+        CacheConfiguration ccfg = new CacheConfiguration(cacheName);
+        ccfg.setAffinity(new RendezvousAffinityFunction(true, 1));
 
         ccfg.setRebalanceMode(CacheRebalanceMode.NONE);
 
@@ -71,12 +88,12 @@ public class IgnitePersistentStoreRecoveryAfterFileCorruptionTest extends GridCo
 
         MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
 
-        memPlcCfg.setName("dfltMemPlc");
+        memPlcCfg.setName(policyName);
         memPlcCfg.setInitialSize(1024 * 1024 * 1024);
         memPlcCfg.setMaxSize(1024 * 1024 * 1024);
 
         dbCfg.setMemoryPolicies(memPlcCfg);
-        dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+        dbCfg.setDefaultMemoryPolicyName(policyName);
 
         cfg.setMemoryConfiguration(dbCfg);
 
@@ -86,12 +103,14 @@ public class IgnitePersistentStoreRecoveryAfterFileCorruptionTest extends GridCo
 
         cfg.setPersistenceConfiguration(pCfg);
 
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
+
         return cfg;
     }
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        System.setProperty(FileWriteAheadLogManager.IGNITE_PDS_WAL_ALWAYS_WRITE_FULL_PAGES, "true");
+        System.setProperty(IGNITE_PDS_WAL_ALWAYS_WRITE_FULL_PAGES, "true");
 
         stopAllGrids();
 
@@ -111,42 +130,42 @@ public class IgnitePersistentStoreRecoveryAfterFileCorruptionTest extends GridCo
      * @throws Exception if failed.
      */
     public void testPageRecoveryAfterFileCorruption() throws Exception {
-        fail(); //todo @Ed
-
         IgniteEx ig = startGrid(0);
 
-        GridCacheSharedContext<Object, Object> shared = ig.context().cache().context();
+        IgniteCache<Integer, Integer> cache = ig.cache(cacheName);
 
-        GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager)shared.database();
+        // Put for create data store and init meta page.
+        cache.put(1, 1);
 
-        IgnitePageStoreManager pageStore = shared.pageStore();
+        GridCacheSharedContext sharedCtx = ig.context().cache().context();
+
+        GridCacheDatabaseSharedManager psMgr = (GridCacheDatabaseSharedManager)sharedCtx.database();
+
+        FilePageStoreManager pageStore = (FilePageStoreManager)sharedCtx.pageStore();
 
         U.sleep(1_000);
 
         // Disable integrated checkpoint thread.
-        dbMgr.enableCheckpoints(false).get();
+        psMgr.enableCheckpoints(false).get();
 
-        PageMemory mem = shared.database().memoryPolicy(null).pageMemory();
+        PageMemory mem = sharedCtx.database().memoryPolicy(policyName).pageMemory();
 
-        int cacheId = shared.cache().cache("partitioned").context().cacheId();
+        int cacheId = sharedCtx.cache().cache(cacheName).context().cacheId();
 
         FullPageId[] pages = new FullPageId[totalPages];
 
-        for (int i = 0; i < totalPages; i++) {
-            FullPageId fullId = new FullPageId(mem.allocatePage(cacheId, 0, PageIdAllocator.FLAG_DATA), cacheId);
-
-            pages[i] = fullId;
-        }
+        for (int i = 0; i < totalPages; i++)
+            pages[i] = new FullPageId(mem.allocatePage(cacheId, 0, PageIdAllocator.FLAG_DATA), cacheId);
 
         generateWal(
             (PageMemoryImpl)mem,
-            pageStore,
-            shared.wal(),
+            sharedCtx.pageStore(),
+            sharedCtx.wal(),
             cacheId,
             pages
         );
 
-        eraseDataFromDisk((FilePageStoreManager)pageStore, cacheId, pages[0]);
+        eraseDataFromDisk(pageStore, cacheId, pages[0]);
 
         stopAllGrids();
 


[05/15] ignite git commit: ignite-gg-12163 local-specific configuration removed from test

Posted by ag...@apache.org.
ignite-gg-12163 local-specific configuration removed from test


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

Branch: refs/heads/ignite-5267
Commit: 89d0cd2e925b3cd45ec8137b3e06d83fa6c8e55e
Parents: 8b89ea5
Author: Sergey Chugunov <se...@gmail.com>
Authored: Mon May 22 12:26:49 2017 +0300
Committer: Sergey Chugunov <se...@gmail.com>
Committed: Mon May 22 12:26:49 2017 +0300

----------------------------------------------------------------------
 .../db/file/IgniteWalHistoryReservationsSelfTest.java        | 8 +-------
 1 file changed, 1 insertion(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/89d0cd2e/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
index d4d8de5..82a6ac1 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/IgniteWalHistoryReservationsSelfTest.java
@@ -73,13 +73,7 @@ public class IgniteWalHistoryReservationsSelfTest extends GridCommonAbstractTest
 
         cfg.setCacheConfiguration(ccfg1);
 
-        PersistenceConfiguration pstCfg = new PersistenceConfiguration();
-
-        pstCfg.setPersistenceStorePath("/ramdisk/ignite/db/database");
-        pstCfg.setWalStorePath("/ramdisk/ignite/db/wal");
-        pstCfg.setWalArchivePath("/ramdisk/ignite/db/wal_archive");
-
-        cfg.setPersistenceConfiguration(pstCfg);
+        cfg.setPersistenceConfiguration(new PersistenceConfiguration());
 
         return cfg;
     }


[06/15] ignite git commit: ignite-gg-12163 fix cache name walRecoveryTxLogicalRecordsTest

Posted by ag...@apache.org.
ignite-gg-12163 fix cache name walRecoveryTxLogicalRecordsTest


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

Branch: refs/heads/ignite-5267
Commit: 90717b4b7230ca5f5e0fd9318d0aa0a6f5e48064
Parents: 89d0cd2
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 12:51:28 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 12:51:28 2017 +0300

----------------------------------------------------------------------
 .../file/WalRecoveryTxLogicalRecordsTest.java   | 34 ++++++++++----------
 1 file changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/90717b4b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/WalRecoveryTxLogicalRecordsTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/WalRecoveryTxLogicalRecordsTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/WalRecoveryTxLogicalRecordsTest.java
index 8b78c21..844f2f5 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/WalRecoveryTxLogicalRecordsTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/db/file/WalRecoveryTxLogicalRecordsTest.java
@@ -525,7 +525,7 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
         try {
             pageSize = 1024;
 
-            extraCcfg = new CacheConfiguration();
+            extraCcfg = new CacheConfiguration(CACHE2_NAME);
             extraCcfg.setAffinity(new RendezvousAffinityFunction(false, PARTS));
 
             Ignite ignite = startGrid(0);
@@ -536,7 +536,7 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
             dbMgr.enableCheckpoints(false).get();
 
             IgniteCache<Integer, IndexedValue> cache1 = ignite.cache(CACHE_NAME);
-            IgniteCache<Object, Object> cache2 = ignite.cache(null);
+            IgniteCache<Object, Object> cache2 = ignite.cache(CACHE2_NAME);
 
             final int KEYS1 = 100;
 
@@ -592,7 +592,7 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
         try {
             pageSize = 1024;
             checkpointFreq = 100L;
-            extraCcfg = new CacheConfiguration();
+            extraCcfg = new CacheConfiguration(CACHE2_NAME);
             extraCcfg.setAffinity(new RendezvousAffinityFunction(false, 32));
 
             List<Integer> pages = null;
@@ -603,12 +603,12 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
                 Ignite ignite = startGrid(0);
 
                 if (pages != null) {
-                    List<Integer> curPags = allocatedPages(ignite, null);
+                    List<Integer> curPags = allocatedPages(ignite, CACHE2_NAME);
 
                     assertEquals("Iter = " + iter, pages, curPags);
                 }
 
-                final IgniteCache<Integer, Object> cache = ignite.cache(null);
+                final IgniteCache<Integer, Object> cache = ignite.cache(CACHE2_NAME);
 
                 final int ops = ThreadLocalRandom.current().nextInt(10) + 10;
 
@@ -629,7 +629,7 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
                     }
                 }, 10, "update");
 
-                pages = allocatedPages(ignite, null);
+                pages = allocatedPages(ignite, CACHE2_NAME);
 
                 Ignition.stop(ignite.name(), false); //will make checkpoint
             }
@@ -646,7 +646,7 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
     private void recoveryNoPageLost(boolean checkpoint) throws Exception {
         try {
             pageSize = 1024;
-            extraCcfg = new CacheConfiguration();
+            extraCcfg = new CacheConfiguration(CACHE2_NAME);
             extraCcfg.setAffinity(new RendezvousAffinityFunction(false, 32));
 
             List<Integer> pages = null;
@@ -665,17 +665,17 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
                     dbMgr.enableCheckpoints(false).get();
 
                 if (pages != null) {
-                    List<Integer> curPags = allocatedPages(ignite, null);
+                    List<Integer> curPags = allocatedPages(ignite, CACHE2_NAME);
 
                     assertEquals(pages, curPags);
                 }
 
-                IgniteCache<Integer, Object> cache = ignite.cache(null);
+                IgniteCache<Integer, Object> cache = ignite.cache(CACHE2_NAME);
 
                 for (int i = 0; i < 128; i++)
                     cache.put(cnt.incrementAndGet(), new byte[256 + iter * 100]);
 
-                pages = allocatedPages(ignite, null);
+                pages = allocatedPages(ignite, CACHE2_NAME);
 
                 ignite.close();
             }
@@ -722,12 +722,12 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
     public void testFreeListRecovery() throws Exception {
         try {
             pageSize = 1024;
-            extraCcfg = new CacheConfiguration();
+            extraCcfg = new CacheConfiguration(CACHE2_NAME);
 
             Ignite ignite = startGrid(0);
 
             IgniteCache<Integer, IndexedValue> cache1 = ignite.cache(CACHE_NAME);
-            IgniteCache<Object, Object> cache2 = ignite.cache(null);
+            IgniteCache<Object, Object> cache2 = ignite.cache(CACHE2_NAME);
 
             final int KEYS1 = 2048;
 
@@ -752,16 +752,16 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
             }
 
             Map<Integer, T2<Map<Integer, long[]>, int[]>> cache1_1 = getFreeListData(ignite, CACHE_NAME);
-            Map<Integer, T2<Map<Integer, long[]>, int[]>> cache2_1 = getFreeListData(ignite, null);
+            Map<Integer, T2<Map<Integer, long[]>, int[]>> cache2_1 = getFreeListData(ignite, CACHE2_NAME);
             T2<long[], Integer> rl1_1 = getReuseListData(ignite, CACHE_NAME);
-            T2<long[], Integer> rl2_1 = getReuseListData(ignite, null);
+            T2<long[], Integer> rl2_1 = getReuseListData(ignite, CACHE2_NAME);
 
             ignite.close();
 
             ignite = startGrid(0);
 
             cache1 = ignite.cache(CACHE_NAME);
-            cache2 = ignite.cache(null);
+            cache2 = ignite.cache(CACHE2_NAME);
 
             for (int i = 0; i < KEYS1; i++) {
                 cache1.get(i);
@@ -769,9 +769,9 @@ public class WalRecoveryTxLogicalRecordsTest extends GridCommonAbstractTest {
             }
 
             Map<Integer, T2<Map<Integer, long[]>, int[]>> cache1_2 = getFreeListData(ignite, CACHE_NAME);
-            Map<Integer, T2<Map<Integer, long[]>, int[]>> cache2_2 = getFreeListData(ignite, null);
+            Map<Integer, T2<Map<Integer, long[]>, int[]>> cache2_2 = getFreeListData(ignite, CACHE2_NAME);
             T2<long[], Integer> rl1_2 = getReuseListData(ignite, CACHE_NAME);
-            T2<long[], Integer> rl2_2 = getReuseListData(ignite, null);
+            T2<long[], Integer> rl2_2 = getReuseListData(ignite, CACHE2_NAME);
 
             checkEquals(cache1_1, cache1_2);
             checkEquals(cache2_1, cache2_2);


[04/15] ignite git commit: Merge branch 'ignite-gg-12163' of https://github.com/gridgain/apache-ignite into ignite-gg-12163

Posted by ag...@apache.org.
Merge branch 'ignite-gg-12163' of https://github.com/gridgain/apache-ignite into ignite-gg-12163


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

Branch: refs/heads/ignite-5267
Commit: 8b89ea50042e3e6a72199bd117fb98fb4d68072c
Parents: 4c575f0 05c6c4e
Author: Sergey Chugunov <se...@gmail.com>
Authored: Mon May 22 12:15:43 2017 +0300
Committer: Sergey Chugunov <se...@gmail.com>
Committed: Mon May 22 12:15:43 2017 +0300

----------------------------------------------------------------------
 .../db/IgniteDbPageEvictionSelfTest.java        | 14 +++++-----
 .../db/IgniteDbWholeClusterRestartSelfTest.java | 28 ++++++++++++++------
 .../db/file/PageStoreEvictionSelfTest.java      |  2 +-
 3 files changed, 28 insertions(+), 16 deletions(-)
----------------------------------------------------------------------



[09/15] ignite git commit: Merge branch 'ignite-gg-12163' of https://github.com/gridgain/apache-ignite into ignite-gg-12163

Posted by ag...@apache.org.
Merge branch 'ignite-gg-12163' of https://github.com/gridgain/apache-ignite into ignite-gg-12163


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

Branch: refs/heads/ignite-5267
Commit: f41c4fbddea4c399b4c9e2d4c5d3f182bb917b1a
Parents: 2625dd8 cd17ee0
Author: Sergey Chugunov <se...@gmail.com>
Authored: Mon May 22 14:45:49 2017 +0300
Committer: Sergey Chugunov <se...@gmail.com>
Committed: Mon May 22 14:45:49 2017 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridDhtAtomicCache.java          |  4 +--
 .../file/WalRecoveryTxLogicalRecordsTest.java   | 34 ++++++++++----------
 2 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[13/15] ignite git commit: Merge remote-tracking branch 'professional/ignite-gg-12163' into ignite-gg-12163

Posted by ag...@apache.org.
Merge remote-tracking branch 'professional/ignite-gg-12163' into ignite-gg-12163


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

Branch: refs/heads/ignite-5267
Commit: a403ca8ca465be3a8a9ddbfea63b41cecfa2e476
Parents: e78441c a760e6e
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 16:15:28 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 16:15:28 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteSystemProperties.java    |  2 +-
 .../cache/database/GridCacheDatabaseSharedManager.java    |  4 ++--
 .../db/file/IgniteWalHistoryReservationsSelfTest.java     | 10 +++++-----
 3 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[15/15] ignite git commit: ignite-gg-12163 testPartitionLossAndRecover refactoring changes

Posted by ag...@apache.org.
ignite-gg-12163 testPartitionLossAndRecover refactoring changes


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

Branch: refs/heads/ignite-5267
Commit: e31f37171dcf3cf578432d93cfffa342f4e69cdd
Parents: c95734e
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon May 22 19:09:20 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon May 22 19:09:20 2017 +0300

----------------------------------------------------------------------
 ...istentStoreCacheRebalancingAbstractTest.java | 71 +++++++++-----------
 1 file changed, 32 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e31f3717/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheRebalancingAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheRebalancingAbstractTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheRebalancingAbstractTest.java
index 108d999..6aa98af 100644
--- a/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheRebalancingAbstractTest.java
+++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheRebalancingAbstractTest.java
@@ -17,15 +17,10 @@
 
 package org.apache.ignite.cache.database;
 
-import java.io.File;
 import java.io.Serializable;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.Map;
-import java.util.Objects;
-import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -35,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -42,14 +38,11 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.PersistenceConfiguration;
-import org.apache.ignite.events.CacheRebalancingEvent;
-import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.database.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -75,6 +68,7 @@ public abstract class IgnitePersistentStoreCacheRebalancingAbstractTest extends
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         CacheConfiguration ccfg1 = cacheConfiguration(cacheName);
+        ccfg1.setPartitionLossPolicy(PartitionLossPolicy.READ_ONLY_SAFE);
         ccfg1.setBackups(1);
         ccfg1.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
@@ -99,10 +93,10 @@ public abstract class IgnitePersistentStoreCacheRebalancingAbstractTest extends
 
         cfg.setCacheConfiguration(ccfg1, ccfg2);
 
-        MemoryConfiguration dbCfg = new MemoryConfiguration();
+        MemoryConfiguration memCfg = new MemoryConfiguration();
 
-        dbCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
-        dbCfg.setPageSize(1024);
+        memCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
+        memCfg.setPageSize(1024);
 
         MemoryPolicyConfiguration memPlcCfg = new MemoryPolicyConfiguration();
 
@@ -111,16 +105,14 @@ public abstract class IgnitePersistentStoreCacheRebalancingAbstractTest extends
         memPlcCfg.setInitialSize(100 * 1024 * 1024);
         memPlcCfg.setSwapFilePath("work/swap");
 
-        dbCfg.setMemoryPolicies(memPlcCfg);
-        dbCfg.setDefaultMemoryPolicyName("dfltMemPlc");
+        memCfg.setMemoryPolicies(memPlcCfg);
+        memCfg.setDefaultMemoryPolicyName("dfltMemPlc");
 
-        cfg.setMemoryConfiguration(dbCfg);
+        cfg.setMemoryConfiguration(memCfg);
 
         cfg.setPersistenceConfiguration(new PersistenceConfiguration());
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
 
         return cfg;
     }
@@ -147,7 +139,7 @@ public abstract class IgnitePersistentStoreCacheRebalancingAbstractTest extends
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        G.stopAll(true);
+        stopAllGrids();
 
         deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
     }
@@ -323,45 +315,46 @@ public abstract class IgnitePersistentStoreCacheRebalancingAbstractTest extends
      * @throws Exception If fails.
      */
     public void testPartitionLossAndRecover() throws Exception {
-        Ignite ignite1 = G.start(getConfiguration("test1"));
-        Ignite ignite2 = G.start(getConfiguration("test2"));
-        IgniteEx ignite3 = (IgniteEx)G.start(getConfiguration("test3"));
-        IgniteEx ignite4 = (IgniteEx)G.start(getConfiguration("test4"));
+        Ignite ignite1 = startGrid(0);
+        Ignite ignite2 = startGrid(1);
+        Ignite ignite3 = startGrid(2);
+        Ignite ignite4 = startGrid(3);
 
         awaitPartitionMapExchange();
 
-        IgniteCache<Integer, Integer> cache1 = ignite1.cache(cacheName);
+        IgniteCache<String, String> cache1 = ignite1.cache(cacheName);
 
-        for (int i = 0; i < 100; i++)
-            cache1.put(i, i);
+        final int offset = 10;
 
-        ignite1.active(false);
+        for (int i = 0; i < 100; i++)
+            cache1.put(String.valueOf(i), String.valueOf(i + offset));
 
         ignite3.close();
         ignite4.close();
 
-        ignite1.active(true);
-
         awaitPartitionMapExchange();
 
         assert !ignite1.cache(cacheName).lostPartitions().isEmpty();
 
-        ignite3 = (IgniteEx)G.start(getConfiguration("test3"));
-        ignite4 = (IgniteEx)G.start(getConfiguration("test4"));
+        ignite3 = startGrid(2);
+        ignite4 = startGrid(3);
 
-        awaitPartitionMapExchange();
+        ignite1.resetLostPartitions(Collections.singletonList(cacheName));
 
-        ignite1.resetLostPartitions(Collections.singletonList(cache1.getName()));
+        IgniteCache<String, String> cache2 = ignite2.cache(cacheName);
+        IgniteCache<String, String> cache3 = ignite3.cache(cacheName);
+        IgniteCache<String, String> cache4 = ignite4.cache(cacheName);
 
-        IgniteCache<Integer, Integer> cache2 = ignite2.cache(cacheName);
-        IgniteCache<Integer, Integer> cache3 = ignite3.cache(cacheName);
-        IgniteCache<Integer, Integer> cache4 = ignite4.cache(cacheName);
+        //Thread.sleep(5_000);
 
         for (int i = 0; i < 100; i++) {
-            assert cache1.get(i).equals(i);
-            assert cache2.get(i).equals(i);
-            assert cache3.get(i).equals(i);
-            assert cache4.get(i).equals(i);
+            String key = String.valueOf(i);
+            String expected = String.valueOf(i + offset);
+
+            assertEquals(expected, cache1.get(key));
+            assertEquals(expected, cache2.get(key));
+            assertEquals(expected, cache3.get(key));
+            assertEquals(expected, cache4.get(key));
         }
     }