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

[1/7] ignite git commit: IGNITE-6639 TCP communication skip local node loopback address. This closes #2909.

Repository: ignite
Updated Branches:
  refs/heads/ignite-6667 46501d92b -> ece4db124


IGNITE-6639 TCP communication skip local node loopback address. This closes #2909.

Signed-off-by: nikolay_tikhonov <nt...@gridgain.com>


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

Branch: refs/heads/ignite-6667
Commit: 6ed872b4e49c2c767dad975aa9613fc881726027
Parents: bbf4853
Author: mcherkasov <mc...@gridgain.com>
Authored: Wed Oct 25 18:44:53 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Oct 25 18:44:53 2017 +0300

----------------------------------------------------------------------
 .../spi/communication/tcp/TcpCommunicationSpi.java    | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ed872b4/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index a0ee389..065609f 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -3012,6 +3012,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
         if (isExtAddrsExist)
             addrs.addAll(extAddrs);
 
+        if (log.isDebugEnabled())
+            log.debug("Addresses resolved from attributes [rmtNode=" + node.id() + ", addrs=" + addrs +
+                ", isRmtAddrsExist=" + isRmtAddrsExist + ']');
+
         if (filterReachableAddresses) {
             Set<InetAddress> allInetAddrs = U.newHashSet(addrs.size());
 
@@ -3041,7 +3045,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
             }
 
             if (log.isDebugEnabled())
-                log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs.toString() + ']');
+                log.debug("Addresses to connect for node [rmtNode=" + node.id() + ", addrs=" + addrs + ']');
         }
 
         return addrs;
@@ -3074,6 +3078,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
             int lastWaitingTimeout = 1;
 
             while (client == null) { // Reconnection on handshake timeout.
+                if (addr.getAddress().isLoopbackAddress() && addr.getPort() == boundTcpPort) {
+                    if (log.isDebugEnabled())
+                        log.debug("Skipping local address [addr=" + addr +
+                            ", locAddrs=" + node.attribute(createSpiAttributeName(ATTR_ADDRS)) +
+                            ", node=" + node + ']');
+                    continue;
+                }
+
                 boolean needWait = false;
 
                 try {


[2/7] ignite git commit: IGNITE-6654 Ignite client can hang in case IgniteOOM on server. This closes #2908.

Posted by sb...@apache.org.
IGNITE-6654 Ignite client can hang in case IgniteOOM on server. This closes #2908.

Signed-off-by: nikolay_tikhonov <nt...@gridgain.com>


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

Branch: refs/heads/ignite-6667
Commit: 918febaa17efa9e109fc68d268afbc7109a800e9
Parents: 6ed872b
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Oct 25 18:46:59 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Oct 25 18:46:59 2017 +0300

----------------------------------------------------------------------
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   7 +-
 .../datastreamer/DataStreamerImpl.java          |  23 +-
 .../cache/IgniteOutOfMemoryPropagationTest.java | 251 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite6.java       |   5 +
 5 files changed, 285 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/918febaa/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index 6ba68c2..e219d6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -290,9 +290,11 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         if (relPtr == INVALID_REL_PTR)
             throw new IgniteOutOfMemoryException("Not enough memory allocated " +
-                "(consider increasing data region size or enabling evictions) " +
                 "[policyName=" + dataRegionCfg.getName() +
-                ", size=" + U.readableSize(dataRegionCfg.getMaxSize(), true) + "]"
+                ", size=" + U.readableSize(dataRegionCfg.getMaxSize(), true) + "]" + U.nl() +
+                "Consider increasing memory policy size, enabling evictions, adding more nodes to the cluster, " +
+                "reducing number of backups or reducing model size."
+
             );
 
         assert (relPtr & ~PageIdUtils.PAGE_IDX_MASK) == 0 : U.hexLong(relPtr & ~PageIdUtils.PAGE_IDX_MASK);

http://git-wip-us.apache.org/repos/asf/ignite/blob/918febaa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 5095f45..a7dd615 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -38,6 +38,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
 import org.apache.ignite.internal.pagemem.wal.StorageException;
 import org.apache.ignite.internal.processors.affinity.AffinityAssignment;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -96,6 +97,7 @@ import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.CO;
 import org.apache.ignite.internal.util.typedef.CX1;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 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.S;
@@ -3213,7 +3215,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 catch (GridDhtInvalidPartitionException ignored) {
                     // Ignore.
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteCheckedException|RuntimeException e) {
+                    if(e instanceof RuntimeException && !X.hasCause(e, IgniteOutOfMemoryException.class))
+                        throw (RuntimeException)e;
+
                     IgniteCheckedException err = new IgniteCheckedException("Failed to update key on backup node: " + key, e);
 
                     if (nearRes != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/918febaa/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 6ed552a..d38132f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -188,6 +188,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     /** {@code True} if data loader has been cancelled. */
     private volatile boolean cancelled;
 
+    /** Cancellation reason. */
+    private volatile Throwable cancellationReason = null;
+
     /** Fail counter. */
     private final LongAdder8 failCntr = new LongAdder8();
 
@@ -210,7 +213,12 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
                 failCntr.increment();
 
-                cancelled = true;
+                synchronized (DataStreamerImpl.this) {
+                    if(cancellationReason == null)
+                        cancellationReason = err;
+
+                    cancelled = true;
+                }
             }
         }
     };
@@ -399,12 +407,12 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             if (disconnectErr != null)
                 throw disconnectErr;
 
-            throw new IllegalStateException("Data streamer has been closed.");
+            closedException();
         }
         else if (cancelled) {
             busyLock.leaveBusy();
 
-            throw new IllegalStateException("Data streamer has been closed.");
+            closedException();
         }
     }
 
@@ -886,7 +894,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                                             @Override public void run() {
                                                 try {
                                                     if (cancelled)
-                                                        throw new IllegalStateException("DataStreamer closed.");
+                                                        closedException();
 
                                                     load0(entriesForNode, resFut, activeKeys, remaps + 1);
                                                 }
@@ -990,6 +998,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     }
 
     /**
+     * Throws stream closed exception.
+     */
+    private void closedException() {
+        throw new IllegalStateException("Data streamer has been closed.", cancellationReason);
+    }
+
+    /**
      * @param key Key to map.
      * @param topVer Topology version.
      * @param cctx Context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/918febaa/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOutOfMemoryPropagationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOutOfMemoryPropagationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOutOfMemoryPropagationTest.java
new file mode 100644
index 0000000..a13cbd4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteOutOfMemoryPropagationTest.java
@@ -0,0 +1,251 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterTopologyException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ *
+ */
+public class IgniteOutOfMemoryPropagationTest extends GridCommonAbstractTest {
+
+    /** */
+    public static final int NODES = 3;
+
+    /** */
+    private CacheAtomicityMode atomicityMode;
+
+    /** */
+    private CacheMode mode;
+
+    /** */
+    private int backupsCount;
+
+    /** */
+    private CacheWriteSynchronizationMode writeSyncMode;
+
+    /** */
+    private IgniteEx client;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        assert G.allGrids().isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 20 * 60 * 1000;
+    }
+
+    /** */
+    public void testPutOOMPropagation() throws Exception {
+        testOOMPropagation(false);
+    }
+
+    /** */
+    public void testStreamerOOMPropagation() throws Exception {
+        testOOMPropagation(true);
+    }
+
+    /** */
+    private void testOOMPropagation(boolean useStreamer) throws Exception {
+        for (CacheAtomicityMode atomicityMode : CacheAtomicityMode.values()) {
+            for (CacheMode cacheMode : CacheMode.values()) {
+                for (CacheWriteSynchronizationMode writeSyncMode : CacheWriteSynchronizationMode.values()) {
+                    for (int backupsCount = 0; backupsCount < 1; backupsCount++) {
+                        if (writeSyncMode == CacheWriteSynchronizationMode.FULL_ASYNC
+                            || cacheMode == CacheMode.REPLICATED)
+                            continue;
+
+                        if (atomicityMode == CacheAtomicityMode.TRANSACTIONAL && !useStreamer) {
+                            for (TransactionConcurrency concurrency : TransactionConcurrency.values()) {
+                                for (TransactionIsolation isolation : TransactionIsolation.values()) {
+                                    checkOOMPropagation(
+                                        false,
+                                        CacheAtomicityMode.TRANSACTIONAL,
+                                        cacheMode,
+                                        writeSyncMode,
+                                        backupsCount,
+                                        concurrency,
+                                        isolation);
+                                }
+                            }
+                        }
+                        else
+                            checkOOMPropagation(useStreamer, atomicityMode, cacheMode, writeSyncMode, backupsCount);
+                    }
+                }
+            }
+        }
+    }
+
+    /** */
+    private void checkOOMPropagation(boolean useStreamer, CacheAtomicityMode atomicityMode, CacheMode cacheMode,
+        CacheWriteSynchronizationMode writeSyncMode, int backupsCount) throws Exception {
+        checkOOMPropagation(useStreamer, atomicityMode, cacheMode, writeSyncMode, backupsCount, null, null);
+    }
+
+    /** */
+    private void checkOOMPropagation(boolean useStreamer, CacheAtomicityMode atomicityMode, CacheMode cacheMode,
+        CacheWriteSynchronizationMode writeSyncMode, int backupsCount,
+        TransactionConcurrency concurrency, TransactionIsolation isolation) throws Exception {
+        Throwable t = null;
+
+        System.out.println("Checking conf: CacheAtomicityMode." + atomicityMode +
+            " CacheMode." + mode + " CacheWriteSynchronizationMode." + writeSyncMode + " backupsCount = " + backupsCount
+            + " TransactionConcurrency." + concurrency + " TransactionIsolation." + isolation);
+
+        initGrid(atomicityMode, cacheMode, writeSyncMode, backupsCount);
+        try {
+            forceOOM(useStreamer, concurrency, isolation);
+        }
+        catch (Throwable t0) {
+            t = t0;
+
+            t.printStackTrace(System.out);
+
+            assertTrue(X.hasCause(t, IgniteOutOfMemoryException.class, ClusterTopologyException.class));
+        }
+        finally {
+            assertNotNull(t);
+
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * Ignite grid of 3 server nodes with passed parameters.
+     *
+     * @param atomicityMode atomicity mode
+     * @param mode cache mode
+     * @param writeSyncMode cache write synchronization mode
+     * @param backupsCount backups count
+     * @throws Exception
+     */
+    private void initGrid(CacheAtomicityMode atomicityMode, CacheMode mode,
+        CacheWriteSynchronizationMode writeSyncMode, int backupsCount) throws Exception {
+
+        this.atomicityMode = atomicityMode;
+        this.mode = mode;
+        this.backupsCount = backupsCount;
+        this.writeSyncMode = writeSyncMode;
+
+        Ignition.setClientMode(false);
+
+        for (int i = 0; i < NODES; i++)
+            startGrid(i);
+
+        Ignition.setClientMode(true);
+
+        client = startGrid(NODES + 1);
+
+        // it is required to start first node in test jvm, but we can not start client node,
+        // because client will fail to connect and test will fail too.
+        // as workaround start first server node in test jvm and then stop it.
+        stopGrid(0);
+    }
+
+
+    /** */
+    public void forceOOM(boolean useStreamer, TransactionConcurrency concurrency,
+        TransactionIsolation isolation) throws Exception {
+        final IgniteCache<Object, Object> cache = client.cache(DEFAULT_CACHE_NAME);
+
+        IgniteDataStreamer<String, String> streamer = client.dataStreamer(DEFAULT_CACHE_NAME);
+
+        Map<String, String> map = new HashMap<>();
+
+        Transaction tx = null;
+
+        for (int i = 0; i < Integer.MAX_VALUE; i++) {
+            map.put("k" + i, "v" + i);
+
+            if (map.size() > 1_000) {
+                if (concurrency != null && isolation != null)
+                    tx = client.transactions().txStart(concurrency, isolation);
+
+                if (useStreamer)
+                    streamer.addData(map);
+                else
+                    cache.putAll(map);
+
+                map.clear();
+
+                if (tx != null) {
+                    tx.commit();
+                    tx.close();
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isMultiJvm() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isRemoteJvm(String igniteInstanceName) {
+        return !(Ignition.isClientMode() || igniteInstanceName.endsWith("0"));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        DataStorageConfiguration memCfg = new DataStorageConfiguration();
+
+        memCfg.setDefaultDataRegionConfiguration(new DataRegionConfiguration()
+            .setMaxSize(10 * 1024 * 1024 + 1));
+
+        cfg.setDataStorageConfiguration(memCfg);
+
+        CacheConfiguration<Object, Object> baseCfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME);
+
+        baseCfg.setAtomicityMode(this.atomicityMode);
+        baseCfg.setCacheMode(this.mode);
+        baseCfg.setBackups(this.backupsCount);
+        baseCfg.setWriteSynchronizationMode(this.writeSyncMode);
+
+        cfg.setCacheConfiguration(baseCfg);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/918febaa/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
index 7c71381..8a2d6a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.IgniteOutOfMemoryPropagationTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheExchangeMergeTest;
 import org.apache.ignite.internal.processors.cache.distributed.CachePartitionStateTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionEvictionDuringReadThroughSelfTest;
@@ -54,6 +55,10 @@ public class IgniteCacheTestSuite6 extends TestSuite {
         suite.addTestSuite(TxRollbackOnTimeoutNearCacheTest.class);
         suite.addTestSuite(IgniteCacheThreadLocalTxTest.class);
 
+
+//        TODO enable this test after IGNITE-6753, now it takes too long
+//        suite.addTestSuite(IgniteOutOfMemoryPropagationTest.class);
+
         return suite;
     }
 }


[3/7] ignite git commit: IGNITE-6751: Skipped upload of "ignite-dev-utils" module to Maven repository. This closes #2926.

Posted by sb...@apache.org.
IGNITE-6751: Skipped upload of "ignite-dev-utils" module to Maven repository. This closes #2926.


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

Branch: refs/heads/ignite-6667
Commit: 5fd13f5d60923eedbbb67743ed8a834d99529d6b
Parents: 6ed872b
Author: Oleg Ostanin <oo...@gridgain.com>
Authored: Wed Oct 25 18:48:48 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Oct 25 18:48:48 2017 +0300

----------------------------------------------------------------------
 modules/dev-utils/pom.xml | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5fd13f5d/modules/dev-utils/pom.xml
----------------------------------------------------------------------
diff --git a/modules/dev-utils/pom.xml b/modules/dev-utils/pom.xml
index 40e58b8..8fc0b68 100644
--- a/modules/dev-utils/pom.xml
+++ b/modules/dev-utils/pom.xml
@@ -47,4 +47,17 @@
             <version>${project.version}</version>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <version>2.8.2</version>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
 </project>


[5/7] ignite git commit: IGNITE-6748 Moved checkpoint buffer size to DataRegionConfiguration

Posted by sb...@apache.org.
IGNITE-6748 Moved checkpoint buffer size to DataRegionConfiguration


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

Branch: refs/heads/ignite-6667
Commit: 8266a98127d000fc4ec3ee71c05ed9671e90f70e
Parents: 2d0e91e
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Oct 26 10:16:35 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 26 10:16:35 2017 +0300

----------------------------------------------------------------------
 .../configuration/DataRegionConfiguration.java  | 34 +++++++++-
 .../configuration/DataStorageConfiguration.java | 29 --------
 .../MemoryPolicyConfiguration.java              | 23 -------
 .../org/apache/ignite/internal/IgnitionEx.java  | 13 ++--
 .../discovery/GridDiscoveryManager.java         |  8 ++-
 .../GridCacheDatabaseSharedManager.java         | 71 +++++++-------------
 .../utils/PlatformConfigurationUtils.java       |  6 +-
 .../node/VisorDataRegionConfiguration.java      | 13 ++++
 .../node/VisorDataStorageConfiguration.java     |  3 +-
 .../node/VisorPersistentStoreConfiguration.java |  1 -
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |  7 +-
 .../pagemem/PagesWriteThrottleSandboxTest.java  |  2 +-
 .../pagemem/PagesWriteThrottleSmokeTest.java    |  2 +-
 .../Config/full-config.xml                      |  6 +-
 .../IgniteConfigurationSerializerTest.cs        |  6 +-
 .../IgniteConfigurationTest.cs                  |  6 +-
 .../Configuration/DataRegionConfiguration.cs    |  8 +++
 .../Configuration/DataStorageConfiguration.cs   |  9 ---
 .../IgniteConfigurationSection.xsd              | 15 +++--
 modules/web-console/backend/app/mongo.js        |  6 +-
 .../generator/ConfigurationGenerator.js         |  5 +-
 .../generator/defaults/Cluster.service.js       |  3 +-
 .../configuration/clusters/data-storage.pug     |  6 ++
 23 files changed, 134 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
index 50edf5c..4ae87e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataRegionConfiguration.java
@@ -129,6 +129,9 @@ public final class DataRegionConfiguration implements Serializable {
      */
     private boolean persistenceEnabled = false;
 
+    /** Temporary buffer size for checkpoints in bytes. */
+    private long checkpointPageBufSize;
+
     /**
      * Gets data region name.
      *
@@ -212,11 +215,11 @@ public final class DataRegionConfiguration implements Serializable {
     /**
      * Sets a path to the memory-mapped files.
      *
-     * @param swapFilePath A Path to the memory mapped file.
+     * @param swapPath A Path to the memory mapped file.
      * @return {@code this} for chaining.
      */
-    public DataRegionConfiguration setSwapPath(String swapFilePath) {
-        this.swapPath = swapFilePath;
+    public DataRegionConfiguration setSwapPath(String swapPath) {
+        this.swapPath = swapPath;
 
         return this;
     }
@@ -403,4 +406,29 @@ public final class DataRegionConfiguration implements Serializable {
 
         return this;
     }
+
+    /**
+     * Gets amount of memory allocated for a checkpoint temporary buffer.
+     *
+     * @return Checkpoint page buffer size in bytes or {@code 0} for Ignite
+     *      to choose the buffer size automatically.
+     */
+    public long getCheckpointPageBufferSize() {
+        return checkpointPageBufSize;
+    }
+
+    /**
+     * Sets amount of memory allocated for the checkpoint temporary buffer. The buffer is used to create temporary
+     * copies of pages that are being written to disk and being update in parallel while the checkpoint is in
+     * progress.
+     *
+     * @param checkpointPageBufSize Checkpoint page buffer size in bytes or {@code 0} for Ignite to
+     *      choose the buffer size automatically.
+     * @return {@code this} for chaining.
+     */
+    public DataRegionConfiguration setCheckpointPageBufferSize(long checkpointPageBufSize) {
+        this.checkpointPageBufSize = checkpointPageBufSize;
+
+        return this;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
index bd314ab..8202ef8 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
@@ -173,9 +173,6 @@ public class DataStorageConfiguration implements Serializable {
     private long lockWaitTime = DFLT_LOCK_WAIT_TIME;
 
     /** */
-    private long checkpointPageBufSize;
-
-    /** */
     private int checkpointThreads = DFLT_CHECKPOINT_THREADS;
 
     /** Checkpoint write order. */
@@ -425,32 +422,6 @@ public class DataStorageConfiguration implements Serializable {
     }
 
     /**
-     * Gets amount of memory allocated for a checkpoint temporary buffer.
-     *
-     * @return Checkpoint page buffer size in bytes or {@code 0} for Ignite
-     *      to choose the buffer size automatically.
-     */
-    public long getCheckpointPageBufferSize() {
-        return checkpointPageBufSize;
-    }
-
-    /**
-     * Sets amount of memory allocated for the checkpoint temporary buffer. The buffer is used to create temporary
-     * copies of pages that are being written to disk and being update in parallel while the checkpoint is in
-     * progress.
-     *
-     * @param checkpointPageBufSize Checkpoint page buffer size in bytes or {@code 0} for Ignite to
-     *      choose the buffer size automatically.
-     * @return {@code this} for chaining.
-     */
-    public DataStorageConfiguration setCheckpointPageBufferSize(long checkpointPageBufSize) {
-        this.checkpointPageBufSize = checkpointPageBufSize;
-
-        return this;
-    }
-
-
-    /**
      * Gets a number of threads to use for the checkpoint purposes.
      *
      * @return Number of checkpoint threads.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index efe7ae2..a1a822f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -124,11 +124,6 @@ public final class MemoryPolicyConfiguration implements Serializable {
     private long rateTimeInterval = DFLT_RATE_TIME_INTERVAL_MILLIS;
 
     /**
-     * Flag to enable Ignite Native Persistence.
-     */
-    private boolean persistenceEnabled = true;
-
-    /**
      * Gets memory policy name.
      *
      * @return Memory policy name.
@@ -320,24 +315,6 @@ public final class MemoryPolicyConfiguration implements Serializable {
     }
 
     /**
-     * Gets whether Ignite Native Persistence is enabled for this memory policy.
-     *
-     * @return Persistence enabled flag.
-     */
-    public boolean isPersistenceEnabled() {
-        return persistenceEnabled;
-    }
-
-    /**
-     * Sets persistence enabled flag.
-     *
-     * @param persistenceEnabled Persistence enabled flag.
-     */
-    public void setPersistenceEnabled(boolean persistenceEnabled) {
-        this.persistenceEnabled = persistenceEnabled;
-    }
-
-    /**
      * Gets time interval for {@link MemoryMetrics#getAllocationRate()}
      * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
      * <p>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 36257e2..67c771b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2779,7 +2779,9 @@ public class IgnitionEx {
      */
     private static void convertLegacyDataStorageConfigurationToNew(
         IgniteConfiguration cfg) throws IgniteCheckedException {
-        boolean persistenceEnabled = cfg.getPersistentStoreConfiguration() != null;
+        PersistentStoreConfiguration psCfg = cfg.getPersistentStoreConfiguration();
+
+        boolean persistenceEnabled = psCfg != null;
 
         DataStorageConfiguration dsCfg = new DataStorageConfiguration();
 
@@ -2814,6 +2816,9 @@ public class IgnitionEx {
                 region.setSwapPath(mpc.getSwapFilePath());
                 region.setMetricsEnabled(mpc.isMetricsEnabled());
 
+                if (persistenceEnabled)
+                    region.setCheckpointPageBufferSize(psCfg.getCheckpointingPageBufferSize());
+
                 if (mpc.getName() == null) {
                     throw new IgniteCheckedException(new IllegalArgumentException(
                         "User-defined MemoryPolicyConfiguration must have non-null and non-empty name."));
@@ -2829,7 +2834,8 @@ public class IgnitionEx {
         }
 
         if (!optionalDataRegions.isEmpty())
-            dsCfg.setDataRegionConfigurations(optionalDataRegions.toArray(new DataRegionConfiguration[optionalDataRegions.size()]));
+            dsCfg.setDataRegionConfigurations(optionalDataRegions.toArray(
+                new DataRegionConfiguration[optionalDataRegions.size()]));
 
         if (!customDfltPlc) {
             if (!DFLT_MEM_PLC_DEFAULT_NAME.equals(memCfg.getDefaultMemoryPolicyName())) {
@@ -2848,10 +2854,7 @@ public class IgnitionEx {
         }
 
         if (persistenceEnabled) {
-            PersistentStoreConfiguration psCfg = cfg.getPersistentStoreConfiguration();
-
             dsCfg.setCheckpointFrequency(psCfg.getCheckpointingFrequency());
-            dsCfg.setCheckpointPageBufferSize(psCfg.getCheckpointingPageBufferSize());
             dsCfg.setCheckpointThreads(psCfg.getCheckpointingThreads());
             dsCfg.setCheckpointWriteOrder(psCfg.getCheckpointWriteOrder());
             dsCfg.setFileIOFactory(psCfg.getFileIOFactory());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index a3b157d..77b0622 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -1543,14 +1543,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         DataRegionConfiguration[] dataRegions = memCfg.getDataRegionConfigurations();
 
         if (dataRegions != null) {
-            for (DataRegionConfiguration dataReg : dataRegions)
+            for (DataRegionConfiguration dataReg : dataRegions) {
                 res += dataReg.getMaxSize();
+
+                res += GridCacheDatabaseSharedManager.checkpointBufferSize(dataReg);
+            }
         }
 
         res += memCfg.getDefaultDataRegionConfiguration().getMaxSize();
 
-        // Add persistence (if any).
-        res += GridCacheDatabaseSharedManager.checkpointBufferSize(ctx.config());
+        res += GridCacheDatabaseSharedManager.checkpointBufferSize(memCfg.getDefaultDataRegionConfiguration());
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/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 de3b60a..920af17 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
@@ -156,8 +156,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** */
     public static final String IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC = "IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC";
 
-    /** Default checkpointing page buffer size (may be adjusted by Ignite). */
-    public static final Long DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE = 256L * 1024 * 1024;
+    /** */
+    private static final long GB = 1024L * 1024 * 1024;
+
+    /** Minimum checkpointing page buffer size (may be adjusted by Ignite). */
+    public static final Long DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE = GB / 4;
+
+    /** Default minimum checkpointing page buffer size (may be adjusted by Ignite). */
+    public static final Long DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE = 2 * GB;
 
     /** Skip sync. */
     private final boolean skipSync = IgniteSystemProperties.getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC);
@@ -249,9 +255,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private long checkpointFreq;
 
     /** */
-    private long checkpointPageBufSize;
-
-    /** */
     private FilePageStoreManager storeMgr;
 
     /** Checkpoint metadata directory ("cp"), contains files with checkpoint start and end */
@@ -408,56 +411,27 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 30_000,
                 new LinkedBlockingQueue<Runnable>()
             );
-
-        checkpointPageBufSize = checkpointBufferSize(cctx.kernalContext().config());
     }
 
     /**
      * Get checkpoint buffer size for the given configuration.
      *
-     * @param cfg Configuration.
+     * @param regCfg Configuration.
      * @return Checkpoint buffer size.
      */
-    public static long checkpointBufferSize(IgniteConfiguration cfg) {
-        DataStorageConfiguration persistenceCfg = cfg.getDataStorageConfiguration();
-
-        if (persistenceCfg == null)
+    public static long checkpointBufferSize(DataRegionConfiguration regCfg) {
+        if (!regCfg.isPersistenceEnabled())
             return 0L;
 
-        long res = persistenceCfg.getCheckpointPageBufferSize();
+        long res = regCfg.getCheckpointPageBufferSize();
 
         if (res == 0L) {
-            res = DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
-
-            DataStorageConfiguration memCfg = cfg.getDataStorageConfiguration();
-
-            assert memCfg != null;
-
-            long totalSize = memCfg.getSystemRegionMaxSize();
-
-            if (memCfg.getDataRegionConfigurations() == null)
-                totalSize += DataStorageConfiguration.DFLT_DATA_REGION_MAX_SIZE;
-            else {
-                for (DataRegionConfiguration memPlc : memCfg.getDataRegionConfigurations()) {
-                    if (Long.MAX_VALUE - memPlc.getMaxSize() > totalSize)
-                        totalSize += memPlc.getMaxSize();
-                    else {
-                        totalSize = Long.MAX_VALUE;
-
-                        break;
-                    }
-                }
-
-                assert totalSize > 0;
-            }
-
-            // Limit the checkpoint page buffer size by 2GB.
-            long dfltSize = 2 * 1024L * 1024L * 1024L;
-
-            long adjusted = Math.min(totalSize / 4, dfltSize);
-
-            if (res < adjusted)
-                res = adjusted;
+            if (regCfg.getMaxSize() < GB)
+                res = Math.min(DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE, regCfg.getMaxSize());
+            else if (regCfg.getMaxSize() < 8 * GB)
+                res = regCfg.getMaxSize() / 4;
+            else
+                res = DFLT_MAX_CHECKPOINTING_PAGE_BUFFER_SIZE;
         }
 
         return res;
@@ -689,13 +663,16 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         long cacheSize = plcCfg.getMaxSize();
 
         // Checkpoint buffer size can not be greater than cache size, it does not make sense.
-        long chpBufSize = Math.min(checkpointPageBufSize, cacheSize);
+        long chpBufSize = checkpointBufferSize(plcCfg);
 
-        if (checkpointPageBufSize > cacheSize)
+        if (chpBufSize > cacheSize) {
             U.quietAndInfo(log,
-                "Checkpoint page buffer size is too big, setting to an adjusted cache size [size="
+                "Configured checkpoint page buffer size is too big, setting to the max region size [size="
                     + U.readableSize(cacheSize, false) + ",  memPlc=" + plcCfg.getName() + ']');
 
+            chpBufSize = cacheSize;
+        }
+
         boolean writeThrottlingEnabled = persistenceCfg.isWriteThrottlingEnabled();
 
         if (IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_OVERRIDE_WRITE_THROTTLING_ENABLED, false))

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 28b6c60..c1a807c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -1668,7 +1668,6 @@ public class PlatformConfigurationUtils {
         DataStorageConfiguration res = new DataStorageConfiguration()
                 .setStoragePath(in.readString())
                 .setCheckpointFrequency(in.readLong())
-                .setCheckpointPageBufferSize(in.readLong())
                 .setCheckpointThreads(in.readInt())
                 .setLockWaitTime((int) in.readLong())
                 .setWalHistorySize(in.readInt())
@@ -1763,7 +1762,6 @@ public class PlatformConfigurationUtils {
 
             w.writeString(cfg.getStoragePath());
             w.writeLong(cfg.getCheckpointFrequency());
-            w.writeLong(cfg.getCheckpointPageBufferSize());
             w.writeInt(cfg.getCheckpointThreads());
             w.writeLong(cfg.getLockWaitTime());
             w.writeInt(cfg.getWalHistorySize());
@@ -1828,6 +1826,7 @@ public class PlatformConfigurationUtils {
         w.writeBoolean(cfg.isMetricsEnabled());
         w.writeInt(cfg.getMetricsSubIntervalCount());
         w.writeLong(cfg.getMetricsRateTimeInterval());
+        w.writeLong(cfg.getCheckpointPageBufferSize());
     }
 
     /**
@@ -1849,7 +1848,8 @@ public class PlatformConfigurationUtils {
                 .setEmptyPagesPoolSize(r.readInt())
                 .setMetricsEnabled(r.readBoolean())
                 .setMetricsSubIntervalCount(r.readInt())
-                .setMetricsRateTimeInterval(r.readLong());
+                .setMetricsRateTimeInterval(r.readLong())
+                .setCheckpointPageBufferSize(r.readLong());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataRegionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataRegionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataRegionConfiguration.java
index 394e294..179e789 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataRegionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataRegionConfiguration.java
@@ -68,6 +68,9 @@ public class VisorDataRegionConfiguration extends VisorDataTransferObject {
     /** Enable Ignite Native Persistence. */
     private boolean persistenceEnabled;
 
+    /** Temporary buffer size for checkpoints in bytes. */
+    private long checkpointPageBufSize;
+
     /**
      * Default constructor.
      */
@@ -94,6 +97,7 @@ public class VisorDataRegionConfiguration extends VisorDataTransferObject {
         metricsSubIntervalCount = plc.getMetricsSubIntervalCount();
         metricsRateTimeInterval = plc.getMetricsRateTimeInterval();
         persistenceEnabled = plc.isPersistenceEnabled();
+        checkpointPageBufSize = plc.getCheckpointPageBufferSize();
     }
 
     /**
@@ -188,6 +192,13 @@ public class VisorDataRegionConfiguration extends VisorDataTransferObject {
         return persistenceEnabled;
     }
 
+    /**
+     * @return Amount of memory allocated for a checkpoint temporary buffer in bytes.
+     */
+    public long getCheckpointPageBufferSize() {
+        return checkpointPageBufSize;
+    }
+
     /** {@inheritDoc} */
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, name);
@@ -201,6 +212,7 @@ public class VisorDataRegionConfiguration extends VisorDataTransferObject {
         out.writeInt(metricsSubIntervalCount);
         out.writeLong(metricsRateTimeInterval);
         out.writeBoolean(persistenceEnabled);
+        out.writeLong(checkpointPageBufSize);
     }
 
     /** {@inheritDoc} */
@@ -216,6 +228,7 @@ public class VisorDataRegionConfiguration extends VisorDataTransferObject {
         metricsSubIntervalCount = in.readInt();
         metricsRateTimeInterval = in.readLong();
         persistenceEnabled = in.readBoolean();
+        checkpointPageBufSize = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
index 78bf1c5..8470fe1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorDataStorageConfiguration.java
@@ -155,7 +155,6 @@ public class VisorDataStorageConfiguration extends VisorDataTransferObject {
         storagePath = cfg.getStoragePath();
         checkpointFreq = cfg.getCheckpointFrequency();
         lockWaitTime = cfg.getLockWaitTime();
-        checkpointPageBufSize = cfg.getCheckpointPageBufferSize();
         checkpointThreads = cfg.getCheckpointThreads();
         checkpointWriteOrder = cfg.getCheckpointWriteOrder();
         walHistSize = cfg.getWalHistorySize();
@@ -391,7 +390,7 @@ public class VisorDataStorageConfiguration extends VisorDataTransferObject {
         U.writeString(out, storagePath);
         out.writeLong(checkpointFreq);
         out.writeLong(lockWaitTime);
-        out.writeLong(checkpointPageBufSize);
+        out.writeLong(0);
         out.writeInt(checkpointThreads);
         U.writeEnum(out, checkpointWriteOrder);
         out.writeInt(walHistSize);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
index f9d7a64..d26ab35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
@@ -103,7 +103,6 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         persistenceStorePath = cfg.getStoragePath();
         checkpointingFreq = cfg.getCheckpointFrequency();
         lockWaitTime = cfg.getLockWaitTime();
-        checkpointingPageBufSize = cfg.getCheckpointPageBufferSize();
         checkpointingThreads = cfg.getCheckpointThreads();
         walHistSize = cfg.getWalHistorySize();
         walSegments = cfg.getWalSegments();

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
index 5700eb3..3b76b63 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
@@ -54,9 +54,10 @@ public class IgnitePdsWalTlbTest extends GridCommonAbstractTest {
 
         DataStorageConfiguration memCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(
-                new DataRegionConfiguration().setMaxSize(100 * 1024 * 1024).setPersistenceEnabled(true))
+                new DataRegionConfiguration().setMaxSize(100 * 1024 * 1024)
+                    .setPersistenceEnabled(true)
+                    .setCheckpointPageBufferSize(DFLT_MIN_CHECKPOINTING_PAGE_BUFFER_SIZE + 1))
             .setWalMode(WALMode.LOG_ONLY)
-            .setCheckpointPageBufferSize(DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE + 1)
             .setWalThreadLocalBufferSize(640000000);
 
         cfg.setDataStorageConfiguration(memCfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
index 30fb492..9529f59 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
@@ -68,12 +68,12 @@ public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
         DataStorageConfiguration dbCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
                 .setMaxSize(4000L * 1024 * 1024)
+                .setCheckpointPageBufferSize(1000L * 1000 * 1000)
                 .setName("dfltDataRegion")
                 .setMetricsEnabled(true)
                 .setPersistenceEnabled(true))
             .setWalMode(WALMode.BACKGROUND)
             .setCheckpointFrequency(20_000)
-            .setCheckpointPageBufferSize(1000L * 1000 * 1000)
             .setWriteThrottlingEnabled(true);
 
         cfg.setDataStorageConfiguration(dbCfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
index ab7aab4..1875cfb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
@@ -78,12 +78,12 @@ public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
         DataStorageConfiguration dbCfg = new DataStorageConfiguration()
             .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
                 .setMaxSize(400 * 1024 * 1024)
+                .setCheckpointPageBufferSize(200 * 1000 * 1000)
                 .setName("dfltDataRegion")
                 .setMetricsEnabled(true)
                 .setPersistenceEnabled(true))
             .setWalMode(WALMode.BACKGROUND)
             .setCheckpointFrequency(20_000)
-            .setCheckpointPageBufferSize(200 * 1000 * 1000)
             .setWriteThrottlingEnabled(true)
             .setCheckpointThreads(1)
             .setFileIOFactory(new SlowCheckpointFileIOFactory());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
index 03559bf..b808bbe 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/full-config.xml
@@ -136,7 +136,7 @@
             <listener type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyEventListener' />
         </localEventListener>
     </localEventListeners>
-    <dataStorageConfiguration alwaysWriteFullPages="false" checkpointFrequency="00:00:01" checkpointPageBufferSize="2"
+    <dataStorageConfiguration alwaysWriteFullPages="false" checkpointFrequency="00:00:01"
                               checkpointThreads="3" concurrencyLevel="4" lockWaitTime="00:00:05" metricsEnabled="true"
                               pageSize="6" storagePath="cde" metricsRateTimeInterval="00:00:07"
                               metricsSubIntervalCount="8" systemRegionInitialSize="9" systemRegionMaxSize="10" 
@@ -147,11 +147,11 @@
         <dataRegionConfigurations>
             <dataRegionConfiguration emptyPagesPoolSize="1" evictionThreshold="2" initialSize="3" metricsEnabled="true"
                                      maxSize="4" name="reg2" pageEvictionMode="RandomLru" metricsRateTimeInterval="00:00:01"
-                                     metricsSubIntervalCount="5" swapPath="swap" />
+                                     metricsSubIntervalCount="5" swapPath="swap" checkpointPageBufferSize="7" />
         </dataRegionConfigurations>
         <defaultDataRegionConfiguration emptyPagesPoolSize="2" evictionThreshold="3" initialSize="4"
                                         maxSize="5" metricsEnabled="false" name="reg1" pageEvictionMode="Disabled"
                                         metricsRateTimeInterval="00:00:03" metricsSubIntervalCount="6"
-                                        swapPath="swap2" />
+                                        swapPath="swap2" checkpointPageBufferSize="8" />
     </dataStorageConfiguration>
 </igniteConfiguration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index b8c1069..4cd3760 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -287,7 +287,6 @@ namespace Apache.Ignite.Core.Tests
             var ds = cfg.DataStorageConfiguration;
             Assert.IsFalse(ds.AlwaysWriteFullPages);
             Assert.AreEqual(TimeSpan.FromSeconds(1), ds.CheckpointFrequency);
-            Assert.AreEqual(2, ds.CheckpointPageBufferSize);
             Assert.AreEqual(3, ds.CheckpointThreads);
             Assert.AreEqual(4, ds.ConcurrencyLevel);
             Assert.AreEqual(TimeSpan.FromSeconds(5), ds.LockWaitTime);
@@ -321,6 +320,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(5, dr.MetricsSubIntervalCount);
             Assert.AreEqual("swap", dr.SwapPath);
             Assert.IsTrue(dr.MetricsEnabled);
+            Assert.AreEqual(7, dr.CheckpointPageBufferSize);
 
             dr = ds.DefaultDataRegionConfiguration;
             Assert.AreEqual(2, dr.EmptyPagesPoolSize);
@@ -927,7 +927,6 @@ namespace Apache.Ignite.Core.Tests
                 {
                     AlwaysWriteFullPages = true,
                     CheckpointFrequency = TimeSpan.FromSeconds(25),
-                    CheckpointPageBufferSize = 28 * 1024 * 1024,
                     CheckpointThreads = 2,
                     LockWaitTime = TimeSpan.FromSeconds(5),
                     StoragePath = Path.GetTempPath(),
@@ -962,7 +961,8 @@ namespace Apache.Ignite.Core.Tests
                         PersistenceEnabled = false,
                         MetricsRateTimeInterval = TimeSpan.FromMinutes(2),
                         MetricsSubIntervalCount = 6,
-                        SwapPath = Path.GetTempPath()
+                        SwapPath = Path.GetTempPath(),
+                        CheckpointPageBufferSize = 7
                     },
                     DataRegionConfigurations = new[]
                     {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index c8c06b2..f68371a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -543,7 +543,6 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(DataStorageConfiguration.DefaultTlbSize, cfg.WalThreadLocalBufferSize);
             Assert.AreEqual(DataStorageConfiguration.DefaultCheckpointFrequency, cfg.CheckpointFrequency);
             Assert.AreEqual(DataStorageConfiguration.DefaultCheckpointThreads, cfg.CheckpointThreads);
-            Assert.AreEqual(default(long), cfg.CheckpointPageBufferSize);
             Assert.AreEqual(DataStorageConfiguration.DefaultLockWaitTime, cfg.LockWaitTime);
             Assert.AreEqual(DataStorageConfiguration.DefaultWalFlushFrequency, cfg.WalFlushFrequency);
             Assert.AreEqual(DataStorageConfiguration.DefaultWalFsyncDelayNanos, cfg.WalFsyncDelayNanos);
@@ -580,6 +579,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(DataRegionConfiguration.DefaultPersistenceEnabled, cfg.PersistenceEnabled);
             Assert.AreEqual(DataRegionConfiguration.DefaultMetricsRateTimeInterval, cfg.MetricsRateTimeInterval);
             Assert.AreEqual(DataRegionConfiguration.DefaultMetricsSubIntervalCount, cfg.MetricsSubIntervalCount);
+            Assert.AreEqual(default(long), cfg.CheckpointPageBufferSize);
         }
 
         /// <summary>
@@ -769,7 +769,6 @@ namespace Apache.Ignite.Core.Tests
                 {
                     AlwaysWriteFullPages = true,
                     CheckpointFrequency = TimeSpan.FromSeconds(25),
-                    CheckpointPageBufferSize = 28 * 1024 * 1024,
                     CheckpointThreads = 2,
                     LockWaitTime = TimeSpan.FromSeconds(5),
                     StoragePath = Path.GetTempPath(),
@@ -804,7 +803,8 @@ namespace Apache.Ignite.Core.Tests
                         PersistenceEnabled = false,
                         MetricsRateTimeInterval = TimeSpan.FromMinutes(2),
                         MetricsSubIntervalCount = 6,
-                        SwapPath = IgniteUtils.GetTempDirectoryName()
+                        SwapPath = IgniteUtils.GetTempDirectoryName(),
+                        CheckpointPageBufferSize = 28 * 1024 * 1024
                     },
                     DataRegionConfigurations = new[]
                     {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
index 5c4240e..d20ce49 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataRegionConfiguration.cs
@@ -104,6 +104,7 @@ namespace Apache.Ignite.Core.Configuration
             MetricsEnabled = reader.ReadBoolean();
             MetricsSubIntervalCount = reader.ReadInt();
             MetricsRateTimeInterval = reader.ReadLongAsTimespan();
+            CheckpointPageBufferSize = reader.ReadLong();
         }
 
         /// <summary>
@@ -122,6 +123,7 @@ namespace Apache.Ignite.Core.Configuration
             writer.WriteBoolean(MetricsEnabled);
             writer.WriteInt(MetricsSubIntervalCount);
             writer.WriteTimeSpanAsLong(MetricsRateTimeInterval);
+            writer.WriteLong(CheckpointPageBufferSize);
         }
 
         /// <summary>
@@ -209,5 +211,11 @@ namespace Apache.Ignite.Core.Configuration
             Justification = "Consistency with Java config")]
         public int MetricsSubIntervalCount { get; set; }
 
+        /// <summary>
+        /// Gets or sets the size of the checkpointing page buffer.
+        /// <para />
+        /// Default is <c>0</c>: Ignite will choose buffer size automatically.
+        /// </summary>
+        public long CheckpointPageBufferSize { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
index 17b4ada..09b3fe4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/DataStorageConfiguration.cs
@@ -189,7 +189,6 @@ namespace Apache.Ignite.Core.Configuration
 
             StoragePath = reader.ReadString();
             CheckpointFrequency = reader.ReadLongAsTimespan();
-            CheckpointPageBufferSize = reader.ReadLong();
             CheckpointThreads = reader.ReadInt();
             LockWaitTime = reader.ReadLongAsTimespan();
             WalHistorySize = reader.ReadInt();
@@ -239,7 +238,6 @@ namespace Apache.Ignite.Core.Configuration
 
             writer.WriteString(StoragePath);
             writer.WriteTimeSpanAsLong(CheckpointFrequency);
-            writer.WriteLong(CheckpointPageBufferSize);
             writer.WriteInt(CheckpointThreads);
             writer.WriteTimeSpanAsLong(LockWaitTime);
             writer.WriteInt(WalHistorySize);
@@ -308,13 +306,6 @@ namespace Apache.Ignite.Core.Configuration
         public TimeSpan CheckpointFrequency { get; set; }
 
         /// <summary>
-        /// Gets or sets the size of the checkpointing page buffer.
-        /// <para />
-        /// Default is <c>0</c>: Ignite will choose buffer size automatically.
-        /// </summary>
-        public long CheckpointPageBufferSize { get; set; }
-
-        /// <summary>
         /// Gets or sets the number of threads for checkpointing.
         /// </summary>
         [DefaultValue(DefaultCheckpointThreads)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 70b1fc4..5050806 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1622,6 +1622,11 @@
                                             <xs:documentation>Enable disk persistence for this region.</xs:documentation>
                                         </xs:annotation>
                                     </xs:attribute>
+                                    <xs:attribute name="checkpointPageBufferSize" type="xs:long">
+                                        <xs:annotation>
+                                            <xs:documentation>Size of the checkpointing page buffer.</xs:documentation>
+                                        </xs:annotation>
+                                    </xs:attribute>
                                 </xs:complexType>
                             </xs:element>
                             <xs:element name="dataRegionConfigurations">
@@ -1690,6 +1695,11 @@
                                                         <xs:documentation>Enable disk persistence for this region.</xs:documentation>
                                                     </xs:annotation>
                                                 </xs:attribute>
+                                                <xs:attribute name="checkpointPageBufferSize" type="xs:long">
+                                                    <xs:annotation>
+                                                        <xs:documentation>Size of the checkpointing page buffer.</xs:documentation>
+                                                    </xs:annotation>
+                                                </xs:attribute>
                                             </xs:complexType>                                            
                                         </xs:element>
                                     </xs:sequence>
@@ -1706,11 +1716,6 @@
                                 <xs:documentation>Checkpointing frequency which is a minimal interval when the dirty pages will be written to the Persistent Store.</xs:documentation>
                             </xs:annotation>
                         </xs:attribute>
-                        <xs:attribute name="checkpointPageBufferSize" type="xs:long">
-                            <xs:annotation>
-                                <xs:documentation>Size of the checkpointing page buffer.</xs:documentation>
-                            </xs:annotation>
-                        </xs:attribute>
                         <xs:attribute name="checkpointThreads" type="xs:int">
                             <xs:annotation>
                                 <xs:documentation>Number of threads for checkpointing.</xs:documentation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/web-console/backend/app/mongo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js
index b4bc9fc..5b02a72 100644
--- a/modules/web-console/backend/app/mongo.js
+++ b/modules/web-console/backend/app/mongo.js
@@ -977,7 +977,8 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
                 metricsEnabled: Boolean,
                 metricsSubIntervalCount: Number,
                 metricsRateTimeInterval: Number,
-                persistenceEnabled: Boolean
+                persistenceEnabled: Boolean,
+                checkpointPageBufferSize: Number
             },
             dataRegionConfigurations: [{
                 name: String,
@@ -990,7 +991,8 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
                 metricsEnabled: Boolean,
                 metricsSubIntervalCount: Number,
                 metricsRateTimeInterval: Number,
-                persistenceEnabled: Boolean
+                persistenceEnabled: Boolean,
+                checkpointPageBufferSize: Number
             }],
             storagePath: String,
             metricsEnabled: Boolean,

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index d9342e7..1b12d52 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -1345,7 +1345,7 @@ export default class IgniteConfigurationGenerator {
     static dataRegionConfiguration(dataRegionCfg) {
         const plcBean = new Bean('org.apache.ignite.configuration.DataRegionConfiguration', 'dataRegionCfg', dataRegionCfg, clusterDflts.dataStorageConfiguration.dataRegionConfigurations);
 
-        return plcBean.stringProperty('name')
+        plcBean.stringProperty('name')
             .longProperty('initialSize')
             .longProperty('maxSize')
             .stringProperty('swapPath')
@@ -1354,8 +1354,11 @@ export default class IgniteConfigurationGenerator {
             .intProperty('emptyPagesPoolSize')
             .intProperty('metricsSubIntervalCount')
             .longProperty('metricsRateTimeInterval')
+            .longProperty('checkpointPageBufferSize')
             .boolProperty('metricsEnabled')
             .boolProperty('persistenceEnabled');
+
+        return plcBean;
     }
 
     // Generate data storage configuration.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
index bafb202..f636869 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
@@ -327,7 +327,8 @@ const DFLT_CLUSTER = {
             emptyPagesPoolSize: 100,
             metricsEnabled: false,
             metricsSubIntervalCount: 5,
-            metricsRateTimeInterval: 60000
+            metricsRateTimeInterval: 60000,
+            checkpointPageBufferSize: 0
         },
         metricsEnabled: false,
         alwaysWriteFullPages: false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/8266a981/modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug
index a635739..82c6dbe 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/data-storage.pug
@@ -102,6 +102,9 @@ include /app/helpers/jade/mixins
                                             +number('Metrics rate time interval:', dfltRegionModel + '.metricsRateTimeInterval', '"DfltRegionRateTimeInterval" + $index',
                                             'true', '60000', '1000', 'Time interval for allocation rate and eviction rate monitoring purposes')
                                         .details-row
+                                            +number('Checkpoint page buffer:', dfltRegionModel + '.checkpointPageBufferSize', '"DfltCheckpointPageBufferSize" + $index',
+                                                'true', '0', '0', 'Amount of memory allocated for a checkpoint temporary buffer in bytes')
+                                        .details-row
                                             +checkbox('Metrics enabled', dfltRegionModel + '.metricsEnabled', '"DfltRegionMetricsEnabled" + $index',
                                             'Whether memory metrics are enabled by default on node startup')
                                         .details-row
@@ -156,6 +159,9 @@ include /app/helpers/jade/mixins
                                         .settings-row
                                             +number('Metrics rate time interval:', 'model.metricsRateTimeInterval', '"DataRegionRateTimeInterval" + $index',
                                                 'true', '60000', '1000', 'Time interval for allocation rate and eviction rate monitoring purposes')
+                                        .details-row
+                                            +number('Checkpoint page buffer:', 'model.checkpointPageBufferSize', '"DataRegionCheckpointPageBufferSize" + $index',
+                                                'true', '0', '0', 'Amount of memory allocated for a checkpoint temporary buffer in bytes')
                                         .settings-row
                                             +checkbox('Metrics enabled', 'model.metricsEnabled', '"DataRegionMetricsEnabled" + $index',
                                             'Whether memory metrics are enabled by default on node startup')


[4/7] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-6667
Commit: 2d0e91ee311e8658978539ab2fc31c1b6319f891
Parents: 5fd13f5 918feba
Author: devozerov <vo...@gridgain.com>
Authored: Wed Oct 25 18:49:10 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Oct 25 18:49:10 2017 +0300

----------------------------------------------------------------------
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   7 +-
 .../datastreamer/DataStreamerImpl.java          |  23 +-
 .../cache/IgniteOutOfMemoryPropagationTest.java | 251 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite6.java       |   5 +
 5 files changed, 285 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[6/7] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-6667

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-6667


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

Branch: refs/heads/ignite-6667
Commit: 1b4bf609b4113de055ddd36c8bb51e75324d5200
Parents: 46501d9 8266a98
Author: sboikov <sb...@gridgain.com>
Authored: Thu Oct 26 11:18:14 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Oct 26 11:18:14 2017 +0300

----------------------------------------------------------------------
 .../configuration/DataRegionConfiguration.java  |  34 ++-
 .../configuration/DataStorageConfiguration.java |  29 ---
 .../MemoryPolicyConfiguration.java              |  23 --
 .../org/apache/ignite/internal/IgnitionEx.java  |  13 +-
 .../discovery/GridDiscoveryManager.java         |   8 +-
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |   6 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   7 +-
 .../GridCacheDatabaseSharedManager.java         |  71 ++----
 .../datastreamer/DataStreamerImpl.java          |  23 +-
 .../utils/PlatformConfigurationUtils.java       |   6 +-
 .../node/VisorDataRegionConfiguration.java      |  13 +
 .../node/VisorDataStorageConfiguration.java     |   3 +-
 .../node/VisorPersistentStoreConfiguration.java |   1 -
 .../communication/tcp/TcpCommunicationSpi.java  |  14 +-
 .../cache/IgniteOutOfMemoryPropagationTest.java | 251 +++++++++++++++++++
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |   7 +-
 .../pagemem/PagesWriteThrottleSandboxTest.java  |   2 +-
 .../pagemem/PagesWriteThrottleSmokeTest.java    |   2 +-
 .../testsuites/IgniteCacheTestSuite6.java       |   5 +
 modules/dev-utils/pom.xml                       |  13 +
 .../Config/full-config.xml                      |   6 +-
 .../IgniteConfigurationSerializerTest.cs        |   6 +-
 .../IgniteConfigurationTest.cs                  |   6 +-
 .../Configuration/DataRegionConfiguration.cs    |   8 +
 .../Configuration/DataStorageConfiguration.cs   |   9 -
 .../IgniteConfigurationSection.xsd              |  15 +-
 modules/web-console/backend/app/mongo.js        |   6 +-
 .../generator/ConfigurationGenerator.js         |   5 +-
 .../generator/defaults/Cluster.service.js       |   3 +-
 .../configuration/clusters/data-storage.pug     |   6 +
 30 files changed, 445 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1b4bf609/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------


[7/7] ignite git commit: ignite-6667

Posted by sb...@apache.org.
ignite-6667


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

Branch: refs/heads/ignite-6667
Commit: ece4db124e9cb85cae44f5a95cd7432ada81836f
Parents: 1b4bf60
Author: sboikov <sb...@gridgain.com>
Authored: Thu Oct 26 11:28:48 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Oct 26 11:28:48 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/discovery/GridDiscoveryManager.java   | 1 +
 .../internal/processors/cluster/ChangeGlobalStateMessage.java      | 2 +-
 2 files changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ece4db12/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 5479f69..a6737dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -3065,6 +3065,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param topVer Topology version.
      * @param discoCache Current disco cache.
+     * @return New discovery cache.
      */
     public DiscoCache createDiscoCacheOnCacheChange(AffinityTopologyVersion topVer,
         DiscoCache discoCache) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ece4db12/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
index 22e376f..6a642bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ChangeGlobalStateMessage.java
@@ -118,7 +118,7 @@ public class ChangeGlobalStateMessage implements DiscoveryCustomMessage {
     /** {@inheritDoc} */
     @Override public DiscoCache createDiscoCache(GridDiscoveryManager mgr, AffinityTopologyVersion topVer,
         DiscoCache discoCache) {
-        return discoCache.copy(topVer, null);
+        return mgr.createDiscoCacheOnCacheChange(topVer, discoCache);
     }
 
     /**