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/20 07:32:16 UTC

[07/11] ignite git commit: IGNITE-6030 Allow enabling persistence per data region

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsSnapshot.java
deleted file mode 100644
index 4e7f90a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryMetricsSnapshot.java
+++ /dev/null
@@ -1,112 +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.MemoryMetrics;
-
-/**
- *
- */
-public class MemoryMetricsSnapshot implements MemoryMetrics {
-    /** */
-    private String name;
-
-    /** */
-    private long totalAllocatedPages;
-
-    /** */
-    private float allocationRate;
-
-    /** */
-    private float evictionRate;
-
-    /** */
-    private float largeEntriesPagesPercentage;
-
-    /** */
-    private float pagesFillFactor;
-
-    /** */
-    private long dirtyPages;
-
-    /** */
-    private float pageReplaceRate;
-
-    /** */
-    private long physicalMemoryPages;
-
-    /**
-     * @param metrics Metrics instance to take a copy.
-     */
-    public MemoryMetricsSnapshot(MemoryMetrics metrics) {
-        name = metrics.getName();
-        totalAllocatedPages = metrics.getTotalAllocatedPages();
-        allocationRate = metrics.getAllocationRate();
-        evictionRate = metrics.getEvictionRate();
-        largeEntriesPagesPercentage = metrics.getLargeEntriesPagesPercentage();
-        pagesFillFactor = metrics.getPagesFillFactor();
-        dirtyPages = metrics.getDirtyPages();
-        pageReplaceRate = metrics.getPagesReplaceRate();
-        physicalMemoryPages = metrics.getPhysicalMemoryPages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getName() {
-        return name;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getTotalAllocatedPages() {
-        return totalAllocatedPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getAllocationRate() {
-        return allocationRate;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getEvictionRate() {
-        return evictionRate;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getLargeEntriesPagesPercentage() {
-        return largeEntriesPagesPercentage;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getPagesFillFactor() {
-        return pagesFillFactor;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getDirtyPages() {
-        return dirtyPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getPagesReplaceRate() {
-        return pageReplaceRate;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getPhysicalMemoryPages() {
-        return physicalMemoryPages;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicy.java
deleted file mode 100644
index 4059c12..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/MemoryPolicy.java
+++ /dev/null
@@ -1,84 +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.configuration.MemoryPolicyConfiguration;
-import org.apache.ignite.internal.pagemem.PageMemory;
-import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictionTracker;
-
-/**
- * Memory policy provides access to objects configured with {@link MemoryPolicyConfiguration} configuration.
- */
-public class MemoryPolicy {
-    /** */
-    private final PageMemory pageMem;
-
-    /** */
-    private final MemoryMetricsImpl memMetrics;
-
-    /** */
-    private final MemoryPolicyConfiguration cfg;
-
-    /** */
-    private final PageEvictionTracker evictionTracker;
-
-    /**
-     * @param pageMem PageMemory instance.
-     * @param memMetrics MemoryMetrics instance.
-     * @param cfg Configuration of given MemoryPolicy.
-     * @param evictionTracker Eviction tracker.
-     */
-    public MemoryPolicy(
-        PageMemory pageMem,
-        MemoryPolicyConfiguration cfg,
-        MemoryMetricsImpl memMetrics,
-        PageEvictionTracker evictionTracker
-    ) {
-        this.pageMem = pageMem;
-        this.memMetrics = memMetrics;
-        this.cfg = cfg;
-        this.evictionTracker = evictionTracker;
-    }
-
-    /**
-     *
-     */
-    public PageMemory pageMemory() {
-        return pageMem;
-    }
-
-    /**
-     * @return Config.
-     */
-    public MemoryPolicyConfiguration config() {
-        return cfg;
-    }
-
-    /**
-     * @return Memory Metrics.
-     */
-    public MemoryMetricsImpl memoryMetrics() {
-        return memMetrics;
-    }
-
-    /**
-     *
-     */
-    public PageEvictionTracker evictionTracker() {
-        return evictionTracker;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsImpl.java
deleted file mode 100644
index 7952937..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsImpl.java
+++ /dev/null
@@ -1,297 +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.internal.pagemem.wal.IgniteWriteAheadLogManager;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
-import org.apache.ignite.mxbean.PersistenceMetricsMXBean;
-
-/**
- *
- */
-public class PersistenceMetricsImpl implements PersistenceMetricsMXBean {
-    /** */
-    private volatile HitRateMetrics walLoggingRate;
-
-    /** */
-    private volatile HitRateMetrics walWritingRate;
-
-    /** */
-    private volatile HitRateMetrics walFsyncTimeDuration;
-
-    /** */
-    private volatile HitRateMetrics walFsyncTimeNumber;
-
-    /** */
-    private volatile long lastCpLockWaitDuration;
-
-    /** */
-    private volatile long lastCpMarkDuration;
-
-    /** */
-    private volatile long lastCpPagesWriteDuration;
-
-    /** */
-    private volatile long lastCpDuration;
-
-    /** */
-    private volatile long lastCpFsyncDuration;
-
-    /** */
-    private volatile long lastCpTotalPages;
-
-    /** */
-    private volatile long lastCpDataPages;
-
-    /** */
-    private volatile long lastCpCowPages;
-
-    /** */
-    private volatile long rateTimeInterval;
-
-    /** */
-    private volatile int subInts;
-
-    /** */
-    private volatile boolean metricsEnabled;
-
-    /** */
-    private IgniteWriteAheadLogManager wal;
-
-    /**
-     * @param metricsEnabled Metrics enabled flag.
-     * @param rateTimeInterval Rate time interval.
-     * @param subInts Number of sub-intervals.
-     */
-    public PersistenceMetricsImpl(
-        boolean metricsEnabled,
-        long rateTimeInterval,
-        int subInts
-    ) {
-        this.metricsEnabled = metricsEnabled;
-        this.rateTimeInterval = rateTimeInterval;
-        this.subInts = subInts;
-
-        resetRates();
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getWalLoggingRate() {
-        if (!metricsEnabled)
-            return 0;
-
-        return ((float)walLoggingRate.getRate()) / rateTimeInterval;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getWalWritingRate() {
-        if (!metricsEnabled)
-            return 0;
-
-        return ((float)walWritingRate.getRate()) / rateTimeInterval;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getWalArchiveSegments() {
-        if (!metricsEnabled)
-            return 0;
-
-        return wal.walArchiveSegments();
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getWalFsyncTimeAverage() {
-        if (!metricsEnabled)
-            return 0;
-
-        long numRate = walFsyncTimeNumber.getRate();
-
-        if (numRate == 0)
-            return 0;
-
-        return (float)walFsyncTimeDuration.getRate() / numRate;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointingDuration() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointLockWaitDuration() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpLockWaitDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointMarkDuration() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpMarkDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointPagesWriteDuration() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpPagesWriteDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointFsyncDuration() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpFsyncDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointTotalPagesNumber() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpTotalPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointDataPagesNumber() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpDataPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointCopiedOnWritePagesNumber() {
-        if (!metricsEnabled)
-            return 0;
-
-        return lastCpCowPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void enableMetrics() {
-        metricsEnabled = true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disableMetrics() {
-        metricsEnabled = false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void rateTimeInterval(long rateTimeInterval) {
-        this.rateTimeInterval = rateTimeInterval;
-
-        resetRates();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void subIntervals(int subInts) {
-        this.subInts = subInts;
-
-        resetRates();
-    }
-
-    /**
-     * @param wal Write-ahead log manager.
-     */
-    public void wal(IgniteWriteAheadLogManager wal) {
-        this.wal = wal;
-    }
-
-    /**
-     * @return Metrics enabled flag.
-     */
-    public boolean metricsEnabled() {
-        return metricsEnabled;
-    }
-
-    /**
-     * @param lockWaitDuration Lock wait duration.
-     * @param markDuration Mark duration.
-     * @param pagesWriteDuration Pages write duration.
-     * @param fsyncDuration Total checkpoint fsync duration.
-     * @param duration Total checkpoint duration.
-     * @param totalPages Total number of all pages in checkpoint.
-     * @param dataPages Total number of data pages in checkpoint.
-     * @param cowPages Total number of COW-ed pages in checkpoint.
-     */
-    public void onCheckpoint(
-        long lockWaitDuration,
-        long markDuration,
-        long pagesWriteDuration,
-        long fsyncDuration,
-        long duration,
-        long totalPages,
-        long dataPages,
-        long cowPages
-    ) {
-        if (metricsEnabled) {
-            lastCpLockWaitDuration = lockWaitDuration;
-            lastCpMarkDuration = markDuration;
-            lastCpPagesWriteDuration = pagesWriteDuration;
-            lastCpFsyncDuration = fsyncDuration;
-            lastCpDuration = duration;
-            lastCpTotalPages = totalPages;
-            lastCpDataPages = dataPages;
-            lastCpCowPages = cowPages;
-        }
-    }
-
-    /**
-     *
-     */
-    public void onWalRecordLogged() {
-        walLoggingRate.onHit();
-    }
-
-    /**
-     * @param size Size written.
-     */
-    public void onWalBytesWritten(int size) {
-        walWritingRate.onHits(size);
-    }
-
-    /**
-     * @param nanoTime Fsync nano time.
-     */
-    public void onFsync(long nanoTime) {
-        long microseconds = nanoTime / 1_000;
-
-        walFsyncTimeDuration.onHits(microseconds);
-        walFsyncTimeNumber.onHit();
-    }
-
-    /**
-     *
-     */
-    private void resetRates() {
-        walLoggingRate = new HitRateMetrics((int)rateTimeInterval, subInts);
-        walWritingRate = new HitRateMetrics((int)rateTimeInterval, subInts);
-
-        walFsyncTimeDuration = new HitRateMetrics((int)rateTimeInterval, subInts);
-        walFsyncTimeNumber = new HitRateMetrics((int)rateTimeInterval, subInts);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsSnapshot.java
deleted file mode 100644
index 0de9950..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/PersistenceMetricsSnapshot.java
+++ /dev/null
@@ -1,144 +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.PersistenceMetrics;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- *
- */
-public class PersistenceMetricsSnapshot implements PersistenceMetrics {
-    /** */
-    private float walLoggingRate;
-
-    /** */
-    private float walWritingRate;
-
-    /** */
-    private int walArchiveSegments;
-
-    /** */
-    private float walFsyncTimeAvg;
-
-    /** */
-    private long lastCpDuration;
-
-    /** */
-    private long lastCpLockWaitDuration;
-
-    /** */
-    private long lastCpMmarkDuration;
-
-    /** */
-    private long lastCpPagesWriteDuration;
-
-    /** */
-    private long lastCpFsyncDuration;
-
-    /** */
-    private long lastCpTotalPages;
-
-    /** */
-    private long lastCpDataPages;
-
-    /** */
-    private long lastCpCowPages;
-
-    /**
-     * @param metrics Metrics.
-     */
-    public PersistenceMetricsSnapshot(PersistenceMetrics metrics) {
-        walLoggingRate = metrics.getWalLoggingRate();
-        walWritingRate = metrics.getWalWritingRate();
-        walArchiveSegments = metrics.getWalArchiveSegments();
-        walFsyncTimeAvg = metrics.getWalFsyncTimeAverage();
-        lastCpDuration = metrics.getLastCheckpointingDuration();
-        lastCpLockWaitDuration = metrics.getLastCheckpointLockWaitDuration();
-        lastCpMmarkDuration = metrics.getLastCheckpointMarkDuration();
-        lastCpPagesWriteDuration = metrics.getLastCheckpointPagesWriteDuration();
-        lastCpFsyncDuration = metrics.getLastCheckpointFsyncDuration();
-        lastCpTotalPages = metrics.getLastCheckpointTotalPagesNumber();
-        lastCpDataPages = metrics.getLastCheckpointDataPagesNumber();
-        lastCpCowPages = metrics.getLastCheckpointCopiedOnWritePagesNumber();
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getWalLoggingRate() {
-        return walLoggingRate;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getWalWritingRate() {
-        return walWritingRate;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getWalArchiveSegments() {
-        return walArchiveSegments;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getWalFsyncTimeAverage() {
-        return walFsyncTimeAvg;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointingDuration() {
-        return lastCpDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointLockWaitDuration() {
-        return lastCpLockWaitDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointMarkDuration() {
-        return lastCpMmarkDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointPagesWriteDuration() {
-        return lastCpPagesWriteDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointFsyncDuration() {
-        return lastCpFsyncDuration;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointTotalPagesNumber() {
-        return lastCpTotalPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointDataPagesNumber() {
-        return lastCpDataPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getLastCheckpointCopiedOnWritePagesNumber() {
-        return lastCpCowPages;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PersistenceMetricsSnapshot.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
index 9cc5c62..2051021 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
@@ -40,7 +40,8 @@ public class RowStore {
     /** */
     protected final CacheObjectContext coctx;
 
-
+    /** */
+    private final boolean persistenceEnabled;
 
     /**
      * @param grp Cache group.
@@ -54,7 +55,9 @@ public class RowStore {
 
         ctx = grp.shared();
         coctx = grp.cacheObjectContext();
-        pageMem = grp.memoryPolicy().pageMemory();
+        pageMem = grp.dataRegion().pageMemory();
+
+        persistenceEnabled = grp.dataRegion().config().isPersistenceEnabled();
     }
 
     /**
@@ -63,13 +66,18 @@ public class RowStore {
      */
     public void removeRow(long link) throws IgniteCheckedException {
         assert link != 0;
-        ctx.database().checkpointReadLock();
 
-        try {
+        if (!persistenceEnabled)
             freeList.removeDataRowByLink(link);
-        }
-        finally {
-            ctx.database().checkpointReadUnlock();
+        else {
+            ctx.database().checkpointReadLock();
+
+            try {
+                freeList.removeDataRowByLink(link);
+            }
+            finally {
+                ctx.database().checkpointReadUnlock();
+            }
         }
     }
 
@@ -78,13 +86,17 @@ public class RowStore {
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(CacheDataRow row) throws IgniteCheckedException {
-        ctx.database().checkpointReadLock();
-
-        try {
+        if (!persistenceEnabled)
             freeList.insertDataRow(row);
-        }
-        finally {
-            ctx.database().checkpointReadUnlock();
+        else {
+            ctx.database().checkpointReadLock();
+
+            try {
+                freeList.insertDataRow(row);
+            }
+            finally {
+                ctx.database().checkpointReadUnlock();
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/FairFifoPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/FairFifoPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/FairFifoPageEvictionTracker.java
index f5c7c8a..8a3d5b0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/FairFifoPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/FairFifoPageEvictionTracker.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.persistence.evict;
 import java.util.LinkedList;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
@@ -34,12 +34,12 @@ public class FairFifoPageEvictionTracker extends PageAbstractEvictionTracker {
 
     /**
      * @param pageMem Page memory.
-     * @param plcCfg Memory policy configuration.
+     * @param plcCfg Data region configuration.
      * @param sharedCtx Shared context.
      */
     public FairFifoPageEvictionTracker(
         PageMemoryNoStoreImpl pageMem,
-        MemoryPolicyConfiguration plcCfg,
+        DataRegionConfiguration plcCfg,
         GridCacheSharedContext sharedCtx) {
         super(pageMem, plcCfg, sharedCtx);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
index a524d5e..5142c59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/PageAbstractEvictionTracker.java
@@ -18,7 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence.evict;
 
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -54,12 +54,12 @@ public abstract class PageAbstractEvictionTracker implements PageEvictionTracker
 
     /**
      * @param pageMem Page memory.
-     * @param plcCfg Memory policy configuration.
+     * @param plcCfg Data region configuration.
      * @param sharedCtx Shared context.
      */
     PageAbstractEvictionTracker(
         PageMemoryNoStoreImpl pageMem,
-        MemoryPolicyConfiguration plcCfg,
+        DataRegionConfiguration plcCfg,
         GridCacheSharedContext sharedCtx
     ) {
         this.pageMem = pageMem;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/Random2LruPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/Random2LruPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/Random2LruPageEvictionTracker.java
index 00f1b16..4d42191 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/Random2LruPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/Random2LruPageEvictionTracker.java
@@ -20,8 +20,8 @@ import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
@@ -55,12 +55,12 @@ public class Random2LruPageEvictionTracker extends PageAbstractEvictionTracker {
      */
     public Random2LruPageEvictionTracker(
         PageMemoryNoStoreImpl pageMem,
-        MemoryPolicyConfiguration plcCfg,
+        DataRegionConfiguration plcCfg,
         GridCacheSharedContext<?, ?> sharedCtx
     ) {
         super(pageMem, plcCfg, sharedCtx);
 
-        MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
+        DataStorageConfiguration memCfg = sharedCtx.kernalContext().config().getDataStorageConfiguration();
 
         assert plcCfg.getMaxSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/RandomLruPageEvictionTracker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/RandomLruPageEvictionTracker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/RandomLruPageEvictionTracker.java
index 035a91a..ed6d2d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/RandomLruPageEvictionTracker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/evict/RandomLruPageEvictionTracker.java
@@ -21,8 +21,8 @@ import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
@@ -57,12 +57,12 @@ public class RandomLruPageEvictionTracker extends PageAbstractEvictionTracker {
      */
     public RandomLruPageEvictionTracker(
         PageMemory pageMem,
-        MemoryPolicyConfiguration plcCfg,
+        DataRegionConfiguration plcCfg,
         GridCacheSharedContext<?, ?> sharedCtx
     ) {
         super((PageMemoryNoStoreImpl)pageMem, plcCfg, sharedCtx);
 
-        MemoryConfiguration memCfg = sharedCtx.kernalContext().config().getMemoryConfiguration();
+        DataStorageConfiguration memCfg = sharedCtx.kernalContext().config().getDataStorageConfiguration();
 
         assert plcCfg.getMaxSize() / memCfg.getPageSize() < Integer.MAX_VALUE;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
index 0fb3052..104697e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/AsyncFileIOFactory.java
@@ -33,20 +33,34 @@ public class AsyncFileIOFactory implements FileIOFactory {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Thread local channel future holder. */
+    private transient volatile ThreadLocal<AsyncFileIO.ChannelOpFuture> holder = initHolder();
+
     /** {@inheritDoc} */
     @Override public FileIO create(File file) throws IOException {
         return create(file, CREATE, READ, WRITE);
     }
 
-    /** */
-    private ThreadLocal<AsyncFileIO.ChannelOpFuture> holder = new ThreadLocal<AsyncFileIO.ChannelOpFuture>() {
-        @Override protected AsyncFileIO.ChannelOpFuture initialValue() {
-            return new AsyncFileIO.ChannelOpFuture();
-        }
-    };
-
     /** {@inheritDoc} */
     @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+        if (holder == null) {
+            synchronized (this) {
+                if (holder == null)
+                    holder = initHolder();
+            }
+        }
+
         return new AsyncFileIO(file, holder, modes);
     }
+
+    /**
+     * Initializes thread local channel future holder.
+     */
+    private ThreadLocal<AsyncFileIO.ChannelOpFuture> initHolder() {
+        return new ThreadLocal<AsyncFileIO.ChannelOpFuture>() {
+            @Override protected AsyncFileIO.ChannelOpFuture initialValue() {
+                return new AsyncFileIO.ChannelOpFuture();
+            }
+        };
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
index 0547dbc..408240c 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
@@ -27,7 +27,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.store.PageStore;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
@@ -60,7 +60,7 @@ public class FilePageStore implements PageStore {
     private final byte type;
 
     /** Database configuration. */
-    protected final MemoryConfiguration dbCfg;
+    protected final DataStorageConfiguration dbCfg;
 
     /** Factory to provide I/O interfaces for read/write operations with files */
     private final FileIOFactory ioFactory;
@@ -92,7 +92,7 @@ public class FilePageStore implements PageStore {
     /**
      * @param file File.
      */
-    public FilePageStore(byte type, File file, FileIOFactory factory, MemoryConfiguration cfg) {
+    public FilePageStore(byte type, File file, FileIOFactory factory, DataStorageConfiguration cfg) {
         this.type = type;
 
         cfgFile = file;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/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..aadcee6 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
@@ -36,7 +36,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
@@ -92,7 +92,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     private final IgniteConfiguration igniteCfg;
 
     /** */
-    private PersistentStoreConfiguration pstCfg;
+    private DataStorageConfiguration dsCfg;
 
     /** Absolute directory for file page store. Includes consistent id based folder. */
     private File storeWorkDir;
@@ -109,11 +109,11 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     public FilePageStoreManager(GridKernalContext ctx) {
         igniteCfg = ctx.config();
 
-        PersistentStoreConfiguration pstCfg = igniteCfg.getPersistentStoreConfiguration();
+        DataStorageConfiguration dsCfg = igniteCfg.getDataStorageConfiguration();
 
-        assert pstCfg != null : "WAL should not be created if persistence is disabled.";
+        assert dsCfg != null;
 
-        this.pstCfg = pstCfg;
+        this.dsCfg = dsCfg;
     }
 
     /** {@inheritDoc} */
@@ -352,7 +352,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
             grpsWithoutIdx.add(grpDesc.groupId());
 
         FileVersionCheckingFactory pageStoreFactory = new FileVersionCheckingFactory(
-            pstCfg.getFileIOFactory(), igniteCfg.getMemoryConfiguration());
+            dsCfg.getFileIOFactory(), igniteCfg.getDataStorageConfiguration());
 
         FilePageStore idxStore = pageStoreFactory.createPageStore(PageMemory.FLAG_IDX, idxFile);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
index 5d044ec..c2e2d36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreV2.java
@@ -17,7 +17,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.file;
 
 import java.io.File;
-import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 
 /**
  *
@@ -35,7 +35,7 @@ public class FilePageStoreV2 extends FilePageStore {
      * @param factory Factory.
      * @param cfg Config.
      */
-    public FilePageStoreV2(byte type, File file, FileIOFactory factory, MemoryConfiguration cfg) {
+    public FilePageStoreV2(byte type, File file, FileIOFactory factory, DataStorageConfiguration cfg) {
         super(type, file, factory, cfg);
 
         hdrSize = cfg.getPageSize();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
index 40870dc..bab2cf0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FileVersionCheckingFactory.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 
 /**
  * Checks version in files if it's present on the disk, creates store with latest version otherwise.
@@ -38,14 +38,14 @@ public class FileVersionCheckingFactory implements FilePageStoreFactory {
     private final FileIOFactory fileIOFactory;
 
     /** Memory configuration. */
-    private final MemoryConfiguration memCfg;
+    private final DataStorageConfiguration memCfg;
 
     /**
      * @param fileIOFactory File io factory.
      * @param memCfg Memory configuration.
      */
     public FileVersionCheckingFactory(
-        FileIOFactory fileIOFactory, MemoryConfiguration memCfg) {
+        FileIOFactory fileIOFactory, DataStorageConfiguration memCfg) {
         this.fileIOFactory = fileIOFactory;
         this.memCfg = memCfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
index c73a952..e7a7e63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
@@ -31,12 +31,13 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
@@ -157,7 +158,7 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
         //here deprecated method is used to get compatible version of consistentId
         final Serializable consistentId = ctx.discovery().consistentId();
 
-        if (!cfg.isPersistentStoreEnabled())
+        if (!CU.isPersistenceEnabled(cfg))
             return compatibleResolve(pstStoreBasePath, consistentId);
 
         if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false))
@@ -442,12 +443,12 @@ public class PdsConsistentIdProcessor extends GridProcessorAdapter implements Pd
      * @throws IgniteCheckedException if I/O failed.
      */
     @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException {
-        final PersistentStoreConfiguration pstCfg = cfg.getPersistentStoreConfiguration();
+        final DataStorageConfiguration dsCfg = cfg.getDataStorageConfiguration();
 
-        if (pstCfg == null)
+        if (dsCfg == null)
             return null;
 
-        final String pstPath = pstCfg.getPersistentStorePath();
+        final String pstPath = dsCfg.getStoragePath();
 
         return U.resolveWorkDirectory(
             cfg.getWorkDirectory(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
index 3eb62ae..6a87d3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeListImpl.java
@@ -31,8 +31,8 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageUpdateRecord;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
-import org.apache.ignite.internal.processors.cache.persistence.MemoryMetricsImpl;
-import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
@@ -81,7 +81,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     private final PageHandler<CacheDataRow, Boolean> updateRow = new UpdateRowHandler();
 
     /** */
-    private final MemoryMetricsImpl memMetrics;
+    private final DataRegionMetricsImpl memMetrics;
 
     /** */
     private final PageEvictionTracker evictionTracker;
@@ -313,7 +313,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
      * @param cacheId Cache ID.
      * @param name Name (for debug purpose).
      * @param memMetrics Memory metrics.
-     * @param memPlc Memory policy.
+     * @param memPlc Data region.
      * @param reuseList Reuse list or {@code null} if this free list will be a reuse list for itself.
      * @param wal Write ahead log manager.
      * @param metaPageId Metadata page ID.
@@ -323,8 +323,8 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     public FreeListImpl(
         int cacheId,
         String name,
-        MemoryMetricsImpl memMetrics,
-        MemoryPolicy memPlc,
+        DataRegionMetricsImpl memMetrics,
+        DataRegion memPlc,
         ReuseList reuseList,
         IgniteWriteAheadLogManager wal,
         long metaPageId,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
index 8a540a0..b113c62 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
@@ -180,8 +180,6 @@ public abstract class PagesList extends DataStructure {
 
                             assert nextId != pageId :
                                 "Loop detected [next=" + U.hexLong(nextId) + ", cur=" + U.hexLong(pageId) + ']';
-
-
                         }
                         finally {
                             readUnlock(pageId, page, pageAddr);
@@ -354,9 +352,8 @@ public abstract class PagesList extends DataStructure {
      * @param pageId Page ID.
      * @param page Page absolute pointer.
      * @param pageAddr Page address.
-     * @throws IgniteCheckedException If failed.
      */
-    private void releaseAndClose(long pageId, long page, long pageAddr) throws IgniteCheckedException {
+    private void releaseAndClose(long pageId, long page, long pageAddr) {
         if (page != 0L) {
             try {
                 // No special WAL record because we most likely changed the whole page.
@@ -924,7 +921,7 @@ public abstract class PagesList extends DataStructure {
      * @param bucket Bucket index.
      * @return Page for take.
      */
-    private Stripe getPageForTake(int bucket) throws IgniteCheckedException {
+    private Stripe getPageForTake(int bucket) {
         Stripe[] tails = getBucket(bucket);
 
         if (tails == null || bucketsSize[bucket].get() == 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 95b81ad..8c64e0e 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -58,7 +58,7 @@ 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.CheckpointLockStateChecker;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.MemoryMetricsImpl;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingPageIO;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
@@ -235,7 +235,7 @@ public class PageMemoryImpl implements PageMemoryEx {
     private long[] sizes;
 
     /** */
-    private MemoryMetricsImpl memMetrics;
+    private DataRegionMetricsImpl memMetrics;
 
     /** */
     private volatile boolean closed;
@@ -256,7 +256,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         GridInClosure3X<FullPageId, ByteBuffer, Integer> flushDirtyPage,
         GridInClosure3X<Long, FullPageId, PageMemoryEx> changeTracker,
         CheckpointLockStateChecker stateChecker,
-        MemoryMetricsImpl memMetrics,
+        DataRegionMetricsImpl memMetrics,
         boolean throttleEnabled
     ) {
         assert sharedCtx != null;
@@ -1817,7 +1817,7 @@ public class PageMemoryImpl implements PageMemoryEx {
                 pageEvictWarned = true;
 
                 U.warn(log, "Page evictions started, this will affect storage performance (consider increasing " +
-                    "MemoryConfiguration#setPageCacheSize).");
+                    "DataStorageConfiguration#setPageCacheSize).");
             }
 
             final ThreadLocalRandom rnd = ThreadLocalRandom.current();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 383c605..b4fc192 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -43,8 +43,8 @@ import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.events.WalSegmentArchivedEvent;
@@ -61,7 +61,7 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.PersistenceMetricsImpl;
+import org.apache.ignite.internal.processors.cache.persistence.DataStorageMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
@@ -76,7 +76,6 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -146,7 +145,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private final long fsyncDelay;
 
     /** */
-    private final PersistentStoreConfiguration psCfg;
+    private final DataStorageConfiguration dsCfg;
 
     /** Events service */
     private final GridEventStorageManager evt;
@@ -155,7 +154,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private IgniteConfiguration igCfg;
 
     /** Persistence metrics tracker. */
-    private PersistenceMetricsImpl metrics;
+    private DataStorageMetricsImpl metrics;
 
     /** */
     private File walWorkDir;
@@ -209,7 +208,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
     /**
      * Positive (non-0) value indicates WAL can be archived even if not complete<br>
-     * See {@link PersistentStoreConfiguration#setWalAutoArchiveAfterInactivity(long)}<br>
+     * See {@link DataStorageConfiguration#setWalAutoArchiveAfterInactivity(long)}<br>
      */
     private final long walAutoArchiveAfterInactivity;
 
@@ -239,20 +238,20 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     public FileWriteAheadLogManager(@NotNull final GridKernalContext ctx) {
         igCfg = ctx.config();
 
-        PersistentStoreConfiguration psCfg = igCfg.getPersistentStoreConfiguration();
+        DataStorageConfiguration dsCfg = igCfg.getDataStorageConfiguration();
 
-        assert psCfg != null : "WAL should not be created if persistence is disabled.";
+        assert dsCfg != null;
 
-        this.psCfg = psCfg;
+        this.dsCfg = dsCfg;
 
-        maxWalSegmentSize = psCfg.getWalSegmentSize();
-        mode = psCfg.getWalMode();
-        tlbSize = psCfg.getTlbSize();
-        flushFreq = psCfg.getWalFlushFrequency();
-        fsyncDelay = psCfg.getWalFsyncDelayNanos();
-        alwaysWriteFullPages = psCfg.isAlwaysWriteFullPages();
-        ioFactory = psCfg.getFileIOFactory();
-        walAutoArchiveAfterInactivity = psCfg.getWalAutoArchiveAfterInactivity();
+        maxWalSegmentSize = dsCfg.getWalSegmentSize();
+        mode = dsCfg.getWalMode();
+        tlbSize = dsCfg.getWalThreadLocalBufferSize();
+        flushFreq = dsCfg.getWalFlushFrequency();
+        fsyncDelay = dsCfg.getWalFsyncDelayNanos();
+        alwaysWriteFullPages = dsCfg.isAlwaysWriteFullPages();
+        ioFactory = dsCfg.getFileIOFactory();
+        walAutoArchiveAfterInactivity = dsCfg.getWalAutoArchiveAfterInactivity();
         evt = ctx.event();
     }
 
@@ -264,15 +263,15 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             checkWalConfiguration();
 
             walWorkDir = initDirectory(
-                psCfg.getWalStorePath(),
-                PersistentStoreConfiguration.DFLT_WAL_STORE_PATH,
+                dsCfg.getWalPath(),
+                DataStorageConfiguration.DFLT_WAL_PATH,
                 resolveFolders.folderName(),
                 "write ahead log work directory"
             );
 
             walArchiveDir = initDirectory(
-                psCfg.getWalArchivePath(),
-                PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH,
+                dsCfg.getWalArchivePath(),
+                DataStorageConfiguration.DFLT_WAL_ARCHIVE_PATH,
                 resolveFolders.folderName(),
                 "write ahead log archive directory"
             );
@@ -305,11 +304,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      * @throws IgniteCheckedException if WAL store path is configured and archive path isn't (or vice versa)
      */
     private void checkWalConfiguration() throws IgniteCheckedException {
-        if (psCfg.getWalStorePath() == null ^ psCfg.getWalArchivePath() == null) {
+        if (dsCfg.getWalPath() == null ^ dsCfg.getWalArchivePath() == null) {
             throw new IgniteCheckedException(
                 "Properties should be either both specified or both null " +
-                    "[walStorePath = " + psCfg.getWalStorePath() +
-                    ", walArchivePath = " + psCfg.getWalArchivePath() + "]"
+                    "[walStorePath = " + dsCfg.getWalPath() +
+                    ", walArchivePath = " + dsCfg.getWalArchivePath() + "]"
             );
         }
     }
@@ -575,7 +574,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             walArchiveDir,
             (FileWALPointer)start,
             end,
-            psCfg,
+            dsCfg,
             serializer,
             ioFactory,
             archiver,
@@ -821,7 +820,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private FileWriteHandle restoreWriteHandle(FileWALPointer lastReadPtr) throws IgniteCheckedException {
         long absIdx = lastReadPtr == null ? 0 : lastReadPtr.index();
 
-        long segNo = absIdx % psCfg.getWalSegments();
+        long segNo = absIdx % dsCfg.getWalSegments();
 
         File curFile = new File(walWorkDir, FileDescriptor.fileName(segNo));
 
@@ -934,9 +933,9 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
         File[] allFiles = walWorkDir.listFiles(WAL_SEGMENT_FILE_FILTER);
 
-        if (allFiles.length != 0 && allFiles.length > psCfg.getWalSegments())
+        if (allFiles.length != 0 && allFiles.length > dsCfg.getWalSegments())
             throw new IgniteCheckedException("Failed to initialize wal (work directory contains " +
-                "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + psCfg.getWalSegments() + ']');
+                "incorrect number of segments) [cur=" + allFiles.length + ", expected=" + dsCfg.getWalSegments() + ']');
 
         // Allocate the first segment synchronously. All other segments will be allocated by archiver in background.
         if (allFiles.length == 0) {
@@ -958,7 +957,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             log.debug("Formatting file [exists=" + file.exists() + ", file=" + file.getAbsolutePath() + ']');
 
         try (FileIO fileIO = ioFactory.create(file, CREATE, READ, WRITE)) {
-            int left = psCfg.getWalSegmentSize();
+            int left = dsCfg.getWalSegmentSize();
 
             if (mode == WALMode.DEFAULT) {
                 while (left > 0) {
@@ -1017,7 +1016,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         // Signal to archiver that we are done with the segment and it can be archived.
         long absNextIdx = archiver.nextAbsoluteSegmentIndex(curIdx);
 
-        long segmentIdx = absNextIdx % psCfg.getWalSegments();
+        long segmentIdx = absNextIdx % dsCfg.getWalSegments();
 
         return new File(walWorkDir, FileDescriptor.fileName(segmentIdx));
     }
@@ -1085,7 +1084,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
     /**
      * File archiver operates on absolute segment indexes. For any given absolute segment index N we can calculate
-     * the work WAL segment: S(N) = N % psCfg.walSegments.
+     * the work WAL segment: S(N) = N % dsCfg.walSegments.
      * When a work segment is finished, it is given to the archiver. If the absolute index of last archived segment
      * is denoted by A and the absolute index of next segment we want to write is denoted by W, then we can allow
      * write to S(W) if W - A <= walSegments. <br>
@@ -1296,7 +1295,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                     // Notify archiver thread.
                     notifyAll();
 
-                    while (curAbsWalIdx - lastAbsArchivedIdx > psCfg.getWalSegments() && cleanException == null)
+                    while (curAbsWalIdx - lastAbsArchivedIdx > dsCfg.getWalSegments() && cleanException == null)
                         wait();
 
                     return curAbsWalIdx;
@@ -1366,7 +1365,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          * @param absIdx Absolute index to archive.
          */
         private SegmentArchiveResult archiveSegment(long absIdx) throws IgniteCheckedException {
-            long segIdx = absIdx % psCfg.getWalSegments();
+            long segIdx = absIdx % dsCfg.getWalSegments();
 
             File origFile = new File(walWorkDir, FileDescriptor.fileName(segIdx));
 
@@ -1427,7 +1426,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
-     * Validate files depending on {@link PersistentStoreConfiguration#getWalSegments()}  and create if need.
+     * Validate files depending on {@link DataStorageConfiguration#getWalSegments()}  and create if need.
      * Check end when exit condition return false or all files are passed.
      *
      * @param startWith Start with.
@@ -1436,14 +1435,14 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
      * @throws IgniteCheckedException if validation or create file fail.
      */
     private void checkFiles(int startWith, boolean create, IgnitePredicate<Integer> p) throws IgniteCheckedException {
-        for (int i = startWith; i < psCfg.getWalSegments() && (p == null || (p != null && p.apply(i))); i++) {
+        for (int i = startWith; i < dsCfg.getWalSegments() && (p == null || (p != null && p.apply(i))); i++) {
             File checkFile = new File(walWorkDir, FileDescriptor.fileName(i));
 
             if (checkFile.exists()) {
                 if (checkFile.isDirectory())
                     throw new IgniteCheckedException("Failed to initialize WAL log segment (a directory with " +
                         "the same name already exists): " + checkFile.getAbsolutePath());
-                else if (checkFile.length() != psCfg.getWalSegmentSize() && mode == WALMode.DEFAULT)
+                else if (checkFile.length() != dsCfg.getWalSegmentSize() && mode == WALMode.DEFAULT)
                     throw new IgniteCheckedException("Failed to initialize WAL log segment " +
                         "(WAL segment size change is not supported):" + checkFile.getAbsolutePath());
             }
@@ -1768,7 +1767,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** Condition activated each time writeBuffer() completes. Used to wait previously flushed write to complete */
         private final Condition writeComplete = lock.newCondition();
 
-        /** Condition for timed wait of several threads, see {@link PersistentStoreConfiguration#getWalFsyncDelayNanos()} */
+        /** Condition for timed wait of several threads, see {@link DataStorageConfiguration#getWalFsyncDelayNanos()} */
         private final Condition fsync = lock.newCondition();
 
         /**
@@ -2488,7 +2487,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         private final FileArchiver archiver;
 
         /** */
-        private final PersistentStoreConfiguration psCfg;
+        private final DataStorageConfiguration psCfg;
 
         /** Optional start pointer. */
         @Nullable
@@ -2516,7 +2515,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             File walArchiveDir,
             @Nullable FileWALPointer start,
             @Nullable FileWALPointer end,
-            PersistentStoreConfiguration psCfg,
+            DataStorageConfiguration psCfg,
             @NotNull RecordSerializer serializer,
             FileIOFactory ioFactory,
             FileArchiver archiver,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
index 0fb8adf..962c4ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.reader;
 import java.io.File;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
@@ -84,8 +84,8 @@ public class IgniteWalIteratorFactory {
         this.binaryMetadataFileStoreDir = binaryMetadataFileStoreDir;
         this.marshallerMappingFileStoreDir = marshallerMappingFileStoreDir;
         this.keepBinary = keepBinary;
-        this.ioFactory = new PersistentStoreConfiguration().getFileIOFactory();
-        new MemoryConfiguration().setPageSize(pageSize); // just for validate
+        this.ioFactory = new DataStorageConfiguration().getFileIOFactory();
+        new DataStorageConfiguration().setPageSize(pageSize); // just for validate
     }
 
     /**
@@ -122,7 +122,7 @@ public class IgniteWalIteratorFactory {
         this.log = log;
         this.pageSize = pageSize;
         this.ioFactory = ioFactory;
-        new MemoryConfiguration().setPageSize(pageSize); // just for validate
+        new DataStorageConfiguration().setPageSize(pageSize); // just for validate
     }
 
     /**
@@ -134,7 +134,7 @@ public class IgniteWalIteratorFactory {
      * according its boundaries.
      */
     public IgniteWalIteratorFactory(@NotNull final IgniteLogger log, int pageSize) {
-        this(log, new PersistentStoreConfiguration().getFileIOFactory(), pageSize);
+        this(log, new DataStorageConfiguration().getFileIOFactory(), pageSize);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index e234766..c0c3650 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -27,8 +27,9 @@ import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridKernalGateway;
@@ -177,8 +178,12 @@ public class StandaloneGridKernalContext implements GridKernalContext {
         final Marshaller marshaller = new BinaryMarshaller();
         cfg.setMarshaller(marshaller);
 
-        PersistentStoreConfiguration pstCfg = new PersistentStoreConfiguration();
-        cfg.setPersistentStoreConfiguration(pstCfg);
+        final DataStorageConfiguration pstCfg = new DataStorageConfiguration();
+        final DataRegionConfiguration regCfg = new DataRegionConfiguration();
+        regCfg.setPersistenceEnabled(true);
+        pstCfg.setDefaultDataRegionConfiguration(regCfg);
+
+        cfg.setDataStorageConfiguration(pstCfg);
 
         marshaller.setContext(marshallerCtx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java
index 824bc7a..9c096a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java
@@ -28,7 +28,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
  *
  * <p>Implementation is nonblocking and protected from hits loss.
  * Maximum relative error is 1/{@link #size}.
- * 2^56 - 1 hits per interval can be accumulated without numeric overflow.
+ * 2^55 - 1 hits per interval can be accumulated without numeric overflow.
  */
 public class HitRateMetrics {
     /** Bits that store actual hit count. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 143e5cb..7da4898 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -648,7 +648,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                     if (dhtVer == null)
                                         dhtVer = explicitVer != null ? explicitVer : writeVersion();
 
-                                    if (cctx.wal() != null && !writeEntries().isEmpty()
+                                    if (cacheCtx.group().persistenceEnabled() && !writeEntries().isEmpty()
                                         && op != NOOP && op != RELOAD && op != READ)
                                         ptr = cctx.wal().log(new DataRecord(new DataEntry(
                                             cacheCtx.cacheId(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
index 36306cb..afa3fd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
@@ -64,8 +64,8 @@ public class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
     ) throws IgniteCheckedException {
         super(name,
             grp.groupId(),
-            grp.memoryPolicy().pageMemory(),
-            grp.shared().wal(),
+            grp.dataRegion().pageMemory(),
+            grp.dataRegion().config().isPersistenceEnabled() ? grp.shared().wal() : null,
             grp.offheap().globalRemoveId(),
             metaPageId,
             reuseList,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java
index fad3a50..a6ec6e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/PendingEntriesTree.java
@@ -55,7 +55,7 @@ public class PendingEntriesTree extends BPlusTree<PendingRow, PendingRow> {
         super(name,
             grp.groupId(),
             pageMem,
-            grp.shared().wal(),
+            grp.dataRegion().config().isPersistenceEnabled() ? grp.shared().wal() : null,
             grp.offheap().globalRemoveId(),
             metaPageId,
             reuseList,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index ff42ff6..3cd0451 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteCallable;
@@ -142,7 +143,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         // Start first node as inactive if persistence is enabled.
-        boolean activeOnStart = !ctx.config().isPersistentStoreEnabled() && ctx.config().isActiveOnStart();
+        boolean activeOnStart = !CU.isPersistenceEnabled(ctx.config()) && ctx.config().isActiveOnStart();
 
         globalState = DiscoveryDataClusterState.createState(activeOnStart);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index e73fa6c..12765df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -65,7 +65,7 @@ import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
-import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters;
 import org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor;
@@ -243,7 +243,7 @@ public class IgfsDataManager extends IgfsManager {
      * @return Maximum number of bytes for IGFS data cache.
      */
     public long maxSpaceSize() {
-        MemoryPolicy plc = dataCachePrj.context().memoryPolicy();
+        DataRegion plc = dataCachePrj.context().dataRegion();
 
         long size = plc != null ? plc.config().getMaxSize() : 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e88a234..98428b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -2554,7 +2554,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     private void saveCacheConfiguration(DynamicCacheDescriptor desc) {
         GridCacheSharedContext cctx = ctx.cache().context();
 
-        if (cctx.pageStore() != null && cctx.database().persistenceEnabled() && !cctx.kernalContext().clientNode()) {
+        if (cctx.pageStore() != null && cctx.database().persistenceEnabled() && !cctx.kernalContext().clientNode() &&
+            CU.isPersistentCache(desc.cacheConfiguration(), cctx.gridConfig().getDataStorageConfiguration())) {
             CacheConfiguration cfg = desc.cacheConfiguration();
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index 6fe056c..f2fd195 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -26,7 +26,7 @@ import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -142,7 +142,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     /** */
     private boolean loadPrevVal;
 
-    /** Name of {@link MemoryPolicyConfiguration} for this cache */
+    /** Name of {@link DataRegionConfiguration} for this cache */
     private String memPlcName;
 
     /** Maximum inline size for sql indexes. */
@@ -219,7 +219,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
         evictFilter = compactClass(ccfg.getEvictionFilter());
         lsnrConfigurations = compactIterable(ccfg.getCacheEntryListenerConfigurations());
         loadPrevVal = ccfg.isLoadPreviousValue();
-        memPlcName = ccfg.getMemoryPolicyName();
+        memPlcName = ccfg.getDataRegionName();
         sqlIdxMaxInlineSize = ccfg.getSqlIndexMaxInlineSize();
         nodeFilter = compactClass(ccfg.getNodeFilter());
         qryDetailMetricsSz = ccfg.getQueryDetailMetricsSize();
@@ -460,7 +460,7 @@ public class VisorCacheConfiguration extends VisorDataTransferObject {
     }
 
     /**
-     * @return {@link MemoryPolicyConfiguration} name.
+     * @return {@link DataRegionConfiguration} name.
      */
     public String getMemoryPolicyName() {
         return memPlcName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ec41370c/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java
index c6cdd5c..37fed5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorMemoryMetrics.java
@@ -20,13 +20,13 @@ package org.apache.ignite.internal.visor.cache;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import org.apache.ignite.MemoryMetrics;
+import org.apache.ignite.DataRegionMetrics;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
- * Data transfer object for {@link MemoryMetrics}
+ * Data transfer object for {@link DataRegionMetrics}
  */
 public class VisorMemoryMetrics extends VisorDataTransferObject {
     /** */
@@ -69,7 +69,7 @@ public class VisorMemoryMetrics extends VisorDataTransferObject {
     /**
      * @param m Metrics instance to create DTO.
      */
-    public VisorMemoryMetrics(MemoryMetrics m) {
+    public VisorMemoryMetrics(DataRegionMetrics m) {
         name = m.getName();
         totalAllocatedPages = m.getTotalAllocatedPages();
         allocationRate = m.getAllocationRate();