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/10/06 15:20:08 UTC

[4/4] ignite git commit: IGNITE-5849 Introduced meta store

IGNITE-5849 Introduced meta store


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

Branch: refs/heads/ignite-5849-2
Commit: 61a4de80d1ad5b59fa1c07b2d7e86ccb8ee6a3aa
Parents: 2410f07
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Oct 6 18:19:02 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 6 18:19:02 2017 +0300

----------------------------------------------------------------------
 .../pagemem/store/IgnitePageStoreManager.java   |    5 +
 .../delta/DataPageInsertFragmentRecord.java     |    5 +-
 .../wal/record/delta/DataPageInsertRecord.java  |    5 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |    5 +-
 .../delta/DataPageSetFreeListPageRecord.java    |    5 +-
 .../wal/record/delta/DataPageUpdateRecord.java  |    5 +-
 .../processors/cache/GridCacheProcessor.java    |    3 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |    6 +-
 .../cache/persistence/CacheDataRow.java         |    6 +-
 .../GridCacheDatabaseSharedManager.java         |  142 +-
 .../persistence/GridCacheOffheapManager.java    |   20 +-
 .../IgniteCacheDatabaseSharedManager.java       |   46 +-
 .../cache/persistence/IndexStorage.java         |   51 +
 .../cache/persistence/IndexStorageImpl.java     |  420 ++++++
 .../cache/persistence/MemoryMetricsImpl.java    |    2 +-
 .../processors/cache/persistence/MetaStore.java |   51 -
 .../cache/persistence/MetadataStorage.java      |  420 ------
 .../processors/cache/persistence/Storable.java  |   38 +
 .../persistence/file/FilePageStoreManager.java  |   30 +-
 .../persistence/freelist/AbstractFreeList.java  |  591 +++++++++
 .../persistence/freelist/CacheFreeListImpl.java |   50 +
 .../cache/persistence/freelist/FreeList.java    |    8 +-
 .../persistence/freelist/FreeListImpl.java      |  605 ---------
 .../cache/persistence/freelist/PagesList.java   |    8 +-
 .../persistence/metastorage/MetaStorage.java    |  338 +++++
 .../metastorage/MetastorageDataRow.java         |   92 ++
 .../metastorage/MetastorageRowStore.java        |   97 ++
 .../metastorage/MetastorageSearchRow.java       |   38 +
 .../metastorage/MetastorageTree.java            |  266 ++++
 .../metastorage/MetsatorageSearchRowImpl.java   |   55 +
 .../persistence/tree/io/AbstractDataPageIO.java | 1244 ++++++++++++++++++
 .../cache/persistence/tree/io/DataPageIO.java   | 1201 +----------------
 .../cache/persistence/tree/io/PageIO.java       |   26 +-
 .../persistence/tree/io/SimpleDataPageIO.java   |  127 ++
 .../db/wal/IgniteWalRecoveryTest.java           |  173 +++
 .../db/wal/WalRecoveryTxLogicalRecordsTest.java |    8 +-
 .../pagemem/IndexStoragePageMemoryImplTest.java |  104 ++
 .../MetadataStoragePageMemoryImplTest.java      |  104 --
 .../pagemem/NoOpPageStoreManager.java           |    5 +
 .../database/CacheFreeListImplSelfTest.java     |  565 ++++++++
 .../database/FreeListImplSelfTest.java          |  565 --------
 .../database/IndexStorageSelfTest.java          |  171 +++
 .../database/MetadataStorageSelfTest.java       |  171 ---
 .../ignite/testsuites/IgniteBasicTestSuite.java |    8 +-
 .../ignite/testsuites/IgnitePdsTestSuite.java   |    4 +-
 45 files changed, 4722 insertions(+), 3167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index 64c5927..6802a3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -52,6 +52,11 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
         throws IgniteCheckedException;
 
     /**
+     * Initializes disk cache store structures.
+     */
+    public void initializeForMetastorage() throws IgniteCheckedException;
+
+    /**
      * Callback called when a cache is stopping. After this callback is invoked, no data associated with
      * the given cache will be stored on disk.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
index e07c388..5324d56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -54,7 +55,7 @@ public class DataPageInsertFragmentRecord extends PageDeltaRecord {
 
     /** {@inheritDoc} */
     @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+        AbstractDataPageIO io = PageIO.getPageIO(pageAddr);
 
         io.addRowFragment(pageAddr, payload, lastLink, pageMem.pageSize());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
index f315058..2c9a8e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
@@ -55,7 +56,7 @@ public class DataPageInsertRecord extends PageDeltaRecord {
     @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         assert payload != null;
 
-        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+        AbstractDataPageIO io = PageIO.getPageIO(pageAddr);
 
         io.addRow(pageAddr, payload, pageMem.pageSize());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
index 484ec87..f7776be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
@@ -50,7 +51,7 @@ public class DataPageRemoveRecord extends PageDeltaRecord {
     /** {@inheritDoc} */
     @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+        AbstractDataPageIO io = PageIO.getPageIO(pageAddr);
 
         io.removeRow(pageAddr, itemId, pageMem.pageSize());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
index 0ade484..e679611 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE;
@@ -51,7 +52,7 @@ public class DataPageSetFreeListPageRecord extends PageDeltaRecord {
 
     /** {@inheritDoc} */
     @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+        AbstractDataPageIO io = PageIO.getPageIO(pageAddr);
 
         io.setFreeListPageId(pageAddr, freeListPage);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
index 8ea2981..ed469a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageUpdateRecord.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
@@ -68,7 +69,7 @@ public class DataPageUpdateRecord extends PageDeltaRecord {
     @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         assert payload != null;
 
-        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+        AbstractDataPageIO io = PageIO.getPageIO(pageAddr);
 
         io.updateRow(pageAddr, itemId, pageMem.pageSize(), payload, null, 0);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/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 f3759e0..9dec24b 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
@@ -91,6 +91,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDataba
 import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
+import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
@@ -3762,7 +3763,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (ccfg != null) {
             cloneCheckSerializable(ccfg);
 
-            if (desc != null) {
+            if (desc != null || MetaStorage.METASTORAGE_CACHE_NAME.equals(cacheName)) {
                 if (failIfExists) {
                     throw new CacheExistsException("Failed to start cache " +
                         "(a cache with the same name is already started): " + cacheName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index ba6f7d0..b9404ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -41,8 +41,8 @@ import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapt
 import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.RowStore;
-import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore;
@@ -1179,12 +1179,12 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             // Use grp.sharedGroup() flag since it is possible cacheId is not yet set here.
             boolean sizeWithCacheId = grp.sharedGroup();
 
-            int oldLen = FreeListImpl.getRowSize(oldRow, sizeWithCacheId);
+            int oldLen = DataPageIO.getRowSize(oldRow, sizeWithCacheId);
 
             if (oldLen > updateValSizeThreshold)
                 return false;
 
-            int newLen = FreeListImpl.getRowSize(dataRow, sizeWithCacheId);
+            int newLen = DataPageIO.getRowSize(dataRow, sizeWithCacheId);
 
             return oldLen == newLen;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java
index 57aeaef..4604826 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRow.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 /**
  * Cache data row.
  */
-public interface CacheDataRow extends CacheSearchRow {
+public interface CacheDataRow extends CacheSearchRow, Storable {
     /**
      * @return Cache value.
      */
@@ -43,12 +43,12 @@ public interface CacheDataRow extends CacheSearchRow {
     /**
      * @return Partition for this key.
      */
-    public int partition();
+    @Override public int partition();
 
     /**
      * @param link Link for this row.
      */
-    public void link(long link);
+    @Override public void link(long link);
 
     /**
      * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 9a2e028..ae27f51 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -77,6 +77,8 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
+import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
+import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -108,6 +110,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl;
@@ -148,6 +151,7 @@ import org.jetbrains.annotations.Nullable;
 import static java.nio.file.StandardOpenOption.READ;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_SKIP_CRC;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PDS_WAL_REBALANCE_THRESHOLD;
+import static org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage.METASTORAGE_CACHE_ID;
 
 /**
  *
@@ -157,6 +161,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** */
     public static final String IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC = "IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC";
 
+    /** MemoryPolicyConfiguration name reserved for meta store. */
+    private static final String METASTORE_MEMORY_POLICY_NAME = "metastoreMemPlc";
+
     /** Default checkpointing page buffer size (may be adjusted by Ignite). */
     public static final Long DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE = 256L * 1024 * 1024;
 
@@ -312,6 +319,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** Number of pages in current checkpoint. */
     private volatile int currCheckpointPagesCnt;
 
+    /** */
+    private MetaStorage metaStorage;
+
     /**
      * @param ctx Kernal context.
      */
@@ -356,6 +366,31 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /** {@inheritDoc} */
+    @Override protected void initPageMemoryPolicies(MemoryConfiguration memCfg) throws IgniteCheckedException {
+        super.initPageMemoryPolicies(memCfg);
+
+        addMemoryPolicy(
+            memCfg,
+            createStoreMemoryPolicy(memCfg),
+            METASTORE_MEMORY_POLICY_NAME
+        );
+    }
+
+    /**
+     * @param memCfg Memory configuration.
+     * @return Memoty polict configuration.
+     */
+    private MemoryPolicyConfiguration createStoreMemoryPolicy(MemoryConfiguration memCfg) {
+        MemoryPolicyConfiguration cfg = new MemoryPolicyConfiguration();
+
+        cfg.setName(METASTORE_MEMORY_POLICY_NAME);
+        cfg.setInitialSize(memCfg.getSystemCacheInitialSize());
+        cfg.setMaxSize(memCfg.getSystemCacheMaxSize());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
         super.start0();
 
@@ -393,6 +428,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), kernalCtx, log);
 
             persStoreMetrics.wal(cctx.wal());
+
+            // Here we can get data from metastorage
         }
     }
 
@@ -484,6 +521,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         super.onActivate(ctx);
+
+        if (!cctx.localNode().isClient()) {
+            cctx.pageStore().initializeForMetastorage();
+
+            metaStorage = new MetaStorage(cctx.wal(), memPlcMap.get(METASTORE_MEMORY_POLICY_NAME),
+                (MemoryMetricsImpl)memMetricsMap.get(METASTORE_MEMORY_POLICY_NAME));
+        }
     }
 
     /** {@inheritDoc} */
@@ -594,6 +638,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             // This method should return a pointer to the last valid record in the WAL.
             WALPointer restore = restoreMemory(status);
 
+            metaStorage.init(this);
+
             cctx.wal().resumeLogging(restore);
 
             cctx.wal().log(new MemoryRecoveryRecord(U.currentTimeMillis()));
@@ -606,6 +652,50 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
     }
 
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void getMetastoreData() throws IgniteCheckedException {
+        try {
+            MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
+
+            MemoryPolicyConfiguration plcCfg = createStoreMemoryPolicy(memCfg);
+
+            File allocPath = buildAllocPath(plcCfg);
+
+            DirectMemoryProvider memProvider = allocPath == null ?
+                new UnsafeMemoryProvider(log) :
+                new MappedFileMemoryProvider(
+                    log,
+                    allocPath);
+
+            MemoryMetricsImpl memMetrics = new MemoryMetricsImpl(plcCfg);
+
+            PageMemoryEx storePageMem = (PageMemoryEx)createPageMemory(memProvider, memCfg, plcCfg, memMetrics);
+
+            MemoryPolicy storeMemPlc = new MemoryPolicy(storePageMem, plcCfg, memMetrics, createPageEvictionTracker(plcCfg, storePageMem));
+
+            CheckpointStatus status = readCheckpointStatus();
+
+            cctx.pageStore().initializeForMetastorage();
+
+            restoreMemory(status, true, storePageMem);
+
+            metaStorage = new MetaStorage(cctx.wal(), storeMemPlc, memMetrics, true);
+
+            metaStorage.init(this);
+
+            // here get some data
+
+            metaStorage = null;
+
+            storePageMem.stop();
+        }
+        catch (StorageException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public void lock() throws IgniteCheckedException {
         if (fileLockHolder != null) {
@@ -1447,6 +1537,17 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @param status Checkpoint status.
      */
     private WALPointer restoreMemory(CheckpointStatus status) throws IgniteCheckedException {
+        return restoreMemory(status, false, (PageMemoryEx)metaStorage.pageMemory());
+    }
+
+    /**
+     * @param status Checkpoint status.
+     * @param storeOnly If {@code True} restores Metastorage only.
+     */
+    private WALPointer restoreMemory(CheckpointStatus status, boolean storeOnly,
+        PageMemoryEx storePageMem) throws IgniteCheckedException {
+        assert !storeOnly || storePageMem != null;
+
         if (log.isInfoEnabled())
             log.info("Checking memory state [lastValidPos=" + status.endPtr + ", lastMarked="
                 + status.startPtr + ", lastCheckpointId=" + status.cpStartId + ']');
@@ -1496,9 +1597,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             // Here we do not require tag check because we may be applying memory changes after
                             // several repetitive restarts and the same pages may have changed several times.
                             int grpId = pageRec.fullPageId().groupId();
+
+                            if (storeOnly && grpId != METASTORAGE_CACHE_ID)
+                                continue;
+
                             long pageId = pageRec.fullPageId().pageId();
 
-                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId);
+                            PageMemoryEx pageMem = grpId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(grpId);
 
                             long page = pageMem.acquirePage(grpId, pageId, true);
 
@@ -1526,9 +1631,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             PartitionDestroyRecord destroyRec = (PartitionDestroyRecord)rec;
 
                             final int gId = destroyRec.groupId();
+
+                            if (storeOnly && gId != METASTORAGE_CACHE_ID)
+                                continue;
+
                             final int pId = destroyRec.partitionId();
 
-                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(gId);
+                            PageMemoryEx pageMem = gId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(gId);
 
                             pageMem.clearAsync(new P3<Integer, Long, Integer>() {
                                 @Override public boolean apply(Integer cacheId, Long pageId, Integer tag) {
@@ -1544,9 +1653,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             PageDeltaRecord r = (PageDeltaRecord)rec;
 
                             int grpId = r.groupId();
+
+                            if (storeOnly && grpId != METASTORAGE_CACHE_ID)
+                                continue;
+
                             long pageId = r.pageId();
 
-                            PageMemoryEx pageMem = getPageMemoryForCacheGroup(grpId);
+                            PageMemoryEx pageMem = grpId == METASTORAGE_CACHE_ID ? storePageMem : getPageMemoryForCacheGroup(grpId);
 
                             // Here we do not require tag check because we may be applying memory changes after
                             // several repetitive restarts and the same pages may have changed several times.
@@ -1572,6 +1685,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             }
         }
 
+        if (storeOnly)
+            return null;
+
         if (status.needRestoreMemory()) {
             if (apply)
                 throw new IgniteCheckedException("Failed to restore memory state (checkpoint marker is present " +
@@ -2601,12 +2717,19 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                     int grpId = fullId.groupId();
 
-                    CacheGroupContext grp = context().cache().cacheGroup(grpId);
+                    PageMemoryEx pageMem;
 
-                    if (grp == null)
-                        continue;
+                    if (grpId != MetaStorage.METASTORAGE_CACHE_ID) {
+                        CacheGroupContext grp = context().cache().cacheGroup(grpId);
+
+                        if (grp == null)
+                            continue;
+
+                        pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
+                    }
+                    else
+                        pageMem = (PageMemoryEx)metaStorage.pageMemory();
 
-                    PageMemoryEx pageMem = (PageMemoryEx)grp.memoryPolicy().pageMemory();
 
                     Integer tag = pageMem.getForCheckpoint(
                         fullId, tmpWriteBuf, persStoreMetrics.metricsEnabled() ? tracker : null);
@@ -3323,4 +3446,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     public PersistenceMetricsImpl persistentStoreMetricsImpl() {
         return persStoreMetrics;
     }
+
+    /** {@inheritDoc} */
+    @Override public MetaStorage metaStorage() {
+        return metaStorage;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 5c91a4f..2fe40cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -50,7 +50,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
-import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeListImpl;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
 import org.apache.ignite.internal.processors.cache.persistence.partstate.PagesAllocationRange;
@@ -80,7 +80,7 @@ import org.jetbrains.annotations.Nullable;
  */
 public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl implements DbCheckpointListener {
     /** */
-    private MetaStore metaStore;
+    private IndexStorage indexStorage;
 
     /** */
     private ReuseListImpl reuseList;
@@ -100,7 +100,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
         RootPage metastoreRoot = metas.treeRoot;
 
-        metaStore = new MetadataStorage(grp.memoryPolicy().pageMemory(),
+        indexStorage = new IndexStorageImpl(grp.memoryPolicy().pageMemory(),
             ctx.wal(),
             globalRemoveId(),
             grp.groupId(),
@@ -121,7 +121,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             try {
                 final String name = "PendingEntries";
 
-                RootPage pendingRootPage = metaStore.getOrAllocateForTree(name);
+                RootPage pendingRootPage = indexStorage.getOrAllocateForTree(name);
 
                 pendingEntries = new PendingEntriesTree(
                     grp,
@@ -177,7 +177,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         boolean wasSaveToMeta = false;
 
         if (rowStore0 != null) {
-            FreeListImpl freeList = (FreeListImpl)rowStore0.freeList();
+            CacheFreeListImpl freeList = (CacheFreeListImpl)rowStore0.freeList();
 
             freeList.saveMetadata();
 
@@ -484,7 +484,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         if (grp.sharedGroup())
             idxName = Integer.toString(cacheId) + "_" + idxName;
 
-        return metaStore.getOrAllocateForTree(idxName);
+        return indexStorage.getOrAllocateForTree(idxName);
     }
 
     /** {@inheritDoc} */
@@ -492,7 +492,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         if (grp.sharedGroup())
             idxName = Integer.toString(cacheId) + "_" + idxName;
 
-        metaStore.dropRootPage(idxName);
+        indexStorage.dropRootPage(idxName);
     }
 
     /** {@inheritDoc} */
@@ -612,7 +612,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         for (CacheDataStore store : partDataStores.values()) {
             assert store instanceof GridCacheDataStore;
 
-            FreeListImpl freeList = ((GridCacheDataStore)store).freeList;
+            CacheFreeListImpl freeList = ((GridCacheDataStore)store).freeList;
 
             if (freeList == null)
                 continue;
@@ -862,7 +862,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         private String name;
 
         /** */
-        private volatile FreeListImpl freeList;
+        private volatile CacheFreeListImpl freeList;
 
         /** */
         private volatile CacheDataStore delegate;
@@ -912,7 +912,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
                     RootPage reuseRoot = metas.reuseListRoot;
 
-                    freeList = new FreeListImpl(
+                    freeList = new CacheFreeListImpl(
                         grp.groupId(),
                         grp.cacheOrGroupName() + "-" + partId,
                         grp.memoryPolicy().memoryMetrics(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index d7682f0..d9f7836 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -51,8 +51,9 @@ import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictio
 import org.apache.ignite.internal.processors.cache.persistence.evict.Random2LruPageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.evict.RandomLruPageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFreeListImpl;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
-import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeListImpl;
+import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 import org.apache.ignite.internal.util.typedef.F;
@@ -92,10 +93,10 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     protected MemoryPolicy dfltMemPlc;
 
     /** */
-    private Map<String, FreeListImpl> freeListMap;
+    private Map<String, CacheFreeListImpl> freeListMap;
 
     /** */
-    private FreeListImpl dfltFreeList;
+    private CacheFreeListImpl dfltFreeList;
 
     /** Page size from memory configuration, may be set only for fake(standalone) IgniteCacheDataBaseSharedManager */
     private int pageSize;
@@ -170,7 +171,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
             MemoryMetricsImpl memMetrics = (MemoryMetricsImpl) memMetricsMap.get(memPlcCfg.getName());
 
-            FreeListImpl freeList = new FreeListImpl(0,
+            CacheFreeListImpl freeList = new CacheFreeListImpl(0,
                     cctx.igniteInstanceName(),
                     memMetrics,
                     memPlc,
@@ -211,9 +212,9 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         MemoryPolicyConfiguration[] memPlcsCfgs = memCfg.getMemoryPolicies();
 
         if (memPlcsCfgs == null) {
-            //reserve place for default and system memory policies
-            memPlcMap = U.newHashMap(2);
-            memMetricsMap = U.newHashMap(2);
+            //reserve place for default, system and store memory policies
+            memPlcMap = U.newHashMap(3);
+            memMetricsMap = U.newHashMap(3);
 
             addMemoryPolicy(
                 memCfg,
@@ -228,8 +229,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
             if (DFLT_MEM_PLC_DEFAULT_NAME.equals(dfltMemPlcName) && !hasCustomDefaultMemoryPolicy(memPlcsCfgs)) {
                 //reserve additional place for default and system memory policies
-                memPlcMap = U.newHashMap(memPlcsCfgs.length + 2);
-                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 2);
+                memPlcMap = U.newHashMap(memPlcsCfgs.length + 3);
+                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 3);
 
                 addMemoryPolicy(
                     memCfg,
@@ -240,9 +241,9 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 U.warn(log, "No user-defined default MemoryPolicy found; system default of 1GB size will be used.");
             }
             else {
-                //reserve additional space for system memory policy only
-                memPlcMap = U.newHashMap(memPlcsCfgs.length + 1);
-                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 1);
+                //reserve additional space for system and store memory policies
+                memPlcMap = U.newHashMap(memPlcsCfgs.length + 2);
+                memMetricsMap = U.newHashMap(memPlcsCfgs.length + 2);
             }
 
             for (MemoryPolicyConfiguration memPlcCfg : memPlcsCfgs)
@@ -265,7 +266,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param memPlcName Memory policy name.
      * @throws IgniteCheckedException If failed to initialize swap path.
      */
-    private void addMemoryPolicy(
+    protected void addMemoryPolicy(
         MemoryConfiguration memCfg,
         MemoryPolicyConfiguration memPlcCfg,
         String memPlcName
@@ -298,11 +299,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      */
     protected IgniteOutClosure<Float> fillFactorProvider(final String memPlcName) {
         return new IgniteOutClosure<Float>() {
-            private FreeListImpl freeList;
+            private CacheFreeListImpl freeList;
 
             @Override public Float apply() {
                 if (freeList == null) {
-                    FreeListImpl freeList0 = freeListMap.get(memPlcName);
+                    CacheFreeListImpl freeList0 = freeListMap.get(memPlcName);
 
                     if (freeList0 == null)
                         return (float) 0;
@@ -583,7 +584,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      */
     public void dumpStatistics(IgniteLogger log) {
         if (freeListMap != null) {
-            for (FreeListImpl freeList : freeListMap.values())
+            for (CacheFreeListImpl freeList : freeListMap.values())
                 freeList.dumpStatistics(log);
         }
     }
@@ -865,7 +866,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
         int sysPageSize = pageMem.systemPageSize();
 
-        FreeListImpl freeListImpl = freeListMap.get(plcCfg.getName());
+        CacheFreeListImpl freeListImpl = freeListMap.get(plcCfg.getName());
 
         for (;;) {
             long allocatedPagesCnt = pageMem.loadedPages();
@@ -912,7 +913,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @param plc Memory Policy Configuration.
      * @param pageMem Page memory.
      */
-    private PageEvictionTracker createPageEvictionTracker(MemoryPolicyConfiguration plc, PageMemory pageMem) {
+    protected PageEvictionTracker createPageEvictionTracker(MemoryPolicyConfiguration plc, PageMemory pageMem) {
         if (plc.getPageEvictionMode() == DataPageEvictionMode.DISABLED || cctx.gridConfig().isPersistentStoreEnabled())
             return new NoOpPageEvictionTracker();
 
@@ -940,7 +941,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      *
      * @throws IgniteCheckedException If resolving swap directory fails.
      */
-    @Nullable private File buildAllocPath(MemoryPolicyConfiguration plc) throws IgniteCheckedException {
+    @Nullable File buildAllocPath(MemoryPolicyConfiguration plc) throws IgniteCheckedException {
         String path = plc.getSwapFilePath();
 
         if (path == null)
@@ -1038,4 +1039,11 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     protected void setPageSize(int pageSize) {
         this.pageSize = pageSize;
     }
+
+    /**
+     * @return MetaStorage
+     */
+    public MetaStorage metaStorage() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorage.java
new file mode 100644
index 0000000..5141b04
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorage.java
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Meta store.
+ */
+public interface IndexStorage {
+    /**
+     * Get or allocate initial page for an index.
+     *
+     * @param idxName Index name.
+     * @return {@link RootPage} that keeps pageId, allocated flag that shows whether the page
+     *      was newly allocated, and rootId that is counter which increments each time new page allocated.
+     * @throws IgniteCheckedException If failed.
+     */
+    public RootPage getOrAllocateForTree(String idxName) throws IgniteCheckedException;
+
+    /**
+     * Deallocate index page and remove from tree.
+     *
+     * @param idxName Index name.
+     * @return Root ID or -1 if no page was removed.
+     * @throws IgniteCheckedException  If failed.
+     */
+    public RootPage dropRootPage(String idxName) throws IgniteCheckedException;
+
+    /**
+     * Destroy this meta store.
+     *
+     * @throws IgniteCheckedException  If failed.
+     */
+    public void destroy() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java
new file mode 100644
index 0000000..7daef3c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IndexStorageImpl.java
@@ -0,0 +1,420 @@
+/*
+ * 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;
+
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
+import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
+import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Metadata storage.
+ */
+public class IndexStorageImpl implements IndexStorage {
+    /** Max index name length (bytes num) */
+    public static final int MAX_IDX_NAME_LEN = 255;
+
+    /** Reserved size for index name. Needed for backward compatibility. */
+    public static final int RESERVED_IDX_NAME_LEN = 768;
+
+    /** Bytes in byte. */
+    private static final int BYTE_LEN = 1;
+
+    /** Page memory. */
+    private final PageMemory pageMem;
+
+    /** Index tree. */
+    private final MetaTree metaTree;
+
+    /** Meta page reuse tree. */
+    private final ReuseList reuseList;
+
+    /** Cache group ID. */
+    private final int grpId;
+
+    /** */
+    private final int allocPartId;
+
+    /** */
+    private final byte allocSpace;
+
+    /**
+     * @param pageMem Page memory.
+     * @param wal Write ahead log manager.
+     */
+    public IndexStorageImpl(
+        final PageMemory pageMem,
+        final IgniteWriteAheadLogManager wal,
+        final AtomicLong globalRmvId,
+        final int grpId,
+        final int allocPartId,
+        final byte allocSpace,
+        final ReuseList reuseList,
+        final long rootPageId,
+        final boolean initNew
+    ) {
+        try {
+            this.pageMem = pageMem;
+            this.grpId = grpId;
+            this.allocPartId = allocPartId;
+            this.allocSpace = allocSpace;
+            this.reuseList = reuseList;
+
+            metaTree = new MetaTree(grpId, allocPartId, allocSpace, pageMem, wal, globalRmvId, rootPageId,
+                reuseList, MetaStoreInnerIO.VERSIONS, MetaStoreLeafIO.VERSIONS, initNew);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public RootPage getOrAllocateForTree(final String idxName) throws IgniteCheckedException {
+        final MetaTree tree = metaTree;
+
+        synchronized (this) {
+            byte[] idxNameBytes = idxName.getBytes(StandardCharsets.UTF_8);
+
+            if (idxNameBytes.length > MAX_IDX_NAME_LEN)
+                throw new IllegalArgumentException("Too long encoded indexName [maxAllowed=" + MAX_IDX_NAME_LEN +
+                    ", currentLength=" + idxNameBytes.length + ", name=" + idxName + "]");
+
+            final IndexItem row = tree.findOne(new IndexItem(idxNameBytes, 0));
+
+            if (row == null) {
+                long pageId = 0;
+
+                if (reuseList != null)
+                    pageId = reuseList.takeRecycledPage();
+
+                pageId = pageId == 0 ? pageMem.allocatePage(grpId, allocPartId, allocSpace) : pageId;
+
+                tree.put(new IndexItem(idxNameBytes, pageId));
+
+                return new RootPage(new FullPageId(pageId, grpId), true);
+            }
+            else {
+                final FullPageId pageId = new FullPageId(row.pageId, grpId);
+
+                return new RootPage(pageId, false);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public RootPage dropRootPage(final String idxName)
+        throws IgniteCheckedException {
+        byte[] idxNameBytes = idxName.getBytes(StandardCharsets.UTF_8);
+
+        final IndexItem row = metaTree.remove(new IndexItem(idxNameBytes, 0));
+
+        if (row != null) {
+            if (reuseList == null)
+                pageMem.freePage(grpId, row.pageId);
+        }
+
+        return row != null ? new RootPage(new FullPageId(row.pageId, grpId), false) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void destroy() throws IgniteCheckedException {
+        metaTree.destroy();
+    }
+
+    /**
+     *
+     */
+    private static class MetaTree extends BPlusTree<IndexItem, IndexItem> {
+        /** */
+        private final int allocPartId;
+
+        /** */
+        private final byte allocSpace;
+
+        /**
+         * @param pageMem Page memory.
+         * @param metaPageId Meta page ID.
+         * @param reuseList Reuse list.
+         * @param innerIos Inner IOs.
+         * @param leafIos Leaf IOs.
+         * @throws IgniteCheckedException If failed.
+         */
+        private MetaTree(
+            final int cacheId,
+            final int allocPartId,
+            final byte allocSpace,
+            final PageMemory pageMem,
+            final IgniteWriteAheadLogManager wal,
+            final AtomicLong globalRmvId,
+            final long metaPageId,
+            final ReuseList reuseList,
+            final IOVersions<? extends BPlusInnerIO<IndexItem>> innerIos,
+            final IOVersions<? extends BPlusLeafIO<IndexItem>> leafIos,
+            final boolean initNew
+        ) throws IgniteCheckedException {
+            super(treeName("meta", "Meta"), cacheId, pageMem, wal, globalRmvId, metaPageId, reuseList, innerIos, leafIos);
+
+            this.allocPartId = allocPartId;
+            this.allocSpace = allocSpace;
+
+            initTree(initNew);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected long allocatePageNoReuse() throws IgniteCheckedException {
+            return pageMem.allocatePage(groupId(), allocPartId, allocSpace);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected int compare(final BPlusIO<IndexItem> io, final long pageAddr, final int idx,
+            final IndexItem row) throws IgniteCheckedException {
+            final int off = ((IndexIO)io).getOffset(pageAddr, idx);
+
+            int shift = 0;
+
+            // Compare index names.
+            final int len = PageUtils.getUnsignedByte(pageAddr, off + shift);
+
+            shift += BYTE_LEN;
+
+            for (int i = 0; i < len && i < row.idxName.length; i++) {
+                final int cmp = Byte.compare(PageUtils.getByte(pageAddr, off + i + shift), row.idxName[i]);
+
+                if (cmp != 0)
+                    return cmp;
+            }
+
+            return Integer.compare(len, row.idxName.length);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final long pageAddr,
+            final int idx, Object ignore) throws IgniteCheckedException {
+            return readRow(pageAddr, ((IndexIO)io).getOffset(pageAddr, idx));
+        }
+    }
+
+    /**
+     *
+     */
+    private static class IndexItem {
+        /** */
+        private byte[] idxName;
+
+        /** */
+        private long pageId;
+
+        /**
+         * @param idxName Index name.
+         * @param pageId Page ID.
+         */
+        private IndexItem(final byte[] idxName, final long pageId) {
+            this.idxName = idxName;
+            this.pageId = pageId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "I [idxName=" + new String(idxName) + ", pageId=" + U.hexLong(pageId) + ']';
+        }
+    }
+
+    /**
+     * Store row to buffer.
+     *
+     * @param pageAddr Page address.
+     * @param off Offset in buf.
+     * @param row Row to store.
+     */
+    private static void storeRow(
+        final long pageAddr,
+        int off,
+        final IndexItem row
+    ) {
+        // Index name length.
+        PageUtils.putUnsignedByte(pageAddr, off, row.idxName.length);
+        off++;
+
+        // Index name.
+        PageUtils.putBytes(pageAddr, off, row.idxName);
+        off += row.idxName.length;
+
+        // Page ID.
+        PageUtils.putLong(pageAddr, off, row.pageId);
+    }
+
+    /**
+     * Copy row data.
+     *
+     * @param dstPageAddr Destination page address.
+     * @param dstOff Destination buf offset.
+     * @param srcPageAddr Source page address.
+     * @param srcOff Src buf offset.
+     */
+    private static void storeRow(
+        final long dstPageAddr,
+        int dstOff,
+        final long srcPageAddr,
+        int srcOff
+    ) {
+        // Index name length.
+        final int len = PageUtils.getUnsignedByte(srcPageAddr, srcOff);
+        srcOff++;
+
+        PageUtils.putUnsignedByte(dstPageAddr, dstOff, len);
+        dstOff++;
+
+        PageHandler.copyMemory(srcPageAddr, srcOff, dstPageAddr, dstOff, len);
+        srcOff += len;
+        dstOff += len;
+
+        // Page ID.
+        PageUtils.putLong(dstPageAddr, dstOff, PageUtils.getLong(srcPageAddr, srcOff));
+    }
+
+    /**
+     * Read row from buffer.
+     *
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @return Read row.
+     */
+    private static IndexItem readRow(final long pageAddr, int off) {
+        // Index name length.
+        final int len = PageUtils.getUnsignedByte(pageAddr, off) & 0xFF;
+        off++;
+
+        // Index name.
+        final byte[] idxName = PageUtils.getBytes(pageAddr, off, len);
+        off += len;
+
+        // Page ID.
+        final long pageId = PageUtils.getLong(pageAddr, off);
+
+        return new IndexItem(idxName, pageId);
+    }
+
+    /**
+     *
+     */
+    private interface IndexIO {
+        /**
+         * @param pageAddr Page address.
+         * @param idx Index.
+         * @return Offset in buffer according to {@code idx}.
+         */
+        int getOffset(long pageAddr, int idx);
+    }
+
+    /**
+     *
+     */
+    public static final class MetaStoreInnerIO extends BPlusInnerIO<IndexItem> implements IndexIO {
+        /** */
+        public static final IOVersions<MetaStoreInnerIO> VERSIONS = new IOVersions<>(
+            new MetaStoreInnerIO(1)
+        );
+
+        /**
+         * @param ver Version.
+         */
+        private MetaStoreInnerIO(final int ver) {
+            // name bytes and 1 byte for length, 8 bytes pageId
+            super(T_METASTORE_INNER, ver, false, RESERVED_IDX_NAME_LEN + 1 + 8);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, IndexItem row) throws IgniteCheckedException {
+            storeRow(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(final long dstPageAddr, final int dstIdx, final BPlusIO<IndexItem> srcIo,
+            final long srcPageAddr,
+            final int srcIdx) throws IgniteCheckedException {
+            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcPageAddr, srcIdx));
+        }
+
+        /** {@inheritDoc} */
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final long pageAddr,
+            final int idx) throws IgniteCheckedException {
+            return readRow(pageAddr, offset(idx));
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getOffset(long pageAddr, final int idx) {
+            return offset(idx);
+        }
+    }
+
+    /**
+     *
+     */
+    public static final class MetaStoreLeafIO extends BPlusLeafIO<IndexItem> implements IndexIO {
+        /** */
+        public static final IOVersions<MetaStoreLeafIO> VERSIONS = new IOVersions<>(
+            new MetaStoreLeafIO(1)
+        );
+
+        /**
+         * @param ver Version.
+         */
+        private MetaStoreLeafIO(final int ver) {
+            // 4 byte cache ID, UTF-16 symbols and 1 byte for length, 8 bytes pageId
+            super(T_METASTORE_LEAF, ver, RESERVED_IDX_NAME_LEN + 1 + 8);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
+            storeRow(buf, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(final long dstPageAddr,
+            final int dstIdx,
+            final BPlusIO<IndexItem> srcIo,
+            final long srcPageAddr,
+            final int srcIdx) throws IgniteCheckedException {
+            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcPageAddr, srcIdx));
+        }
+
+        /** {@inheritDoc} */
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree,
+            final long pageAddr,
+            final int idx) throws IgniteCheckedException {
+            return readRow(pageAddr, offset(idx));
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getOffset(long pageAddr, final int idx) {
+            return offset(idx);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
index 3261874..1da52ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsImpl.java
@@ -69,7 +69,7 @@ public class MemoryMetricsImpl implements MemoryMetrics {
 
     /**
      * @param memPlcCfg MemoryPolicyConfiguration.
-    */
+     */
     public MemoryMetricsImpl(MemoryPolicyConfiguration memPlcCfg) {
         this(memPlcCfg, null);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetaStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetaStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetaStore.java
deleted file mode 100644
index c09ce4e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetaStore.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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;
-
-import org.apache.ignite.IgniteCheckedException;
-
-/**
- * Meta store.
- */
-public interface MetaStore {
-    /**
-     * Get or allocate initial page for an index.
-     *
-     * @param idxName Index name.
-     * @return {@link RootPage} that keeps pageId, allocated flag that shows whether the page
-     *      was newly allocated, and rootId that is counter which increments each time new page allocated.
-     * @throws IgniteCheckedException If failed.
-     */
-    public RootPage getOrAllocateForTree(String idxName) throws IgniteCheckedException;
-
-    /**
-     * Deallocate index page and remove from tree.
-     *
-     * @param idxName Index name.
-     * @return Root ID or -1 if no page was removed.
-     * @throws IgniteCheckedException  If failed.
-     */
-    public RootPage dropRootPage(String idxName) throws IgniteCheckedException;
-
-    /**
-     * Destroy this meta store.
-     *
-     * @throws IgniteCheckedException  If failed.
-     */
-    public void destroy() throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
deleted file mode 100644
index e667807..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MetadataStorage.java
+++ /dev/null
@@ -1,420 +0,0 @@
-/*
- * 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;
-
-import java.nio.charset.StandardCharsets;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.pagemem.FullPageId;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.pagemem.PageUtils;
-import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
-import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusInnerIO;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusLeafIO;
-import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersions;
-import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
-import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-/**
- * Metadata storage.
- */
-public class MetadataStorage implements MetaStore {
-    /** Max index name length (bytes num) */
-    public static final int MAX_IDX_NAME_LEN = 255;
-
-    /** Reserved size for index name. Needed for backward compatibility. */
-    public static final int RESERVED_IDX_NAME_LEN = 768;
-
-    /** Bytes in byte. */
-    private static final int BYTE_LEN = 1;
-
-    /** Page memory. */
-    private final PageMemory pageMem;
-
-    /** Index tree. */
-    private final MetaTree metaTree;
-
-    /** Meta page reuse tree. */
-    private final ReuseList reuseList;
-
-    /** Cache group ID. */
-    private final int grpId;
-
-    /** */
-    private final int allocPartId;
-
-    /** */
-    private final byte allocSpace;
-
-    /**
-     * @param pageMem Page memory.
-     * @param wal Write ahead log manager.
-     */
-    public MetadataStorage(
-        final PageMemory pageMem,
-        final IgniteWriteAheadLogManager wal,
-        final AtomicLong globalRmvId,
-        final int grpId,
-        final int allocPartId,
-        final byte allocSpace,
-        final ReuseList reuseList,
-        final long rootPageId,
-        final boolean initNew
-    ) {
-        try {
-            this.pageMem = pageMem;
-            this.grpId = grpId;
-            this.allocPartId = allocPartId;
-            this.allocSpace = allocSpace;
-            this.reuseList = reuseList;
-
-            metaTree = new MetaTree(grpId, allocPartId, allocSpace, pageMem, wal, globalRmvId, rootPageId,
-                reuseList, MetaStoreInnerIO.VERSIONS, MetaStoreLeafIO.VERSIONS, initNew);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public RootPage getOrAllocateForTree(final String idxName) throws IgniteCheckedException {
-        final MetaTree tree = metaTree;
-
-        synchronized (this) {
-            byte[] idxNameBytes = idxName.getBytes(StandardCharsets.UTF_8);
-
-            if (idxNameBytes.length > MAX_IDX_NAME_LEN)
-                throw new IllegalArgumentException("Too long encoded indexName [maxAllowed=" + MAX_IDX_NAME_LEN +
-                    ", currentLength=" + idxNameBytes.length + ", name=" + idxName + "]");
-
-            final IndexItem row = tree.findOne(new IndexItem(idxNameBytes, 0));
-
-            if (row == null) {
-                long pageId = 0;
-
-                if (reuseList != null)
-                    pageId = reuseList.takeRecycledPage();
-
-                pageId = pageId == 0 ? pageMem.allocatePage(grpId, allocPartId, allocSpace) : pageId;
-
-                tree.put(new IndexItem(idxNameBytes, pageId));
-
-                return new RootPage(new FullPageId(pageId, grpId), true);
-            }
-            else {
-                final FullPageId pageId = new FullPageId(row.pageId, grpId);
-
-                return new RootPage(pageId, false);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public RootPage dropRootPage(final String idxName)
-        throws IgniteCheckedException {
-        byte[] idxNameBytes = idxName.getBytes(StandardCharsets.UTF_8);
-
-        final IndexItem row = metaTree.remove(new IndexItem(idxNameBytes, 0));
-
-        if (row != null) {
-            if (reuseList == null)
-                pageMem.freePage(grpId, row.pageId);
-        }
-
-        return row != null ? new RootPage(new FullPageId(row.pageId, grpId), false) : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void destroy() throws IgniteCheckedException {
-        metaTree.destroy();
-    }
-
-    /**
-     *
-     */
-    private static class MetaTree extends BPlusTree<IndexItem, IndexItem> {
-        /** */
-        private final int allocPartId;
-
-        /** */
-        private final byte allocSpace;
-
-        /**
-         * @param pageMem Page memory.
-         * @param metaPageId Meta page ID.
-         * @param reuseList Reuse list.
-         * @param innerIos Inner IOs.
-         * @param leafIos Leaf IOs.
-         * @throws IgniteCheckedException If failed.
-         */
-        private MetaTree(
-            final int cacheId,
-            final int allocPartId,
-            final byte allocSpace,
-            final PageMemory pageMem,
-            final IgniteWriteAheadLogManager wal,
-            final AtomicLong globalRmvId,
-            final long metaPageId,
-            final ReuseList reuseList,
-            final IOVersions<? extends BPlusInnerIO<IndexItem>> innerIos,
-            final IOVersions<? extends BPlusLeafIO<IndexItem>> leafIos,
-            final boolean initNew
-        ) throws IgniteCheckedException {
-            super(treeName("meta", "Meta"), cacheId, pageMem, wal, globalRmvId, metaPageId, reuseList, innerIos, leafIos);
-
-            this.allocPartId = allocPartId;
-            this.allocSpace = allocSpace;
-
-            initTree(initNew);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected long allocatePageNoReuse() throws IgniteCheckedException {
-            return pageMem.allocatePage(groupId(), allocPartId, allocSpace);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected int compare(final BPlusIO<IndexItem> io, final long pageAddr, final int idx,
-            final IndexItem row) throws IgniteCheckedException {
-            final int off = ((IndexIO)io).getOffset(pageAddr, idx);
-
-            int shift = 0;
-
-            // Compare index names.
-            final int len = PageUtils.getUnsignedByte(pageAddr, off + shift);
-
-            shift += BYTE_LEN;
-
-            for (int i = 0; i < len && i < row.idxName.length; i++) {
-                final int cmp = Byte.compare(PageUtils.getByte(pageAddr, off + i + shift), row.idxName[i]);
-
-                if (cmp != 0)
-                    return cmp;
-            }
-
-            return Integer.compare(len, row.idxName.length);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final long pageAddr,
-            final int idx, Object ignore) throws IgniteCheckedException {
-            return readRow(pageAddr, ((IndexIO)io).getOffset(pageAddr, idx));
-        }
-    }
-
-    /**
-     *
-     */
-    private static class IndexItem {
-        /** */
-        private byte[] idxName;
-
-        /** */
-        private long pageId;
-
-        /**
-         * @param idxName Index name.
-         * @param pageId Page ID.
-         */
-        private IndexItem(final byte[] idxName, final long pageId) {
-            this.idxName = idxName;
-            this.pageId = pageId;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return "I [idxName=" + new String(idxName) + ", pageId=" + U.hexLong(pageId) + ']';
-        }
-    }
-
-    /**
-     * Store row to buffer.
-     *
-     * @param pageAddr Page address.
-     * @param off Offset in buf.
-     * @param row Row to store.
-     */
-    private static void storeRow(
-        final long pageAddr,
-        int off,
-        final IndexItem row
-    ) {
-        // Index name length.
-        PageUtils.putUnsignedByte(pageAddr, off, row.idxName.length);
-        off++;
-
-        // Index name.
-        PageUtils.putBytes(pageAddr, off, row.idxName);
-        off += row.idxName.length;
-
-        // Page ID.
-        PageUtils.putLong(pageAddr, off, row.pageId);
-    }
-
-    /**
-     * Copy row data.
-     *
-     * @param dstPageAddr Destination page address.
-     * @param dstOff Destination buf offset.
-     * @param srcPageAddr Source page address.
-     * @param srcOff Src buf offset.
-     */
-    private static void storeRow(
-        final long dstPageAddr,
-        int dstOff,
-        final long srcPageAddr,
-        int srcOff
-    ) {
-        // Index name length.
-        final int len = PageUtils.getUnsignedByte(srcPageAddr, srcOff);
-        srcOff++;
-
-        PageUtils.putUnsignedByte(dstPageAddr, dstOff, len);
-        dstOff++;
-
-        PageHandler.copyMemory(srcPageAddr, srcOff, dstPageAddr, dstOff, len);
-        srcOff += len;
-        dstOff += len;
-
-        // Page ID.
-        PageUtils.putLong(dstPageAddr, dstOff, PageUtils.getLong(srcPageAddr, srcOff));
-    }
-
-    /**
-     * Read row from buffer.
-     *
-     * @param pageAddr Page address.
-     * @param off Offset.
-     * @return Read row.
-     */
-    private static IndexItem readRow(final long pageAddr, int off) {
-        // Index name length.
-        final int len = PageUtils.getUnsignedByte(pageAddr, off) & 0xFF;
-        off++;
-
-        // Index name.
-        final byte[] idxName = PageUtils.getBytes(pageAddr, off, len);
-        off += len;
-
-        // Page ID.
-        final long pageId = PageUtils.getLong(pageAddr, off);
-
-        return new IndexItem(idxName, pageId);
-    }
-
-    /**
-     *
-     */
-    private interface IndexIO {
-        /**
-         * @param pageAddr Page address.
-         * @param idx Index.
-         * @return Offset in buffer according to {@code idx}.
-         */
-        int getOffset(long pageAddr, int idx);
-    }
-
-    /**
-     *
-     */
-    public static final class MetaStoreInnerIO extends BPlusInnerIO<IndexItem> implements IndexIO {
-        /** */
-        public static final IOVersions<MetaStoreInnerIO> VERSIONS = new IOVersions<>(
-            new MetaStoreInnerIO(1)
-        );
-
-        /**
-         * @param ver Version.
-         */
-        private MetaStoreInnerIO(final int ver) {
-            // name bytes and 1 byte for length, 8 bytes pageId
-            super(T_METASTORE_INNER, ver, false, RESERVED_IDX_NAME_LEN + 1 + 8);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void storeByOffset(long pageAddr, int off, IndexItem row) throws IgniteCheckedException {
-            storeRow(pageAddr, off, row);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void store(final long dstPageAddr, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final long srcPageAddr,
-            final int srcIdx) throws IgniteCheckedException {
-            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcPageAddr, srcIdx));
-        }
-
-        /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final long pageAddr,
-            final int idx) throws IgniteCheckedException {
-            return readRow(pageAddr, offset(idx));
-        }
-
-        /** {@inheritDoc} */
-        @Override public int getOffset(long pageAddr, final int idx) {
-            return offset(idx);
-        }
-    }
-
-    /**
-     *
-     */
-    public static final class MetaStoreLeafIO extends BPlusLeafIO<IndexItem> implements IndexIO {
-        /** */
-        public static final IOVersions<MetaStoreLeafIO> VERSIONS = new IOVersions<>(
-            new MetaStoreLeafIO(1)
-        );
-
-        /**
-         * @param ver Version.
-         */
-        private MetaStoreLeafIO(final int ver) {
-            // 4 byte cache ID, UTF-16 symbols and 1 byte for length, 8 bytes pageId
-            super(T_METASTORE_LEAF, ver, RESERVED_IDX_NAME_LEN + 1 + 8);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
-            storeRow(buf, off, row);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void store(final long dstPageAddr,
-            final int dstIdx,
-            final BPlusIO<IndexItem> srcIo,
-            final long srcPageAddr,
-            final int srcIdx) throws IgniteCheckedException {
-            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcPageAddr, srcIdx));
-        }
-
-        /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree,
-            final long pageAddr,
-            final int idx) throws IgniteCheckedException {
-            return readRow(pageAddr, offset(idx));
-        }
-
-        /** {@inheritDoc} */
-        @Override public int getOffset(long pageAddr, final int idx) {
-            return offset(idx);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java
new file mode 100644
index 0000000..ae200df
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+/**
+ * Simple interface for data, store in some RowStore.
+ */
+public interface Storable {
+    /**
+     * @param link Link for this row.
+     */
+    public void link(long link);
+
+    /**
+     * @return Link for this row.
+     */
+    public long link();
+
+    /**
+     * @return Partition.
+     */
+    public int partition();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/61a4de80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index ed82127..e2df6e3 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -48,6 +48,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
@@ -195,6 +196,20 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /** {@inheritDoc} */
+    @Override public void initializeForMetastorage()
+        throws IgniteCheckedException {
+        int grpId = MetaStorage.METASTORAGE_CACHE_ID;
+
+        if (!idxCacheStores.containsKey(grpId)) {
+            CacheStoreHolder holder = initDir(new File(storeWorkDir, "metastorage"), grpId, 1);
+
+            CacheStoreHolder old = idxCacheStores.put(grpId, holder);
+
+            assert old == null : "Non-null old store holder for metastorage";
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void storeCacheData(StoredCacheData cacheData, boolean overwrite) throws IgniteCheckedException {
         File cacheWorkDir = cacheWorkDirectory(cacheData.config());
         File file;
@@ -344,19 +359,30 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
         File cacheWorkDir = cacheWorkDirectory(ccfg);
 
+        return initDir(cacheWorkDir, grpDesc.groupId(), grpDesc.config().getAffinity().partitions());
+    }
+
+    /**
+     * @param cacheWorkDir Work directory.
+     * @param grpId Group ID.
+     * @param partitions Number of partitions.
+     * @return Cache store holder.
+     * @throws IgniteCheckedException If failed.
+     */
+    private CacheStoreHolder initDir(File cacheWorkDir, int grpId, int partitions) throws IgniteCheckedException {
         boolean dirExisted = checkAndInitCacheWorkDir(cacheWorkDir);
 
         File idxFile = new File(cacheWorkDir, INDEX_FILE_NAME);
 
         if (dirExisted && !idxFile.exists())
-            grpsWithoutIdx.add(grpDesc.groupId());
+            grpsWithoutIdx.add(grpId);
 
         FileVersionCheckingFactory pageStoreFactory = new FileVersionCheckingFactory(
             pstCfg.getFileIOFactory(), igniteCfg.getMemoryConfiguration());
 
         FilePageStore idxStore = pageStoreFactory.createPageStore(PageMemory.FLAG_IDX, idxFile);
 
-        FilePageStore[] partStores = new FilePageStore[grpDesc.config().getAffinity().partitions()];
+        FilePageStore[] partStores = new FilePageStore[partitions];
 
         for (int partId = 0; partId < partStores.length; partId++) {
             FilePageStore partStore = pageStoreFactory.createPageStore(