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/03/10 07:23:42 UTC

[21/43] ignite git commit: IGNITE-4694 Add tests to check there are no memory leaks in PageMemory

IGNITE-4694 Add tests to check there are no memory leaks in PageMemory


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

Branch: refs/heads/ignite-4712
Commit: ef8e4799ab20f5451494ed660abb77d8fe645a5b
Parents: ee28b9c
Author: Igor Seliverstov <gv...@gmail.com>
Authored: Wed Feb 15 13:41:08 2017 +0300
Committer: Igor Seliverstov <gv...@gmail.com>
Committed: Wed Mar 1 10:44:53 2017 +0300

----------------------------------------------------------------------
 .../database/IgniteDbAbstractTest.java          | 360 +++++++++++++++++++
 .../IgniteDbMemoryLeakAbstractTest.java         |  84 +++++
 .../database/IgniteDbMemoryLeakIndexedTest.java |  85 +++++
 .../IgniteDbMemoryLeakLargeObjectsTest.java     |  95 +++++
 .../IgniteDbMemoryLeakLargePagesTest.java       |  90 +++++
 .../database/IgniteDbMemoryLeakTest.java        |  85 +++++
 .../IgniteDbMemoryLeakWithExpirationTest.java   |  92 +++++
 .../database/IgniteDbPutGetAbstractTest.java    | 347 +-----------------
 8 files changed, 903 insertions(+), 335 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
new file mode 100644
index 0000000..3bc7004
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbAbstractTest.java
@@ -0,0 +1,360 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Random;
+
+/**
+ *
+ */
+public abstract class IgniteDbAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /**
+     * @return Node count.
+     */
+    protected abstract int gridCount();
+
+    /**
+     * @return {@code True} if indexing is enabled.
+     */
+    protected abstract boolean indexingEnabled();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        MemoryConfiguration dbCfg = new MemoryConfiguration();
+
+        dbCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
+
+        if (isLargePage())
+            dbCfg.setPageSize(16 * 1024);
+        else
+            dbCfg.setPageSize(1024);
+
+
+        dbCfg.setPageCacheSize(200 * 1024 * 1024);
+
+        configure(dbCfg);
+
+        cfg.setMemoryConfiguration(dbCfg);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        if (indexingEnabled())
+            ccfg.setIndexedTypes(Integer.class, DbValue.class);
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+
+        CacheConfiguration ccfg2 = new CacheConfiguration("non-primitive");
+
+        if (indexingEnabled())
+            ccfg2.setIndexedTypes(DbKey.class, DbValue.class);
+
+        ccfg2.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg2.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg2.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg2.setAffinity(new RendezvousAffinityFunction(false, 32));
+
+        CacheConfiguration ccfg3 = new CacheConfiguration("large");
+
+        if (indexingEnabled())
+            ccfg3.setIndexedTypes(Integer.class, LargeDbValue.class);
+
+        ccfg3.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg3.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg3.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg3.setAffinity(new RendezvousAffinityFunction(false, 32));
+
+        CacheConfiguration ccfg4 = new CacheConfiguration("tiny");
+
+        ccfg4.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg4.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        ccfg4.setRebalanceMode(CacheRebalanceMode.SYNC);
+        ccfg4.setAffinity(new RendezvousAffinityFunction(false, 32));
+
+        final AffinityFunction aff = new RendezvousAffinityFunction(1, null);
+
+        ccfg4.setAffinity(aff);
+
+        cfg.setCacheConfiguration(ccfg, ccfg2, ccfg3, ccfg4);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setMarshaller(null);
+
+        configure(cfg);
+
+        return cfg;
+    }
+
+    protected void configure(IgniteConfiguration cfg){
+        //NOP
+    }
+
+    protected void configure(MemoryConfiguration mCfg){
+        //NOP
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+
+        long seed = 1464583813940L; // System.currentTimeMillis();
+
+        info("Seed: " + seed + "L");
+
+        BPlusTree.rnd = new Random(seed);
+
+        startGrids(gridCount());
+
+        awaitPartitionMapExchange();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        BPlusTree.rnd = null;
+
+        stopAllGrids();
+
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+    }
+
+    /**
+     * @return {@code True} if use large page.
+     */
+    protected boolean isLargePage() {
+        return false;
+    }
+
+    /**
+     *
+     */
+    static class DbKey implements Serializable {
+        /** */
+        int val;
+
+        /**
+         * @param val Value.
+         */
+        DbKey(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || !(o instanceof DbKey))
+                return false;
+
+            DbKey key = (DbKey)o;
+
+            return val == key.val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+    }
+
+    /**
+     *
+     */
+    static class LargeDbKey implements Serializable {
+        /** */
+        int val;
+
+        /** */
+        byte[] data;
+
+        /**
+         * @param val Value.
+         * @param size Key payload size.
+         */
+        LargeDbKey(int val, int size) {
+            this.val = val;
+
+            data = new byte[size];
+
+            Arrays.fill(data, (byte)val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || !(o instanceof LargeDbKey))
+                return false;
+
+            LargeDbKey key = (LargeDbKey)o;
+
+            return val == key.val && Arrays.equals(data, key.data);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val + Arrays.hashCode(data);
+        }
+    }
+
+    /**
+     *
+     */
+    static class DbValue implements Serializable {
+        /** */
+        @QuerySqlField(index = true)
+        int iVal;
+
+        /** */
+        @QuerySqlField(index = true)
+        String sVal;
+
+        /** */
+        @QuerySqlField
+        long lVal;
+
+
+
+        /**
+         * @param iVal Integer value.
+         * @param sVal String value.
+         * @param lVal Long value.
+         */
+        DbValue(int iVal, String sVal, long lVal) {
+            this.iVal = iVal;
+            this.sVal = sVal;
+            this.lVal = lVal;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            DbValue dbVal = (DbValue)o;
+
+            return iVal == dbVal.iVal && lVal == dbVal.lVal &&
+                    !(sVal != null ? !sVal.equals(dbVal.sVal) : dbVal.sVal != null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = iVal;
+
+            res = 31 * res + (sVal != null ? sVal.hashCode() : 0);
+            res = 31 * res + (int)(lVal ^ (lVal >>> 32));
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DbValue.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class LargeDbValue {
+        /** */
+        @QuerySqlField(index = true)
+        String str1;
+
+        /** */
+        @QuerySqlField(index = true)
+        String str2;
+
+        /** */
+        int[] arr;
+
+        /**
+         * @param str1 String 1.
+         * @param str2 String 2.
+         * @param arr Big array.
+         */
+        LargeDbValue(final String str1, final String str2, final int[] arr) {
+            this.str1 = str1;
+            this.str2 = str2;
+            this.arr = arr;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(final Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            final LargeDbValue that = (LargeDbValue) o;
+
+            if (str1 != null ? !str1.equals(that.str1) : that.str1 != null) return false;
+            if (str2 != null ? !str2.equals(that.str2) : that.str2 != null) return false;
+
+            return Arrays.equals(arr, that.arr);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = str1 != null ? str1.hashCode() : 0;
+
+            res = 31 * res + (str2 != null ? str2.hashCode() : 0);
+            res = 31 * res + Arrays.hashCode(arr);
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(LargeDbValue.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
new file mode 100644
index 0000000..6a5d039
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakAbstractTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.compute.ComputeTaskFuture;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ *
+ */
+public abstract class IgniteDbMemoryLeakAbstractTest extends IgniteDbAbstractTest {
+
+    /** Test duration in seconds*/
+    protected abstract int duration();
+
+    @Override
+    protected long getTestTimeout() {
+        return duration() * 1200;
+    }
+
+    /** */
+    protected abstract void operation(IgniteEx ig);
+
+    /** */
+    public void testMemoryLeak() throws Exception {
+
+        final long end = System.nanoTime() + TimeUnit.SECONDS.toNanos(duration());
+
+        int tasksCount = Runtime.getRuntime().availableProcessors() * 4;
+
+        IgniteCompute compute = grid(0).compute().withAsync();
+
+        ComputeTaskFuture[] futs = new ComputeTaskFuture[tasksCount];
+
+        for (int i = 0; i < tasksCount; i++) {
+            compute.run(new IgniteRunnable() {
+                @IgniteInstanceResource
+                private Ignite ig;
+
+                @Override
+                public void run() {
+                    int i = 0;
+                    while (System.nanoTime() < end) {
+                        operation((IgniteEx) ig);
+
+                        if(i++ == 100) {
+                            check((IgniteEx) ig);
+                            i = 0;
+                        }
+                    }
+                }
+            });
+
+            futs[i] = compute.future();
+        }
+
+        for (ComputeTaskFuture fut : futs) {
+            fut.get();
+        }
+    }
+
+    protected void check(IgniteEx ig) {}
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakIndexedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakIndexedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakIndexedTest.java
new file mode 100644
index 0000000..4cd74d0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakIndexedTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ *
+ */
+public class IgniteDbMemoryLeakIndexedTest extends IgniteDbMemoryLeakAbstractTest {
+
+    @Override
+    protected int duration() {
+        return 300;
+    }
+
+    @Override
+    protected int gridCount() {
+        return 1;
+    }
+
+    @Override
+    protected void configure(IgniteConfiguration cfg) {
+        cfg.setMetricsLogFrequency(5000);
+    }
+
+    @Override
+    protected void configure(MemoryConfiguration mCfg) {
+        mCfg.setPageCacheSize(1024 * 1024);
+    }
+
+    @Override
+    protected boolean indexingEnabled() {
+        return true;
+    }
+
+    protected void operation(IgniteEx ig){
+        IgniteCache<Object, Object> cache = ig.cache("non-primitive");
+        Random rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 1000; i++) {
+            DbKey key = new DbKey(rnd.nextInt(200_000));
+
+            DbValue v0 = new DbValue(key.val, "test-value-" + rnd.nextInt(200), rnd.nextInt(500));
+
+            switch (rnd.nextInt(3)) {
+                case 0:
+                    cache.getAndPut(key, v0);
+                case 1:
+                    cache.get(key);
+                    break;
+                case 2:
+                    cache.getAndRemove(key);
+            }
+        }
+    }
+
+    @Override
+    protected void check(IgniteEx ig) {
+        long pages = ig.context().cache().context().database().pageMemory().loadedPages();
+
+        assertTrue(pages < 19100);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargeObjectsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargeObjectsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargeObjectsTest.java
new file mode 100644
index 0000000..a4d88e1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargeObjectsTest.java
@@ -0,0 +1,95 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ *
+ */
+public class IgniteDbMemoryLeakLargeObjectsTest extends IgniteDbMemoryLeakAbstractTest {
+
+    private final static int[] ARRAY;
+    static {
+        ARRAY = new int[1024];
+        Random rnd = new Random();
+        for (int i = 0; i < ARRAY.length; i++) {
+            ARRAY[i] = rnd.nextInt();
+        }
+
+    }
+
+    @Override
+    protected int duration() {
+        return 300;
+    }
+
+    @Override
+    protected int gridCount() {
+        return 1;
+    }
+
+    @Override
+    protected void configure(IgniteConfiguration cfg) {
+        cfg.setMetricsLogFrequency(5000);
+    }
+
+    @Override
+    protected void configure(MemoryConfiguration mCfg) {
+        mCfg.setPageCacheSize(60 * 1024 * 1024);
+    }
+
+    @Override
+    protected boolean indexingEnabled() {
+        return false;
+    }
+
+    protected void operation(IgniteEx ig){
+        IgniteCache<Object, Object> cache = ig.cache("large");
+        Random rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 1000; i++) {
+            LargeDbKey key = new LargeDbKey(rnd.nextInt(10_000), 1024);
+
+            LargeDbValue v0 = new LargeDbValue("test-value-1-" + rnd.nextInt(200), "test-value-2-" + rnd.nextInt(200), ARRAY);
+
+            switch (rnd.nextInt(3)) {
+                case 0:
+                    cache.getAndPut(key, v0);
+                case 1:
+                    cache.get(key);
+                    break;
+                case 2:
+                    cache.getAndRemove(key);
+            }
+        }
+    }
+
+    @Override
+    protected void check(IgniteEx ig) {
+        long pages = ig.context().cache().context().database().pageMemory().loadedPages();
+
+        assertTrue(pages < 50000);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargePagesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargePagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargePagesTest.java
new file mode 100644
index 0000000..bfa4aa9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakLargePagesTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ *
+ */
+public class IgniteDbMemoryLeakLargePagesTest extends IgniteDbMemoryLeakAbstractTest {
+
+    @Override
+    protected int duration() {
+        return 300;
+    }
+
+    @Override
+    protected int gridCount() {
+        return 1;
+    }
+
+    @Override
+    protected void configure(IgniteConfiguration cfg) {
+        cfg.setMetricsLogFrequency(5000);
+    }
+
+    @Override
+    protected void configure(MemoryConfiguration mCfg) {
+        mCfg.setPageCacheSize(100 * 1024 * 1024);
+    }
+
+    @Override
+    protected boolean indexingEnabled() {
+        return false;
+    }
+
+    @Override
+    protected boolean isLargePage() {
+        return true;
+    }
+
+    protected void operation(IgniteEx ig){
+        IgniteCache<Object, Object> cache = ig.cache("non-primitive");
+        Random rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 1000; i++) {
+            DbKey key = new DbKey(rnd.nextInt(200_000));
+
+            DbValue v0 = new DbValue(key.val, "test-value-" + rnd.nextInt(200), rnd.nextInt(500));
+
+            switch (rnd.nextInt(3)) {
+                case 0:
+                    cache.getAndPut(key, v0);
+                case 1:
+                    cache.get(key);
+                    break;
+                case 2:
+                    cache.getAndRemove(key);
+            }
+        }
+    }
+
+    @Override
+    protected void check(IgniteEx ig) {
+        long pages = ig.context().cache().context().database().pageMemory().loadedPages();
+
+        assertTrue(pages < 4600);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakTest.java
new file mode 100644
index 0000000..6af4e41
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ *
+ */
+public class IgniteDbMemoryLeakTest extends IgniteDbMemoryLeakAbstractTest {
+
+    @Override
+    protected int duration() {
+        return 300;
+    }
+
+    @Override
+    protected int gridCount() {
+        return 1;
+    }
+
+    @Override
+    protected void configure(IgniteConfiguration cfg) {
+        cfg.setMetricsLogFrequency(5000);
+    }
+
+    @Override
+    protected void configure(MemoryConfiguration mCfg) {
+        mCfg.setPageCacheSize(1024 * 1024);
+    }
+
+    @Override
+    protected boolean indexingEnabled() {
+        return false;
+    }
+
+    protected void operation(IgniteEx ig){
+        IgniteCache<Object, Object> cache = ig.cache("non-primitive");
+        Random rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 1000; i++) {
+            DbKey key = new DbKey(rnd.nextInt(200_000));
+
+            DbValue v0 = new DbValue(key.val, "test-value-" + rnd.nextInt(200), rnd.nextInt(500));
+
+            switch (rnd.nextInt(3)) {
+                case 0:
+                    cache.getAndPut(key, v0);
+                case 1:
+                    cache.get(key);
+                    break;
+                case 2:
+                    cache.getAndRemove(key);
+            }
+        }
+    }
+
+    @Override
+    protected void check(IgniteEx ig) {
+        long pages = ig.context().cache().context().database().pageMemory().loadedPages();
+
+        assertTrue(pages < 19100);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
new file mode 100644
index 0000000..d9e3f34
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbMemoryLeakWithExpirationTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+
+import javax.cache.expiry.CreatedExpiryPolicy;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ *
+ */
+public class IgniteDbMemoryLeakWithExpirationTest extends IgniteDbMemoryLeakAbstractTest {
+
+    private static final ExpiryPolicy EXPIRY = new CreatedExpiryPolicy(new Duration(MILLISECONDS, 10L));
+
+    @Override
+    protected int duration() {
+        return 300;
+    }
+
+    @Override
+    protected int gridCount() {
+        return 1;
+    }
+
+    @Override
+    protected void configure(IgniteConfiguration cfg) {
+        cfg.setMetricsLogFrequency(5000);
+    }
+
+    @Override
+    protected void configure(MemoryConfiguration mCfg) {
+        mCfg.setPageCacheSize(1024 * 1024);
+    }
+
+    @Override
+    protected boolean indexingEnabled() {
+        return false;
+    }
+
+    protected void operation(IgniteEx ig) {
+        IgniteCache<Object, Object> cache = ig.cache("non-primitive").withExpiryPolicy(EXPIRY);
+        Random rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 1000; i++) {
+            DbKey key = new DbKey(rnd.nextInt(200_000));
+
+            DbValue v0 = new DbValue(key.val, "test-value-" + rnd.nextInt(200), rnd.nextInt(500));
+
+            switch (rnd.nextInt(3)) {
+                case 0:
+                    cache.getAndPut(key, v0);
+                case 1:
+                    cache.get(key);
+                    break;
+                case 2:
+                    cache.getAndRemove(key);
+            }
+        }
+    }
+
+    @Override
+    protected void check(IgniteEx ig) {
+        long pages = ig.context().cache().context().database().pageMemory().loadedPages();
+
+        assertTrue(pages < 10000);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef8e4799/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
index c7a07e3..228a262 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/IgniteDbPutGetAbstractTest.java
@@ -17,175 +17,39 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ThreadLocalRandom;
-import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.CacheRebalanceMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.MemoryConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
 
+import javax.cache.Cache;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
 /**
  *
  */
-public abstract class IgniteDbPutGetAbstractTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /**
-     * @return Node count.
-     */
-    protected abstract int gridCount();
-
-    /**
-     * @return {@code True} if indexing is enabled.
-     */
-    protected abstract boolean indexingEnabled();
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        MemoryConfiguration dbCfg = new MemoryConfiguration();
-
-        if (isLargePage()) {
-            dbCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
-
-            dbCfg.setPageSize(16 * 1024);
-
-            dbCfg.setPageCacheSize(200 * 1024 * 1024);
-        }
-        else {
-            dbCfg.setConcurrencyLevel(Runtime.getRuntime().availableProcessors() * 4);
-
-            dbCfg.setPageSize(1024);
-
-            dbCfg.setPageCacheSize(200 * 1024 * 1024);
-        }
-
-        cfg.setMemoryConfiguration(dbCfg);
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        if (indexingEnabled())
-            ccfg.setIndexedTypes(Integer.class, DbValue.class);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg.setRebalanceMode(CacheRebalanceMode.SYNC);
-        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
-
-        CacheConfiguration ccfg2 = new CacheConfiguration("non-primitive");
-
-        if (indexingEnabled())
-            ccfg2.setIndexedTypes(DbKey.class, DbValue.class);
-
-        ccfg2.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg2.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg2.setRebalanceMode(CacheRebalanceMode.SYNC);
-        ccfg2.setAffinity(new RendezvousAffinityFunction(false, 32));
-
-        CacheConfiguration ccfg3 = new CacheConfiguration("large");
-
-        if (indexingEnabled())
-            ccfg3.setIndexedTypes(Integer.class, LargeDbValue.class);
-
-        ccfg3.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg3.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg3.setRebalanceMode(CacheRebalanceMode.SYNC);
-        ccfg3.setAffinity(new RendezvousAffinityFunction(false, 32));
-
-        CacheConfiguration ccfg4 = new CacheConfiguration("tiny");
-
-        ccfg4.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg4.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        ccfg4.setRebalanceMode(CacheRebalanceMode.SYNC);
-        ccfg4.setAffinity(new RendezvousAffinityFunction(false, 32));
-
-        final AffinityFunction aff = new RendezvousAffinityFunction(1, null);
-
-        ccfg4.setAffinity(aff);
-
-        cfg.setCacheConfiguration(ccfg, ccfg2, ccfg3, ccfg4);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setMarshaller(null);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
-
-        long seed = 1464583813940L; // System.currentTimeMillis();
-
-        info("Seed: " + seed + "L");
-
-        BPlusTree.rnd = new Random(seed);
-
-        startGrids(gridCount());
-
-        awaitPartitionMapExchange();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        BPlusTree.rnd = null;
-
-        stopAllGrids();
-
-        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
-    }
-
-    /**
-     * @return {@code True} if use large page.
-     */
-    protected boolean isLargePage() {
-        return false;
-    };
-
+public abstract class IgniteDbPutGetAbstractTest extends IgniteDbAbstractTest {
     /**
      *
      */
@@ -1349,191 +1213,4 @@ public abstract class IgniteDbPutGetAbstractTest extends GridCommonAbstractTest
 
         assertNull(internalCache.peekEx(key));
     }
-
-    /**
-     *
-     */
-    private static class DbKey implements Serializable {
-        /** */
-        private int val;
-
-        /**
-         * @param val Value.
-         */
-        private DbKey(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || !(o instanceof DbKey))
-                return false;
-
-            DbKey key = (DbKey)o;
-
-            return val == key.val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-    }
-
-    /**
-     *
-     */
-    private static class LargeDbKey implements Serializable {
-        /** */
-        private int val;
-
-        /** */
-        private byte[] data;
-
-        /**
-         * @param val Value.
-         * @param size Key payload size.
-         */
-        private LargeDbKey(int val, int size) {
-            this.val = val;
-
-            data = new byte[size];
-
-            Arrays.fill(data, (byte)val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || !(o instanceof LargeDbKey))
-                return false;
-
-            LargeDbKey key = (LargeDbKey)o;
-
-            return val == key.val && Arrays.equals(data, key.data);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val + Arrays.hashCode(data);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class DbValue implements Serializable {
-        /** */
-        @QuerySqlField(index = true)
-        private int iVal;
-
-        /** */
-        @QuerySqlField(index = true)
-        private String sVal;
-
-        /** */
-        @QuerySqlField
-        private long lVal;
-
-        /**
-         * @param iVal Integer value.
-         * @param sVal String value.
-         * @param lVal Long value.
-         */
-        public DbValue(int iVal, String sVal, long lVal) {
-            this.iVal = iVal;
-            this.sVal = sVal;
-            this.lVal = lVal;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            DbValue dbVal = (DbValue)o;
-
-            return iVal == dbVal.iVal && lVal == dbVal.lVal &&
-                !(sVal != null ? !sVal.equals(dbVal.sVal) : dbVal.sVal != null);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = iVal;
-
-            res = 31 * res + (sVal != null ? sVal.hashCode() : 0);
-            res = 31 * res + (int)(lVal ^ (lVal >>> 32));
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(DbValue.class, this);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class LargeDbValue {
-        /** */
-        @QuerySqlField(index = true)
-        private String str1;
-
-        /** */
-        @QuerySqlField(index = true)
-        private String str2;
-
-        /** */
-        private int[] arr;
-
-        /**
-         * @param str1 String 1.
-         * @param str2 String 2.
-         * @param arr Big array.
-         */
-        public LargeDbValue(final String str1, final String str2, final int[] arr) {
-            this.str1 = str1;
-            this.str2 = str2;
-            this.arr = arr;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(final Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            final LargeDbValue that = (LargeDbValue) o;
-
-            if (str1 != null ? !str1.equals(that.str1) : that.str1 != null) return false;
-            if (str2 != null ? !str2.equals(that.str2) : that.str2 != null) return false;
-
-            return Arrays.equals(arr, that.arr);
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = str1 != null ? str1.hashCode() : 0;
-
-            res = 31 * res + (str2 != null ? str2.hashCode() : 0);
-            res = 31 * res + Arrays.hashCode(arr);
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(LargeDbValue.class, this);
-        }
-    }
 }