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 2015/06/03 17:39:16 UTC

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

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-990 [created] 3440a8c29


#ignite-990: Add getAllOutTx method 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/746cee2f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/746cee2f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/746cee2f

Branch: refs/heads/ignite-990
Commit: 746cee2f102a9b8b25df2f57ff519fc6b4b1bfdb
Parents: 97d0bc1
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jun 3 17:33:13 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jun 3 17:33:13 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     | 11 ++++++++++
 .../processors/cache/GridCacheAdapter.java      |  9 +++++++-
 .../processors/cache/GridCacheProxyImpl.java    | 14 +++++++++++-
 .../processors/cache/IgniteCacheProxy.java      | 23 ++++++++++++++++++++
 .../processors/cache/IgniteInternalCache.java   | 12 +++++++++-
 .../processors/igfs/IgfsMetaManager.java        |  2 +-
 .../GridCacheExAbstractFullApiSelfTest.java     |  2 +-
 7 files changed, 68 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/746cee2f/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..ed03f9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -311,6 +311,17 @@ 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 Keys to get values for.
+     * @return Value.
+     * @throws IgniteCheckedException If failed.
+     */
+    @IgniteAsyncSupported
+    @Nullable public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException;
+
     /** {@inheritDoc} */
     @IgniteAsyncSupported
     @Override public boolean containsKey(K key);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/746cee2f/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..3267438 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,13 +1220,20 @@ 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 {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(keys, !ctx.config().isReadFromBackup(), /*skip tx*/true, null, null, taskName, true, false)
             .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);
+    }
+
     /**
      * @param keys Keys.
      * @param reload Reload flag.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/746cee2f/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..43bb19d 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) throws IgniteCheckedException {
+        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/746cee2f/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/746cee2f/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..e291f45 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,17 @@ 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;
+    @Nullable 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.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) throws IgniteCheckedException;
 
     /**
      * Checks whether this cache is IGFS data cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/746cee2f/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/746cee2f/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
index 7a1b7cc..dada45d 100644
--- 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
@@ -85,7 +85,7 @@ public abstract class GridCacheExAbstractFullApiSelfTest extends GridCacheAbstra
                     }
                 }
 
-                cache.getAllOutTx(F.asList("key" + key));
+                cache.getAllOutTx(F.asSet("key" + key));
             }
 
             assertTrue(GridTestUtils.waitForCondition(new PA() {


[3/3] incubator-ignite git commit: #ignite-990: Move tests for getOutTx.

Posted by sb...@apache.org.
#ignite-990: Move tests for getOutTx.


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

Branch: refs/heads/ignite-990
Commit: 3440a8c293cadc57510db33ad7df65606374d9ba
Parents: 3222175
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jun 3 18:38:56 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jun 3 18:38:56 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java |  92 +++++++++++++++++
 .../GridCacheExAbstractFullApiSelfTest.java     | 103 -------------------
 .../GridCacheExColocatedFullApiSelfTest.java    |  33 ------
 .../near/GridCacheExNearFullApiSelfTest.java    |  39 -------
 .../GridCacheExReplicatedFullApiSelfTest.java   |  33 ------
 .../local/GridCacheExLocalFullApiSelfTest.java  |  30 ------
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 --
 7 files changed, 92 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3440a8c2/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..9568d84 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,98 @@ 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 {
+            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+
+            IgniteCache<String, Integer> cache = grid(0).cache(null);
+
+            if (async)
+                cache = cache.withAsync();
+
+            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                int key = 0;
+
+                for (int i = 0; i < 1000; i++) {
+                    ClusterNode node = grid(0).affinity(null).mapKeyToNode("key" + i);
+
+                    //Client only mode.
+                    if (node == null)
+                        return;
+
+                    if (node.id().equals(grid(0).localNode().id())) {
+                        key = i;
+
+                        break;
+                    }
+                }
+
+                cache.get("key" + key);
+
+                if (async)
+                    cache.future().get();
+
+                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.asSet("key" + key));
+
+                if (async)
+                    cache.future().get();
+            }
+
+            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/3440a8c2/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 dada45d..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.asSet("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/3440a8c2/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/3440a8c2/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/3440a8c2/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/3440a8c2/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/3440a8c2/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);


[2/3] incubator-ignite git commit: #ignite-990: Remove Nullable annotation from getAllOutTx.

Posted by sb...@apache.org.
#ignite-990: Remove Nullable annotation from getAllOutTx.


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

Branch: refs/heads/ignite-990
Commit: 322217554a52ddcd872d3ed10ea5ef6a756153df
Parents: 746cee2
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jun 3 17:34:24 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jun 3 17:34:24 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteCache.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/32221755/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 ed03f9b..2578a7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -320,7 +320,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * @throws IgniteCheckedException If failed.
      */
     @IgniteAsyncSupported
-    @Nullable public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException;
+    public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
     @IgniteAsyncSupported