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/10/04 08:45:02 UTC

[19/25] ignite git commit: IGNITE-6285 Enhance persistent store paths handling - Fixes #2775.

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
new file mode 100644
index 0000000..fe7e4df
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
@@ -0,0 +1,712 @@
+/*
+ * 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.persistence.db.filename;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.regex.Pattern;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridStringLogger;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName;
+
+/**
+ * Test for new and old style persistent storage folders generation
+ */
+public class IgniteUidAsConsistentIdMigrationTest extends GridCommonAbstractTest {
+    /** Cache name for test. */
+    public static final String CACHE_NAME = "dummy";
+
+    /** Clear DB folder after each test. May be set to false for local debug */
+    private static final boolean deleteAfter = true;
+
+    /** Clear DB folder before each test. */
+    private static final boolean deleteBefore = true;
+
+    /** Fail test if delete of DB folder was not completed. */
+    private static final boolean failIfDeleteNotCompleted = true;
+
+    /** Configured consistent id. */
+    private String configuredConsistentId;
+
+    /** Logger to accumulate messages, null will cause logger won't be customized */
+    private GridStringLogger strLog;
+
+    /** Clear properties after this test run. Flag protects from failed test */
+    private boolean clearPropsAfterTest = false;
+
+    /** Place storage in temp folder for current test run. */
+    private boolean placeStorageInTemp;
+
+    /** A path to persistent store custom path for current test run. */
+    private File pstStoreCustomPath;
+
+    /** A path to persistent store WAL work custom path. */
+    private File pstWalStoreCustomPath;
+
+    /** A path to persistent store WAL archive custom path. */
+    private File pstWalArchCustomPath;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+
+        if (deleteBefore)
+            deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        if (deleteAfter)
+            deleteWorkFiles();
+
+        if (clearPropsAfterTest) {
+            System.clearProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID);
+            System.clearProperty(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        boolean ok = true;
+
+        if (pstStoreCustomPath != null)
+            ok &= deleteRecursively(pstStoreCustomPath);
+        else
+            ok &= deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+
+        if (pstWalArchCustomPath != null)
+            ok &= deleteRecursively(pstWalArchCustomPath);
+
+        if (pstWalStoreCustomPath != null)
+            ok &= deleteRecursively(pstWalStoreCustomPath);
+
+        ok &= deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false));
+
+        if (failIfDeleteNotCompleted)
+            assertTrue(ok);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (configuredConsistentId != null)
+            cfg.setConsistentId(configuredConsistentId);
+
+        final PersistentStoreConfiguration psCfg = new PersistentStoreConfiguration();
+
+        if (placeStorageInTemp) {
+            final File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+            pstStoreCustomPath = new File(tempDir, "Store");
+            pstWalStoreCustomPath = new File(tempDir, "WalStore");
+            pstWalArchCustomPath = new File(tempDir, "WalArchive");
+
+            psCfg.setPersistentStorePath(pstStoreCustomPath.getAbsolutePath());
+            psCfg.setWalStorePath(pstWalStoreCustomPath.getAbsolutePath());
+            psCfg.setWalArchivePath(pstWalArchCustomPath.getAbsolutePath());
+        }
+
+        cfg.setPersistentStoreConfiguration(psCfg);
+
+        final MemoryConfiguration memCfg = new MemoryConfiguration();
+        final MemoryPolicyConfiguration memPolCfg = new MemoryPolicyConfiguration();
+
+        memPolCfg.setMaxSize(32 * 1024 * 1024); // we don't need much memory for this test
+        memCfg.setMemoryPolicies(memPolCfg);
+        cfg.setMemoryConfiguration(memCfg);
+
+        if (strLog != null)
+            cfg.setGridLogger(strLog);
+
+        return cfg;
+    }
+
+    /**
+     * Checks start on empty PDS folder, in that case node 0 should start with random UUID.
+     *
+     * @throws Exception if failed.
+     */
+    public void testNewStyleIdIsGenerated() throws Exception {
+        final Ignite ignite = startActivateFillDataGrid(0);
+
+        //test UUID is parsable from consistent ID test
+        UUID.fromString(ignite.cluster().localNode().consistentId().toString());
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite));
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on empty PDS folder, in that case node 0 should start with random UUID.
+     *
+     * @throws Exception if failed.
+     */
+    public void testNewStyleIdIsGeneratedInCustomStorePath() throws Exception {
+        placeStorageInTemp = true;
+        final Ignite ignite = startActivateFillDataGrid(0);
+
+        //test UUID is parsable from consistent ID test
+        UUID.fromString(ignite.cluster().localNode().consistentId().toString());
+        final String subfolderName = genNewStyleSubfolderName(0, ignite);
+
+        assertDirectoryExist("binary_meta", subfolderName);
+
+        assertDirectoryExist(pstWalArchCustomPath, subfolderName);
+        assertDirectoryExist(pstWalArchCustomPath, subfolderName);
+        assertDirectoryExist(pstStoreCustomPath, subfolderName);
+
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on empty PDS folder using configured ConsistentId. We should start using this ID in compatible mode.
+     *
+     * @throws Exception if failed.
+     */
+    public void testPreconfiguredConsitentIdIsApplied() throws Exception {
+        this.configuredConsistentId = "someConfiguredConsistentId";
+        Ignite ignite = startActivateFillDataGrid(0);
+
+        assertPdsDirsDefaultExist(configuredConsistentId);
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on configured ConsistentId with same value as default, this emulate old style folder is already
+     * available. We should restart using this folder.
+     *
+     * @throws Exception if failed
+     */
+    public void testRestartOnExistingOldStyleId() throws Exception {
+        final String expDfltConsistentId = "127.0.0.1:47500";
+
+        this.configuredConsistentId = expDfltConsistentId; //this is for create old node folder
+
+        final Ignite igniteEx = startActivateGrid(0);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        igniteEx.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertPdsDirsDefaultExist(U.maskForFileName(configuredConsistentId));
+        stopGrid(0);
+
+        this.configuredConsistentId = null; //now set up grid on existing folder
+
+        final Ignite igniteRestart = startActivateGrid(0);
+
+        assertEquals(expDfltConsistentId, igniteRestart.cluster().localNode().consistentId());
+        final IgniteCache<Object, Object> cache = igniteRestart.cache(CACHE_NAME);
+
+        assertNotNull("Expected to have cache [" + CACHE_NAME + "] using [" + expDfltConsistentId + "] as PDS folder", cache);
+        final Object valFromCache = cache.get("hi");
+
+        assertNotNull("Expected to load data from cache using [" + expDfltConsistentId + "] as PDS folder", valFromCache);
+        assertTrue(expVal.equals(valFromCache));
+        stopGrid(0);
+    }
+
+    /**
+     * Start stop grid without activation should cause lock to be released and restarted node should have index 0
+     *
+     * @throws Exception if failed
+     */
+    public void testStartWithoutActivate() throws Exception {
+        //start stop grid without activate
+        startGrid(0);
+        stopGrid(0);
+
+        Ignite igniteRestart = startActivateFillDataGrid(0);
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, igniteRestart));
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on empty PDS folder, in that case node 0 should start with random UUID
+     *
+     * @throws Exception if failed
+     */
+    public void testRestartOnSameFolderWillCauseSameUuidGeneration() throws Exception {
+        final UUID uuid;
+        {
+            final Ignite ignite = startActivateFillDataGrid(0);
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite));
+
+            uuid = (UUID)ignite.cluster().localNode().consistentId();
+            stopGrid(0);
+        }
+
+        {
+            final Ignite igniteRestart = startActivateGrid(0);
+
+            assertTrue("there!".equals(igniteRestart.cache(CACHE_NAME).get("hi")));
+
+            final Object consIdRestart = igniteRestart.cluster().localNode().consistentId();
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, igniteRestart));
+            stopGrid(0);
+
+            assertEquals(uuid, consIdRestart);
+        }
+    }
+
+    /**
+     * This test starts node, activates, deactivates node, and then start second node.
+     * Expected behaviour is following: second node will join topology with separate node folder
+     *
+     * @throws Exception if failed
+     */
+    public void testStartNodeAfterDeactivate() throws Exception {
+        final UUID uuid;
+        {
+            final Ignite ignite = startActivateFillDataGrid(0);
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite));
+
+            uuid = (UUID)ignite.cluster().localNode().consistentId();
+            ignite.active(false);
+        }
+        {
+            final Ignite igniteRestart = startActivateGrid(1);
+
+            grid(0).active(true);
+            final Object consIdRestart = igniteRestart.cluster().localNode().consistentId();
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(1, igniteRestart));
+
+            stopGrid(1);
+            assertFalse(consIdRestart.equals(uuid));
+        }
+        stopGrid(0);
+        assertNodeIndexesInFolder(0, 1);
+    }
+
+    /**
+     * @param idx Index of the grid to start.
+     * @return Started and activated grid.
+     * @throws Exception If failed.
+     */
+    @NotNull private Ignite startActivateFillDataGrid(int idx) throws Exception {
+        final Ignite ignite = startActivateGrid(idx);
+
+        ignite.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+
+        return ignite;
+    }
+
+    /**
+     * Starts and activates new grid with given index.
+     *
+     * @param idx Index of the grid to start.
+     * @return Started and activated grid.
+     * @throws Exception If anything failed.
+     */
+    @NotNull private Ignite startActivateGrid(int idx) throws Exception {
+        final Ignite ignite = startGrid(idx);
+
+        ignite.active(true);
+
+        return ignite;
+    }
+
+    /**
+     * Generates folder name in new style using constant prefix and UUID
+     *
+     * @param nodeIdx expected node index to check
+     * @param ignite ignite instance
+     * @return name of storage related subfolders
+     */
+    @NotNull private String genNewStyleSubfolderName(final int nodeIdx, final Ignite ignite) {
+        final Object consistentId = ignite.cluster().localNode().consistentId();
+
+        assertTrue("For new style folders consistent ID should be UUID," +
+                " but actual class is " + (consistentId == null ? null : consistentId.getClass()),
+            consistentId instanceof UUID);
+
+        return PdsConsistentIdProcessor.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId);
+    }
+
+    /**
+     * test two nodes started at the same db root folder, second node should get index 1
+     *
+     * @throws Exception if failed
+     */
+    public void testNodeIndexIncremented() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+        final Ignite ignite1 = startGrid(1);
+
+        ignite0.active(true);
+
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+        ignite1.getOrCreateCache(CACHE_NAME).put("hi1", "there!");
+
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite0));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(1, ignite1));
+
+        stopGrid(0);
+        stopGrid(1);
+        assertNodeIndexesInFolder(0, 1);
+    }
+
+    /**
+     * Test verified that new style folder is taken always with lowest index
+     *
+     * @throws Exception if failed
+     */
+    public void testNewStyleAlwaysSmallestNodeIndexIsCreated() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+        final Ignite ignite1 = startGrid(1);
+        final Ignite ignite2 = startGrid(2);
+        final Ignite ignite3 = startGrid(3);
+
+        ignite0.active(true);
+
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+        ignite3.getOrCreateCache(CACHE_NAME).put("hi1", "there!");
+
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite0));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(1, ignite1));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(2, ignite2));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(3, ignite3));
+
+        assertNodeIndexesInFolder(0, 1, 2, 3);
+        stopAllGrids();
+
+        //this grid should take folder with index 0 as unlocked
+        final Ignite ignite4Restart = startActivateGrid(3);
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite4Restart));
+
+        assertNodeIndexesInFolder(0, 1, 2, 3);
+        stopAllGrids();
+    }
+
+    /**
+     * Test verified that new style folder is taken always with lowest index
+     *
+     * @throws Exception if failed
+     */
+    public void testNewStyleAlwaysSmallestNodeIndexIsCreatedMultithreaded() throws Exception {
+        final Ignite ignite0 = startGridsMultiThreaded(11);
+
+        ignite0.active(true);
+
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi1", "there!");
+
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite0));
+
+        assertNodeIndexesInFolder(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+        stopAllGrids();
+
+        //this grid should take folder with index 0 as unlocked
+        final Ignite ignite4Restart = startActivateGrid(4);
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite4Restart));
+        stopAllGrids();
+
+        assertNodeIndexesInFolder(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+    }
+
+    /**
+     * Test start two nodes with predefined conistent ID (emulate old fashion node). Then restart two nodes. Expected
+     * both nodes will get its own old folders
+     *
+     * @throws Exception if failed.
+     */
+    public void testStartTwoOldStyleNodes() throws Exception {
+        final String expDfltConsistentId1 = "127.0.0.1:47500";
+
+        this.configuredConsistentId = expDfltConsistentId1; //this is for create old node folder
+        final Ignite ignite = startGrid(0);
+
+        final String expDfltConsistentId2 = "127.0.0.1:47501";
+
+        this.configuredConsistentId = expDfltConsistentId2; //this is for create old node folder
+        final Ignite ignite2 = startGrid(1);
+
+        ignite.active(true);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        ignite2.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertPdsDirsDefaultExist(U.maskForFileName(expDfltConsistentId1));
+        assertPdsDirsDefaultExist(U.maskForFileName(expDfltConsistentId2));
+        stopAllGrids();
+
+        this.configuredConsistentId = null; //now set up grid on existing folder
+
+        final Ignite igniteRestart = startGrid(0);
+        final Ignite igniteRestart2 = startGrid(1);
+
+        igniteRestart2.active(true);
+
+        assertEquals(expDfltConsistentId1, igniteRestart.cluster().localNode().consistentId());
+        assertEquals(expDfltConsistentId2, igniteRestart2.cluster().localNode().consistentId());
+
+        final IgniteCache<Object, Object> cache = igniteRestart.cache(CACHE_NAME);
+
+        assertNotNull("Expected to have cache [" + CACHE_NAME + "] using [" + expDfltConsistentId1 + "] as PDS folder", cache);
+        final Object valFromCache = cache.get("hi");
+
+        assertNotNull("Expected to load data from cache using [" + expDfltConsistentId1 + "] as PDS folder", valFromCache);
+        assertTrue(expVal.equals(valFromCache));
+
+        assertNodeIndexesInFolder(); //no new style nodes should be found
+        stopGrid(0);
+    }
+
+    /**
+     * Tests compatible mode enabled by this test to start.
+     * Expected to be 2 folders and no new style folders in this case.
+     *
+     * @throws Exception if failed.
+     */
+    public void testStartOldStyleNodesByCompatibleProperty() throws Exception {
+        clearPropsAfterTest = true;
+        System.setProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, "true");
+
+        final Ignite ignite1 = startGrid(0);
+        final Ignite ignite2 = startGrid(1);
+
+        ignite1.active(true);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        ignite2.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertNodeIndexesInFolder(); // expected to have no new style folders
+
+        final Object consistentId1 = ignite1.cluster().localNode().consistentId();
+
+        assertPdsDirsDefaultExist(U.maskForFileName(consistentId1.toString()));
+        final Object consistentId2 = ignite2.cluster().localNode().consistentId();
+
+        assertPdsDirsDefaultExist(U.maskForFileName(consistentId2.toString()));
+        stopAllGrids();
+
+        System.clearProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID);
+        final Ignite igniteRestart = startGrid(0);
+        final Ignite igniteRestart2 = startGrid(1);
+
+        igniteRestart2.active(true);
+
+        assertEquals(consistentId1, igniteRestart.cluster().localNode().consistentId());
+        assertEquals(consistentId2, igniteRestart2.cluster().localNode().consistentId());
+
+        assertNodeIndexesInFolder(); //new style nodes should not be found
+        stopGrid(0);
+    }
+
+    /**
+     * Tests compatible mode enabled by this test to start, also no port is enabled.
+     * Expected to be 1 folder and no new style folders in this case.
+     *
+     * @throws Exception if failed.
+     */
+    public void testStartOldStyleNoPortsNodesByCompatibleProperty() throws Exception {
+        clearPropsAfterTest = true;
+        System.setProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, "true");
+        System.setProperty(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT, "true");
+
+        final Ignite ignite1 = startGrid(0);
+
+        ignite1.active(true);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        ignite1.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertNodeIndexesInFolder(); // expected to have no new style folders
+
+        final Object consistentId1 = ignite1.cluster().localNode().consistentId();
+
+        assertPdsDirsDefaultExist(U.maskForFileName(consistentId1.toString()));
+        stopAllGrids();
+
+        System.clearProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID);
+
+        final Ignite igniteRestart = startGrid(0);
+
+        igniteRestart.active(true);
+
+        assertEquals(consistentId1, igniteRestart.cluster().localNode().consistentId());
+
+        assertNodeIndexesInFolder(); //new style nodes should not be found
+        stopGrid(0);
+        System.clearProperty(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT);
+    }
+
+    /**
+     * Test case If there are no matching folders,
+     * but the directory contains old-style consistent IDs.
+     * Ignite should print out a warning.
+     *
+     * @throws Exception if failed.
+     */
+    public void testOldStyleNodeWithUnexpectedPort() throws Exception {
+        this.configuredConsistentId = "127.0.0.1:49999"; //emulated old-style node with not appropriate consistent ID
+        final Ignite ignite = startActivateFillDataGrid(0);
+        final IgniteCache<Object, Object> second = ignite.getOrCreateCache("second");
+
+        final int entries = 100;
+
+        for (int i = 0; i < entries; i++)
+            second.put((int)(Math.random() * entries), getClass().getName());
+
+        final String prevVerFolder = U.maskForFileName(ignite.cluster().localNode().consistentId().toString());
+        final String path = new File(new File(U.defaultWorkDirectory(), "db"), prevVerFolder).getCanonicalPath();
+
+        assertPdsDirsDefaultExist(prevVerFolder);
+        stopAllGrids();
+
+        this.configuredConsistentId = null;
+        this.strLog = new GridStringLogger();
+        startActivateGrid(0);
+        assertNodeIndexesInFolder(0); //one 0 index folder is created
+
+        final String wholeNodeLog = strLog.toString();
+        stopAllGrids();
+
+        String foundWarning = null;
+        for (String line : wholeNodeLog.split("\n")) {
+            if (line.contains("There is other non-empty storage folder under storage base directory")) {
+                foundWarning = line;
+                break;
+            }
+        }
+
+        if (foundWarning != null)
+            log.info("\nWARNING generated successfully [\n" + foundWarning + "\n]");
+
+        assertTrue("Expected to warn user on existence of old style path",
+            foundWarning != null);
+
+        assertTrue("Expected to warn user on existence of old style path [" + path + "]",
+            foundWarning.contains(path));
+
+        assertTrue("Expected to print some size for [" + path + "]",
+            Pattern.compile(" [0-9]* bytes").matcher(foundWarning).find());
+
+        strLog = null;
+        startActivateGrid(0);
+        assertNodeIndexesInFolder(0); //one 0 index folder is created
+        stopAllGrids();
+    }
+
+    /**
+     * @param indexes expected new style node indexes in folders
+     * @throws IgniteCheckedException if failed
+     */
+    private void assertNodeIndexesInFolder(Integer... indexes) throws IgniteCheckedException {
+        assertEquals(new TreeSet<>(Arrays.asList(indexes)), getAllNodeIndexesInFolder());
+    }
+
+    /**
+     * @return set of all indexes of nodes found in work folder
+     * @throws IgniteCheckedException if failed.
+     */
+    @NotNull private Set<Integer> getAllNodeIndexesInFolder() throws IgniteCheckedException {
+        final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER);
+        final Set<Integer> indexes = new TreeSet<>();
+        final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER);
+
+        for (File file : files) {
+            final PdsConsistentIdProcessor.FolderCandidate uid = parseSubFolderName(file, log);
+
+            if (uid != null)
+                indexes.add(uid.nodeIndex());
+        }
+
+        return indexes;
+    }
+
+    /**
+     * Checks existence of all storage-related directories
+     *
+     * @param subDirName sub directories name expected
+     * @throws IgniteCheckedException if IO error occur
+     */
+    private void assertPdsDirsDefaultExist(String subDirName) throws IgniteCheckedException {
+        assertDirectoryExist("binary_meta", subDirName);
+        assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_STORE_PATH, subDirName);
+        assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName);
+        assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName);
+    }
+
+    /**
+     * Checks one folder existence.
+     *
+     * @param subFolderNames sub folders chain array to touch.
+     * @throws IgniteCheckedException if IO error occur.
+     */
+    private void assertDirectoryExist(String... subFolderNames) throws IgniteCheckedException {
+        final File curFolder = new File(U.defaultWorkDirectory());
+
+        assertDirectoryExist(curFolder, subFolderNames);
+    }
+
+
+    /**
+     * Checks one folder existence.
+     *
+     * @param workFolder current work folder.
+     * @param subFolderNames sub folders chain array to touch.
+     * @throws IgniteCheckedException if IO error occur.
+     */
+    private void assertDirectoryExist(final File workFolder, String... subFolderNames) throws IgniteCheckedException {
+        File curFolder = workFolder;
+
+        for (String name : subFolderNames) {
+            curFolder = new File(curFolder, name);
+        }
+
+        final String path;
+
+        try {
+            path = curFolder.getCanonicalPath();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to convert path: [" + curFolder.getAbsolutePath() + "]", e);
+        }
+        assertTrue("Directory " + Arrays.asList(subFolderNames).toString()
+            + " is expected to exist [" + path + "]", curFolder.exists() && curFolder.isDirectory());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index c160f60..bf8cd85 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Random;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -68,6 +69,7 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingPageIO;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -575,9 +577,11 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
             for (int i = 0; i < 100; i++)
                 cache.put(i, new IndexedObject(i));
 
+            final Object consistentId = ignite.cluster().localNode().consistentId();
+
             stopGrid(1);
 
-            final File cacheDir = cacheDir("partitioned", ignite.context().discovery().consistentId().toString());
+            final File cacheDir = cacheDir("partitioned", consistentId.toString());
 
             final boolean renamed = cacheDir.renameTo(new File(cacheDir.getParent(), "cache-partitioned0"));
 
@@ -605,14 +609,15 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
      * @return Cache dir.
      * @throws IgniteCheckedException If fail.
      */
-    private File cacheDir(final String cacheName, String consId) throws IgniteCheckedException {
-        consId = consId.replaceAll("[\\.:]", "_");
+    private File cacheDir(final String cacheName, final String consId) throws IgniteCheckedException {
+        final String subfolderName
+            = PdsConsistentIdProcessor.genNewStyleSubfolderName(0, UUID.fromString(consId));
 
         final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
 
         assert dbDir.exists();
 
-        final File consIdDir = new File(dbDir.getAbsolutePath(), consId);
+        final File consIdDir = new File(dbDir.getAbsolutePath(), subfolderName);
 
         assert consIdDir.exists();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index e2f58bd..b9c60b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -30,6 +30,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -64,14 +65,17 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
@@ -87,9 +91,6 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     /** additional cache for testing different combinations of types in WAL */
     private static final String CACHE_ADDL_NAME = "cache1";
 
-    /** Fill wal with some data before iterating. Should be true for non local run */
-    private static final boolean fillWalBeforeTest = true;
-
     /** Delete DB dir before test. */
     private static final boolean deleteBefore = true;
 
@@ -152,7 +153,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
+    @Override protected void beforeTest() throws Exception {
         stopAllGrids();
 
         if (deleteBefore)
@@ -171,8 +172,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        if (fillWalBeforeTest)
-            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**
@@ -181,27 +181,23 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     public void testFillWalAndReadRecords() throws Exception {
         final int cacheObjectsToWrite = 10000;
 
-        final String consistentId;
-        if (fillWalBeforeTest) {
-            final Ignite ignite0 = startGrid("node0");
+        final Ignite ignite0 = startGrid("node0");
 
-            ignite0.active(true);
+        ignite0.active(true);
 
-            consistentId = U.maskForFileName(ignite0.cluster().localNode().consistentId().toString());
+        final Serializable consistentId = (Serializable)ignite0.cluster().localNode().consistentId();
+        final String subfolderName = genNewStyleSubfolderName(0, (UUID)consistentId);
 
-            putDummyRecords(ignite0, cacheObjectsToWrite);
+        putDummyRecords(ignite0, cacheObjectsToWrite);
 
-            stopGrid("node0");
-        }
-        else
-            consistentId = "127_0_0_1_47500";
+        stopGrid("node0");
 
         final String workDir = U.defaultWorkDirectory();
         final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false);
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
 
-        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, WAL_SEGMENTS);
+        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, subfolderName, WAL_SEGMENTS);
         final WALIterator it = mockItFactory.iterator(wal, walArchive);
         final int cntUsingMockIter = iterateAndCount(it, false);
 
@@ -209,11 +205,11 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         assert cntUsingMockIter > 0;
         assert cntUsingMockIter > cacheObjectsToWrite;
 
-        final File walArchiveDirWithConsistentId = new File(walArchive, consistentId);
-        final File walWorkDirWithConsistentId = new File(wal, consistentId);
+        final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName);
+        final File walWorkDirWithConsistentId = new File(wal, subfolderName);
 
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
-        final File binaryMetaWithConsId = new File(binaryMeta, consistentId);
+        final File binaryMetaWithConsId = new File(binaryMeta, subfolderName);
         final File marshaller = U.resolveWorkDirectory(workDir, "marshaller", false);
 
         final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, PAGE_SIZE, binaryMetaWithConsId, marshaller);
@@ -304,7 +300,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final IgniteEvents evts = ignite.events();
 
         if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED))
-            return; //nothing to test
+            assertTrue("nothing to test", false);
 
         evts.localListen(new IgnitePredicate<Event>() {
             @Override public boolean apply(Event e) {
@@ -428,29 +424,23 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final int cntEntries = 1000;
         final int txCnt = 100;
 
-        final Map<Object, Object> ctrlMap = new HashMap<>();
-        final String consistentId;
-        if (fillWalBeforeTest) {
-            final Ignite ignite0 = startGrid("node0");
-
-            ignite0.active(true);
-
-            final IgniteCache<Object, Object> entries = txPutDummyRecords(ignite0, cntEntries, txCnt);
+        final Ignite ignite0 = startGrid("node0");
 
-            for (Cache.Entry<Object, Object> next : entries) {
-                ctrlMap.put(next.getKey(), next.getValue());
-            }
+        ignite0.active(true);
 
-            consistentId = U.maskForFileName(ignite0.cluster().localNode().consistentId().toString());
+        final IgniteCache<Object, Object> entries = txPutDummyRecords(ignite0, cntEntries, txCnt);
 
-            stopGrid("node0");
+        final Map<Object, Object> ctrlMap = new HashMap<>();
+        for (Cache.Entry<Object, Object> next : entries) {
+            ctrlMap.put(next.getKey(), next.getValue());
         }
-        else
-            consistentId = "127_0_0_1_47500";
+
+        final String subfolderName = genDbSubfolderName(ignite0, 0);
+        stopGrid("node0");
 
         final String workDir = U.defaultWorkDirectory();
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
-        final File binaryMetaWithConsId = new File(binaryMeta, consistentId);
+        final File binaryMetaWithConsId = new File(binaryMeta, subfolderName);
         final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false);
 
         final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log,
@@ -474,17 +464,28 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 }
             }
         };
-        scanIterateAndCount(factory, workDir, consistentId, cntEntries, txCnt, objConsumer, null);
+        scanIterateAndCount(factory, workDir, subfolderName, cntEntries, txCnt, objConsumer, null);
 
         assert ctrlMap.isEmpty() : " Control Map is not empty after reading entries " + ctrlMap;
     }
 
     /**
+     * Generates DB subfolder name for provided node index (local) and UUID (consistent ID)
+     *
+     * @param ignite ignite instance.
+     * @param nodeIdx node index.
+     * @return folder file name
+     */
+    @NotNull private String genDbSubfolderName(Ignite ignite, int nodeIdx) {
+        return genNewStyleSubfolderName(nodeIdx, (UUID)ignite.cluster().localNode().consistentId());
+    }
+
+    /**
      * Scan WAL and WAL archive for logical records and its entries.
      *
      * @param factory WAL iterator factory.
      * @param workDir Ignite work directory.
-     * @param consistentId consistent ID.
+     * @param subfolderName DB subfolder name based on consistent ID.
      * @param expCntEntries minimum expected entries count to find.
      * @param expTxCnt minimum expected transaction count to find.
      * @param objConsumer object handler, called for each object found in logical data records.
@@ -494,7 +495,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     private void scanIterateAndCount(
         final IgniteWalIteratorFactory factory,
         final String workDir,
-        final String consistentId,
+        final String subfolderName,
         final int expCntEntries,
         final int expTxCnt,
         @Nullable final BiConsumer<Object, Object> objConsumer,
@@ -504,11 +505,10 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
 
-        final File walArchiveDirWithConsistentId = new File(walArchive, consistentId);
+        final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName);
 
         final File[] files = walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
-
-        assert files != null : "Can't iterate over files [" + walArchiveDirWithConsistentId + "] Directory is N/A";
+        A.notNull(files, "Can't iterate over files [" + walArchiveDirWithConsistentId + "] Directory is N/A");
         final WALIterator iter = factory.iteratorArchiveFiles(files);
 
         final Map<GridCacheVersion, Integer> cntArch = iterateAndCountDataRecord(iter, objConsumer, dataRecordHnd);
@@ -520,8 +520,8 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
 
         log.info("Total tx found loaded using archive directory (file-by-file): " + txCntObservedArch);
 
-        final File walWorkDirWithConsistentId = new File(wal, consistentId);
-        final File[] workFiles = walWorkDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
+        final File walWorkDirWithNodeSubDir = new File(wal, subfolderName);
+        final File[] workFiles = walWorkDirWithNodeSubDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
 
         final WALIterator tuples = factory.iteratorWorkFiles(workFiles);
         final Map<GridCacheVersion, Integer> cntWork = iterateAndCountDataRecord(tuples, objConsumer, dataRecordHnd);
@@ -541,71 +541,66 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      */
     public void testFillWalWithDifferentTypes() throws Exception {
         int cntEntries = 0;
-        final String consistentId;
 
         final Map<Object, Object> ctrlMap = new HashMap<>();
         final Map<Object, Object> ctrlMapForBinaryObjects = new HashMap<>();
         final Collection<String> ctrlStringsToSearch = new HashSet<>();
         final Collection<String> ctrlStringsForBinaryObjSearch = new HashSet<>();
-        if (fillWalBeforeTest) {
-            final Ignite ignite0 = startGrid("node0");
-            ignite0.active(true);
-
-            final IgniteCache<Object, Object> addlCache = ignite0.getOrCreateCache(CACHE_ADDL_NAME);
-            addlCache.put("1", "2");
-            addlCache.put(1, 2);
-            addlCache.put(1L, 2L);
-            addlCache.put(TestEnum.A, "Enum_As_Key");
-            addlCache.put("Enum_As_Value", TestEnum.B);
-            addlCache.put(TestEnum.C, TestEnum.C);
-
-            addlCache.put("Serializable", new TestSerializable(42));
-            addlCache.put(new TestSerializable(42), "Serializable_As_Key");
-            addlCache.put("Externalizable", new TestExternalizable(42));
-            addlCache.put(new TestExternalizable(42), "Externalizable_As_Key");
-            addlCache.put(292, new IndexedObject(292));
-
-            final String search1 = "SomeUnexpectedStringValueAsKeyToSearch";
-            ctrlStringsToSearch.add(search1);
-            ctrlStringsForBinaryObjSearch.add(search1);
-            addlCache.put(search1, "SearchKey");
-
-            String search2 = "SomeTestStringContainerToBePrintedLongLine";
-            final TestStringContainerToBePrinted val = new TestStringContainerToBePrinted(search2);
-            ctrlStringsToSearch.add(val.toString()); //will validate original toString() was called
-            ctrlStringsForBinaryObjSearch.add(search2);
-            addlCache.put("SearchValue", val);
-
-            String search3 = "SomeTestStringContainerToBePrintedLongLine2";
-            final TestStringContainerToBePrinted key = new TestStringContainerToBePrinted(search3);
-            ctrlStringsToSearch.add(key.toString()); //will validate original toString() was called
-            ctrlStringsForBinaryObjSearch.add(search3); //validate only string itself
-            addlCache.put(key, "SearchKey");
-
-            cntEntries = addlCache.size();
-            for (Cache.Entry<Object, Object> next : addlCache) {
-                ctrlMap.put(next.getKey(), next.getValue());
-            }
+        final Ignite ignite0 = startGrid("node0");
+        ignite0.active(true);
+
+        final IgniteCache<Object, Object> addlCache = ignite0.getOrCreateCache(CACHE_ADDL_NAME);
+        addlCache.put("1", "2");
+        addlCache.put(1, 2);
+        addlCache.put(1L, 2L);
+        addlCache.put(TestEnum.A, "Enum_As_Key");
+        addlCache.put("Enum_As_Value", TestEnum.B);
+        addlCache.put(TestEnum.C, TestEnum.C);
+
+        addlCache.put("Serializable", new TestSerializable(42));
+        addlCache.put(new TestSerializable(42), "Serializable_As_Key");
+        addlCache.put("Externalizable", new TestExternalizable(42));
+        addlCache.put(new TestExternalizable(42), "Externalizable_As_Key");
+        addlCache.put(292, new IndexedObject(292));
+
+        final String search1 = "SomeUnexpectedStringValueAsKeyToSearch";
+        ctrlStringsToSearch.add(search1);
+        ctrlStringsForBinaryObjSearch.add(search1);
+        addlCache.put(search1, "SearchKey");
+
+        String search2 = "SomeTestStringContainerToBePrintedLongLine";
+        final TestStringContainerToBePrinted val = new TestStringContainerToBePrinted(search2);
+        ctrlStringsToSearch.add(val.toString()); //will validate original toString() was called
+        ctrlStringsForBinaryObjSearch.add(search2);
+        addlCache.put("SearchValue", val);
+
+        String search3 = "SomeTestStringContainerToBePrintedLongLine2";
+        final TestStringContainerToBePrinted key = new TestStringContainerToBePrinted(search3);
+        ctrlStringsToSearch.add(key.toString()); //will validate original toString() was called
+        ctrlStringsForBinaryObjSearch.add(search3); //validate only string itself
+        addlCache.put(key, "SearchKey");
+
+        cntEntries = addlCache.size();
+        for (Cache.Entry<Object, Object> next : addlCache) {
+            ctrlMap.put(next.getKey(), next.getValue());
+        }
 
             for (Cache.Entry<Object, Object> next : addlCache) {
                 ctrlMapForBinaryObjects.put(next.getKey(), next.getValue());
             }
 
-            consistentId = U.maskForFileName(ignite0.cluster().localNode().consistentId().toString());
+        final String subfolderName = genDbSubfolderName(ignite0, 0);
 
-            stopGrid("node0");
-        }
-        else
-            consistentId = "127_0_0_1_47500";
+        stopGrid("node0");
 
         final String workDir = U.defaultWorkDirectory();
 
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
-        final File binaryMetaWithConsId = new File(binaryMeta, consistentId);
+        final File binaryMetaWithNodeSubfolder = new File(binaryMeta, subfolderName);
         final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false);
 
         final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, PAGE_SIZE,
-            binaryMetaWithConsId,
+            binaryMetaWithNodeSubfolder,
             marshallerMapping);
         final BiConsumer<Object, Object> objConsumer = new BiConsumer<Object, Object>() {
             @Override public void accept(Object key, Object val) {
@@ -634,7 +629,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 }
             }
         };
-        scanIterateAndCount(factory, workDir, consistentId, cntEntries, 0, objConsumer, toStrChecker);
+        scanIterateAndCount(factory, workDir, subfolderName, cntEntries, 0, objConsumer, toStrChecker);
 
         assert ctrlMap.isEmpty() : " Control Map is not empty after reading entries: " + ctrlMap;
         assert ctrlStringsToSearch.isEmpty() : " Control Map for strings in entries is not empty after" +
@@ -642,7 +637,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
 
         //Validate same WAL log with flag binary objects only
         final IgniteWalIteratorFactory keepBinFactory = new IgniteWalIteratorFactory(log, PAGE_SIZE,
-            binaryMetaWithConsId,
+            binaryMetaWithNodeSubfolder,
             marshallerMapping,
             true);
         final BiConsumer<Object, Object> binObjConsumer = new BiConsumer<Object, Object>() {
@@ -693,7 +688,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
             }
         };
 
-        final Consumer<DataRecord> binObjToStringChecker = new Consumer<DataRecord>() {
+        final Consumer<DataRecord> binObjToStrChecker = new Consumer<DataRecord>() {
             @Override public void accept(DataRecord record) {
                 String strRepresentation = record.toString();
                 for (Iterator<String> iter = ctrlStringsForBinaryObjSearch.iterator(); iter.hasNext(); ) {
@@ -705,7 +700,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 }
             }
         };
-        scanIterateAndCount(keepBinFactory, workDir, consistentId, cntEntries, 0, binObjConsumer, binObjToStringChecker);
+        scanIterateAndCount(keepBinFactory, workDir, subfolderName, cntEntries, 0, binObjConsumer, binObjToStrChecker);
 
         assert ctrlMapForBinaryObjects.isEmpty() : " Control Map is not empty after reading entries: " + ctrlMapForBinaryObjects;
         assert ctrlStringsForBinaryObjSearch.isEmpty() : " Control Map for strings in entries is not empty after" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
index 4030e53..05636eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.db.wal.reader;
 
 import java.io.File;
+import java.io.Serializable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -29,6 +30,8 @@ import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.jetbrains.annotations.Nullable;
 import org.mockito.Mockito;
@@ -47,25 +50,34 @@ public class MockWalIteratorFactory {
     private final int pageSize;
 
     /** Consistent node id. */
-    private final String consistentId;
+    private final Serializable consistentId;
+
+    /** DB storage subfolder based node index and consistent node ID. */
+    private String subfolderName;
 
     /** Segments count in work dir. */
     private int segments;
 
+
     /**
      * Creates factory
      * @param log Logger.
      * @param pageSize Page size.
      * @param consistentId Consistent id.
+     * @param subfolderName
      * @param segments Segments.
      */
-    public MockWalIteratorFactory(@Nullable IgniteLogger log, int pageSize, String consistentId, int segments) {
+    public MockWalIteratorFactory(@Nullable IgniteLogger log,
+        int pageSize,
+        Serializable consistentId,
+        String subfolderName,
+        int segments) {
         this.log = log == null ? Mockito.mock(IgniteLogger.class) : log;
         this.pageSize = pageSize;
         this.consistentId = consistentId;
+        this.subfolderName = subfolderName;
         this.segments = segments;
     }
-
     /**
      * Creates iterator
      * @param wal WAL directory without node consistent id
@@ -93,10 +105,13 @@ public class MockWalIteratorFactory {
 
         when(ctx.config()).thenReturn(cfg);
         when(ctx.clientNode()).thenReturn(false);
+        when(ctx.pdsFolderResolver()).thenReturn(new PdsFoldersResolver() {
+            @Override public PdsFolderSettings resolveFolders() {
+                return new PdsFolderSettings(new File("."), subfolderName, consistentId, null, false);
+            }
+        });
 
         final GridDiscoveryManager disco = Mockito.mock(GridDiscoveryManager.class);
-
-        when(disco.consistentId()).thenReturn(consistentId);
         when(ctx.discovery()).thenReturn(disco);
 
         final IgniteWriteAheadLogManager mgr = new FileWriteAheadLogManager(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index 29f7255..d92d848 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageE
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.filename.IgniteUidAsConsistentIdMigrationTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFailoverTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest;
@@ -78,6 +79,9 @@ public class IgnitePdsTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgnitePdsExchangeDuringCheckpointTest.class);
 
+        // new style folders with generated consistent ID test
+        suite.addTestSuite(IgniteUidAsConsistentIdMigrationTest.class);
+
         suite.addTestSuite(IgniteWalSerializerVersionTest.class);
 
         return suite;