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 2016/02/17 13:06:03 UTC

[1/2] ignite git commit: ignite-2669 Changed BinaryUtils.isBinaryType to always convert enums to BinaryObject (cherry picked from commit 8be62f0)

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5.7 72500026c -> 528d6aea6


ignite-2669 Changed BinaryUtils.isBinaryType to always convert enums to BinaryObject
(cherry picked from commit 8be62f0)


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

Branch: refs/heads/ignite-1.5.7
Commit: e0eefa50588efcf3b836f93a4b1ca32f92d8973d
Parents: 7250002
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 17 15:57:27 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 17 15:03:10 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryUtils.java     |   5 +-
 .../cache/CacheEntryPredicateContainsValue.java |   8 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  10 +-
 .../cache/CacheEnumOperationsAbstractTest.java  | 307 +++++++++++++++++++
 .../CacheEnumOperationsSingleNodeTest.java      |  28 ++
 .../cache/CacheEnumOperationsTest.java          |  28 ++
 .../testsuites/IgniteCacheTestSuite2.java       |   5 +
 7 files changed, 380 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index c3343d4..6b2dd98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -584,10 +584,7 @@ public class BinaryUtils {
     public static boolean isBinaryType(Class<?> cls) {
         assert cls != null;
 
-        return BinaryObject.class.isAssignableFrom(cls) ||
-            BINARY_CLS.contains(cls) ||
-            cls.isEnum() ||
-            (cls.isArray() && cls.getComponentType().isEnum());
+        return BinaryObject.class.isAssignableFrom(cls) || BINARY_CLS.contains(cls);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
index e89fff4..3db8ae8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryPredicateContainsValue.java
@@ -58,8 +58,12 @@ public class CacheEntryPredicateContainsValue extends CacheEntryPredicateAdapter
     @Override public boolean apply(GridCacheEntryEx e) {
         CacheObject val = peekVisibleValue(e);
 
-        return F.eq(this.val.value(e.context().cacheObjectContext(), false),
-            CU.value(val, e.context(), false));
+        GridCacheContext cctx = e.context();
+
+        Object thisVal = CU.value(this.val, cctx, false);
+        Object cacheVal = CU.value(val, cctx, false);
+
+        return F.eq(thisVal, cacheVal);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 2769e13..f00c721 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -518,7 +518,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         Object obj0 = binaryMarsh.unmarshal(arr, null);
 
         // Possible if a class has writeObject method.
-        if (obj0 instanceof BinaryObject)
+        if (obj0 instanceof BinaryObjectImpl)
             ((BinaryObjectImpl)obj0).detachAllowed(true);
 
         return obj0;
@@ -786,8 +786,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
         if (((CacheObjectBinaryContext)ctx).binaryEnabled()) {
             obj = toBinary(obj);
 
-            if (obj instanceof BinaryObject)
-                return (BinaryObjectImpl)obj;
+            if (obj instanceof KeyCacheObject)
+                return (KeyCacheObject)obj;
         }
 
         return toCacheKeyObject0(obj, userObj);
@@ -804,8 +804,8 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
         obj = toBinary(obj);
 
-        if (obj instanceof BinaryObject)
-            return (BinaryObjectImpl)obj;
+        if (obj instanceof CacheObject)
+            return (CacheObject)obj;
 
         return toCacheObject0(obj, userObj);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
new file mode 100644
index 0000000..3957e46
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsAbstractTest.java
@@ -0,0 +1,307 @@
+/*
+ * 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 javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+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 static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_VALUES;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public abstract class CacheEnumOperationsAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @return Number of nodes.
+     */
+    protected abstract boolean singleNode();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        if (!singleNode()) {
+            startGridsMultiThreaded(4);
+
+            client = true;
+
+            startGridsMultiThreaded(4, 2);
+        }
+        else
+            startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomic() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapValues() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_VALUES);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicOffheapTiered() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTx() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, ONHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapValues() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_VALUES);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxOffheapTiered() throws Exception {
+        CacheConfiguration<Object, Object> ccfg = cacheConfiguration(PARTITIONED, 1, ATOMIC, OFFHEAP_TIERED);
+
+        enumOperations(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     */
+    private void enumOperations(CacheConfiguration<Object, Object> ccfg) {
+        ignite(0).createCache(ccfg);
+
+        try {
+            int key = 0;
+
+            int nodes;
+
+            if (!singleNode()) {
+                nodes = 6;
+
+                ignite(nodes - 1).createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
+            }
+            else
+                nodes = 1;
+
+            for (int i = 0; i < nodes; i++) {
+                IgniteCache<Object, Object> cache = ignite(i).cache(ccfg.getName());
+
+                for (int j = 0; j < 100; j++)
+                    enumOperations(cache, key++);
+            }
+        }
+        finally {
+            ignite(0).destroyCache(ccfg.getName());
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     */
+    private void enumOperations(IgniteCache<Object, Object> cache, int key) {
+        assertNull(cache.get(key));
+
+        assertFalse(cache.replace(key, TestEnum.VAL1));
+
+        assertTrue(cache.putIfAbsent(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertFalse(cache.putIfAbsent(key, TestEnum.VAL2));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertTrue(cache.replace(key, TestEnum.VAL2));
+
+        assertEquals(TestEnum.VAL2, cache.get(key));
+
+        assertFalse(cache.replace(key, TestEnum.VAL1, TestEnum.VAL3));
+
+        assertEquals(TestEnum.VAL2, cache.get(key));
+
+        assertTrue(cache.replace(key, TestEnum.VAL2, TestEnum.VAL3));
+
+        assertEquals(TestEnum.VAL3, cache.get(key));
+
+        assertEquals(TestEnum.VAL3, cache.getAndPut(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertEquals(TestEnum.VAL1, cache.invoke(key, new EnumProcessor(TestEnum.VAL2, TestEnum.VAL1)));
+
+        assertEquals(TestEnum.VAL2, cache.get(key));
+
+        assertEquals(TestEnum.VAL2, cache.getAndReplace(key, TestEnum.VAL3));
+
+        assertEquals(TestEnum.VAL3, cache.get(key));
+
+        assertEquals(TestEnum.VAL3, cache.getAndPutIfAbsent(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL3, cache.get(key));
+
+        cache.put(key, TestEnum.VAL1);
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+
+        assertEquals(TestEnum.VAL1, cache.getAndRemove(key));
+
+        assertNull(cache.get(key));
+
+        assertFalse(cache.replace(key, TestEnum.VAL2, TestEnum.VAL3));
+
+        assertNull(cache.getAndPutIfAbsent(key, TestEnum.VAL1));
+
+        assertEquals(TestEnum.VAL1, cache.get(key));
+    }
+
+    /**
+     *
+     * @param cacheMode Cache mode.
+     * @param backups Number of backups.
+     * @param atomicityMode Cache atomicity mode.
+     * @param memoryMode Cache memory mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration(
+        CacheMode cacheMode,
+        int backups,
+        CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memoryMode) {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setMemoryMode(memoryMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(PRIMARY);
+
+        if (memoryMode == OFFHEAP_TIERED)
+            ccfg.setOffHeapMaxMemory(0);
+
+        if (cacheMode == PARTITIONED)
+            ccfg.setBackups(backups);
+
+        return ccfg;
+    }
+
+    /**
+     *
+     */
+    public enum TestEnum {
+        /** */
+        VAL1,
+        /** */
+        VAL2,
+        /** */
+        VAL3
+    }
+
+    /**
+     *
+     */
+    static class EnumProcessor implements EntryProcessor<Object, Object, Object> {
+        /** */
+        private TestEnum newVal;
+
+        /** */
+        private TestEnum expOldVal;
+
+        /**
+         * @param newVal New value.
+         * @param expOldVal Expected old value.
+         */
+        public EnumProcessor(TestEnum newVal, TestEnum expOldVal) {
+            this.newVal = newVal;
+            this.expOldVal = expOldVal;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... args) {
+            TestEnum val = (TestEnum)entry.getValue();
+
+            assertEquals(expOldVal, val);
+
+            entry.setValue(newVal);
+
+            return val;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java
new file mode 100644
index 0000000..baaaac3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsSingleNodeTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class CacheEnumOperationsSingleNodeTest extends CacheEnumOperationsAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean singleNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java
new file mode 100644
index 0000000..8e4964f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEnumOperationsTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class CacheEnumOperationsTest extends CacheEnumOperationsAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean singleNode() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0eefa50/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index cadcba7..8bde7a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -23,6 +23,8 @@ import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionExcludeNeighbor
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDhtLocalPartitionAfterRemoveSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheEnumOperationsSingleNodeTest;
+import org.apache.ignite.internal.processors.cache.CacheEnumOperationsTest;
 import org.apache.ignite.internal.processors.cache.CrossCacheTxRandomOperationsTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAtomicMessageCountSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheFinishPartitionsSelfTest;
@@ -248,6 +250,9 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
         suite.addTest(new TestSuite(CacheLockReleaseNodeLeaveTest.class));
         suite.addTest(new TestSuite(NearCacheSyncUpdateTest.class));
+        suite.addTest(new TestSuite(CacheConfigurationLeakTest.class));
+        suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
+        suite.addTest(new TestSuite(CacheEnumOperationsTest.class));
 
         return suite;
     }


[2/2] ignite git commit: ignite-2669 Changed BinaryUtils.isBinaryType to always convert enums to BinaryObject (cherry picked from commit 8be62f0)

Posted by sb...@apache.org.
ignite-2669 Changed BinaryUtils.isBinaryType to always convert enums to BinaryObject
(cherry picked from commit 8be62f0)


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

Branch: refs/heads/ignite-1.5.7
Commit: 528d6aea65e5d0053649442e704476ee964ca218
Parents: e0eefa5
Author: sboikov <sb...@gridgain.com>
Authored: Wed Feb 17 15:04:30 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Feb 17 15:04:30 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java    | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/528d6aea/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 8bde7a7..d0943b5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -250,7 +250,6 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteDynamicCacheAndNodeStop.class));
         suite.addTest(new TestSuite(CacheLockReleaseNodeLeaveTest.class));
         suite.addTest(new TestSuite(NearCacheSyncUpdateTest.class));
-        suite.addTest(new TestSuite(CacheConfigurationLeakTest.class));
         suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
         suite.addTest(new TestSuite(CacheEnumOperationsTest.class));