You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/06/09 00:29:05 UTC

[27/42] incubator-ignite git commit: #ignite-990: Add method getAllOutTx to IgniteCache.

#ignite-990: Add method getAllOutTx to IgniteCache.


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

Branch: refs/heads/ignite-389
Commit: ab20fd9059c21c65f3b34da67d75fc94098850e3
Parents: 5f06f57
Author: ivasilinets <iv...@gridgain.com>
Authored: Fri Jun 5 19:06:10 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Fri Jun 5 19:06:10 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  11 ++
 .../processors/cache/GridCacheAdapter.java      |  17 ++-
 .../processors/cache/GridCacheProxyImpl.java    |  14 ++-
 .../processors/cache/IgniteCacheProxy.java      |  23 +++++
 .../processors/cache/IgniteInternalCache.java   |  11 +-
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  83 +++++++++++++++
 .../GridCacheExAbstractFullApiSelfTest.java     | 103 -------------------
 .../GridCacheExColocatedFullApiSelfTest.java    |  33 ------
 .../near/GridCacheExNearFullApiSelfTest.java    |  39 -------
 .../GridCacheExReplicatedFullApiSelfTest.java   |  33 ------
 .../local/GridCacheExLocalFullApiSelfTest.java  |  30 ------
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 --
 13 files changed, 155 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index f9007a2..07f4f3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -311,10 +311,21 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     @IgniteAsyncSupported
     @Override public Map<K, V> getAll(Set<? extends K> keys);
 
+    /**
+     * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
+     * and will not lock any keys if pessimistic transaction is started by thread.
+     *
+     * @param keys The keys whose associated values are to be returned.
+     * @return A map of entries that were found for the given keys.
+     */
+    @IgniteAsyncSupported
+    public Map<K, V> getAllOutTx(Set<? extends K> keys);
+
     /** {@inheritDoc} */
     @IgniteAsyncSupported
     @Override public boolean containsKey(K key);
 
+    /** {@inheritDoc} */
     @IgniteAsyncSupported
     public boolean containsKeys(Set<? extends K> keys);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 9d98ce7..56b5c15 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1220,11 +1220,22 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException {
+    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
+        return getAllOutTxAsync(keys).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return getAllAsync(keys, !ctx.config().isReadFromBackup(), /*skip tx*/true, null, null, taskName, true, false)
-            .get();
+        return getAllAsync(keys,
+            !ctx.config().isReadFromBackup(),
+            /*skip tx*/true,
+            null,
+            null,
+            taskName,
+            !ctx.keepPortable(),
+            false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 9a6d08a..63ba242 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -329,7 +329,7 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException {
+    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
         try {
@@ -341,6 +341,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getAllOutTxAsync(keys);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isIgfsDataCache() {
         CacheOperationContext prev = gate.enter(opCtx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f840015..4390993 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -699,6 +699,29 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
+        try {
+            CacheOperationContext prev = onEnter(opCtx);
+
+            try {
+                if (isAsync()) {
+                    setFuture(delegate.getAllOutTxAsync(keys));
+
+                    return null;
+                }
+                else
+                    return delegate.getAllOutTx(keys);
+            }
+            finally {
+                onLeave(prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+    }
+
     /**
      * @param keys Keys.
      * @return Values map.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index ccce1b7..d98379c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -1623,7 +1623,16 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * @return Value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException;
+    public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException;
+
+    /**
+     * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
+     * and will not lock any keys if pessimistic transaction is started by thread.
+     *
+     * @param keys Keys to get values for.
+     * @return Future for getAllOutTx operation.
+     */
+    public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys);
 
     /**
      * Checks whether this cache is IGFS data cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index e33e0d4..b98c5d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -669,7 +669,7 @@ public class IgfsMetaManager extends IgfsManager {
     private Map<String, IgfsListingEntry> directoryListing(IgniteUuid fileId, boolean skipTx) throws IgniteCheckedException {
         assert fileId != null;
 
-        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singletonList(fileId)).get(fileId) :
+        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
             id2InfoPrj.get(fileId);
 
         return info == null ? Collections.<String, IgfsListingEntry>emptyMap() : info.listing();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index da20869..9bfbd15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -4893,6 +4893,89 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testGetOutTx() throws Exception {
+        checkGetOutTx(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOutTxAsync() throws Exception {
+        checkGetOutTx(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkGetOutTx(boolean async) throws Exception {
+        final AtomicInteger lockEvtCnt = new AtomicInteger();
+
+        IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                lockEvtCnt.incrementAndGet();
+
+                return true;
+            }
+        };
+
+        try {
+            IgniteCache<String, Integer> cache = grid(0).cache(null);
+
+            List<String> keys = primaryKeysForCache(cache, 2);
+
+            assertEquals(2, keys.size());
+
+            cache.put(keys.get(0), 0);
+            cache.put(keys.get(1), 1);
+
+            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+
+            if (async)
+                cache = cache.withAsync();
+
+            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                Integer val0 = cache.get(keys.get(0));
+
+                if (async)
+                    val0 = cache.<Integer>future().get();
+
+                assertEquals(0, val0.intValue());
+
+                Map<String, Integer> allOutTx = cache.getAllOutTx(F.asSet(keys.get(1)));
+
+                if (async)
+                    allOutTx = cache.<Map<String, Integer>>future().get();
+
+                assertEquals(1, allOutTx.size());
+
+                assertTrue(allOutTx.containsKey(keys.get(1)));
+
+                assertEquals(1, allOutTx.get(keys.get(1)).intValue());
+            }
+
+            assertTrue(GridTestUtils.waitForCondition(new PA() {
+                @Override public boolean apply() {
+                    info("Lock event count: " + lockEvtCnt.get());
+                    if (atomicityMode() == ATOMIC)
+                        return lockEvtCnt.get() == 0;
+
+                    if (cacheMode() == PARTITIONED && nearEnabled()) {
+                        if (!grid(0).configuration().isClientMode())
+                            return lockEvtCnt.get() == 4;
+                    }
+
+                    return lockEvtCnt.get() == 2;
+                }
+            }, 15000));
+        }
+        finally {
+            grid(0).events().stopLocalListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+        }
+    }
+
+    /**
      * Sets given value, returns old value.
      */
     public static final class SetValueProcessor implements EntryProcessor<String, Integer, Integer> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
deleted file mode 100644
index 7a1b7cc..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.transactions.*;
-
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.transactions.TransactionConcurrency.*;
-import static org.apache.ignite.transactions.TransactionIsolation.*;
-
-/**
- * Abstract test for private cache interface.
- */
-public abstract class GridCacheExAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetOutTx() throws Exception {
-        final AtomicInteger lockEvtCnt = new AtomicInteger();
-
-        IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
-            @Override public boolean apply(Event evt) {
-                lockEvtCnt.incrementAndGet();
-
-                return true;
-            }
-        };
-
-        try {
-            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
-
-            GridCacheAdapter<String, Integer> cache = ((IgniteKernal)grid(0)).internalCache();
-
-            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                int key = 0;
-
-                for (int i = 0; i < 1000; i++) {
-                    if (grid(0).affinity(null).mapKeyToNode("key" + i).id().equals(grid(0).localNode().id())) {
-                        key = i;
-
-                        break;
-                    }
-                }
-
-                cache.get("key" + key);
-
-                for (int i = key + 1; i < 1000; i++) {
-                    if (grid(0).affinity(null).mapKeyToNode("key" + i).id().equals(grid(0).localNode().id())) {
-                        key = i;
-
-                        break;
-                    }
-                }
-
-                cache.getAllOutTx(F.asList("key" + key));
-            }
-
-            assertTrue(GridTestUtils.waitForCondition(new PA() {
-                @Override public boolean apply() {
-                    info("Lock event count: " + lockEvtCnt.get());
-
-                    return lockEvtCnt.get() == (nearEnabled() ? 4 : 2);
-                }
-            }, 15000));
-        }
-        finally {
-            grid(0).events().stopLocalListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
deleted file mode 100644
index 55b1f92..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.dht;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on colocated cache.
- */
-public class GridCacheExColocatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
deleted file mode 100644
index 11ec96a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on partitioned cache with near enabled.
- */
-public class GridCacheExNearFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
deleted file mode 100644
index 32ee784..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.replicated;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on replicated cache.
- */
-public class GridCacheExReplicatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
deleted file mode 100644
index 71c0495..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.local;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-/**
- * Tests private cache interface on local cache.
- */
-public class GridCacheExLocalFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    @Override protected CacheMode cacheMode() {
-        return CacheMode.LOCAL;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab20fd90/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 369e041..5fa3283 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -130,12 +130,6 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.class);
 
-        // Private cache API.
-        suite.addTestSuite(GridCacheExLocalFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExReplicatedFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExNearFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExColocatedFullApiSelfTest.class);
-
 //        Multithreaded // TODO: GG-708
 //        suite.addTestSuite(GridCacheLocalFullApiMultithreadedSelfTest.class);
 //        suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);