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/01/13 14:03:59 UTC

incubator-ignite git commit: # ignite-1 Implemented loadCache/localLoadCache

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1 aceb58689 -> bb32d468d


# ignite-1 Implemented loadCache/localLoadCache


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

Branch: refs/heads/ignite-1
Commit: bb32d468db01d1f6aab485e9f86c46a9184e6cc0
Parents: aceb586
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jan 13 16:03:46 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jan 13 16:03:46 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  39 ++---
 .../processors/cache/IgniteCacheProxy.java      | 124 +++++++++++++-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |   5 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   2 +-
 .../cache/GridCacheAbstractSelfTest.java        |   4 +-
 .../dht/GridCacheGlobalLoadTest.java            | 163 +++++++++++++++++++
 .../GridCachePartitionedLoadCacheSelfTest.java  |  20 ++-
 .../junits/common/GridCommonAbstractTest.java   |   7 +
 .../bamboo/GridDataGridTestSuite.java           |   1 +
 9 files changed, 328 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/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 f7b2c34..4644911 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -14,6 +14,7 @@ import org.apache.ignite.cache.query.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.store.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
@@ -86,20 +87,20 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * @param p Optional predicate (may be {@code null}). If provided, will be used to
      *      filter values to be put into cache.
      * @param args Optional user arguments to be passed into
-     *      {@link org.gridgain.grid.cache.store.GridCacheStore#loadCache(IgniteBiInClosure, Object...)} method.
+     *      {@link GridCacheStore#loadCache(IgniteBiInClosure, Object...)} method.
      * @throws CacheException If loading failed.
      */
     public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException;
 
     /**
-     * Delegates to {@link org.gridgain.grid.cache.store.GridCacheStore#loadCache(IgniteBiInClosure,Object...)} method
+     * Delegates to {@link GridCacheStore#loadCache(IgniteBiInClosure,Object...)} method
      * to load state from the underlying persistent storage. The loaded values
      * will then be given to the optionally passed in predicate, and, if the predicate returns
      * {@code true}, will be stored in cache. If predicate is {@code null}, then
      * all loaded values will be stored in cache.
      * <p>
      * Note that this method does not receive keys as a parameter, so it is up to
-     * {@link org.gridgain.grid.cache.store.GridCacheStore} implementation to provide all the data to be loaded.
+     * {@link GridCacheStore} implementation to provide all the data to be loaded.
      * <p>
      * This method is not transactional and may end up loading a stale value into
      * cache if another thread has updated the value immediately after it has been
@@ -109,7 +110,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * @param p Optional predicate (may be {@code null}). If provided, will be used to
      *      filter values to be put into cache.
      * @param args Optional user arguments to be passed into
-     *      {@link org.gridgain.grid.cache.store.GridCacheStore#loadCache(IgniteBiInClosure, Object...)} method.
+     *      {@link GridCacheStore#loadCache(IgniteBiInClosure, Object...)} method.
      * @throws CacheException If loading failed.
      */
     public void localLoadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException;
@@ -121,14 +122,14 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * the value will be loaded from the primary node, which in its turn may load the value
      * from the swap storage, and consecutively, if it's not in swap,
      * from the underlying persistent storage. If value has to be loaded from persistent
-     * storage, {@link org.gridgain.grid.cache.store.GridCacheStore#load(IgniteTx, Object)} method will be used.
+     * storage, {@link GridCacheStore#load(IgniteTx, Object)} method will be used.
      * <p>
      * If the returned value is not needed, method {@link #putIfAbsent(Object, Object)} should
      * always be used instead of this one to avoid the overhead associated with returning of the
      * previous value.
      * <p>
-     * If write-through is enabled, the stored value will be persisted to {@link org.gridgain.grid.cache.store.GridCacheStore}
-     * via {@link org.gridgain.grid.cache.store.GridCacheStore#put(IgniteTx, Object, Object)} method.
+     * If write-through is enabled, the stored value will be persisted to {@link GridCacheStore}
+     * via {@link GridCacheStore#put(IgniteTx, Object, Object)} method.
      * <h2 class="header">Transactions</h2>
      * This method is transactional and will enlist the entry into ongoing transaction
      * if there is one.
@@ -155,8 +156,8 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * are acquired in undefined order, so it may cause a deadlock when used with
      * other concurrent transactional updates.
      * <p>
-     * If write-through is enabled, the values will be removed from {@link org.gridgain.grid.cache.store.GridCacheStore}
-     * via {@link org.gridgain.grid.cache.store.GridCacheStore#removeAll(IgniteTx, java.util.Collection)} method.
+     * If write-through is enabled, the values will be removed from {@link GridCacheStore}
+     * via {@link GridCacheStore#removeAll(IgniteTx, java.util.Collection)} method.
      * <h2 class="header">Transactions</h2>
      * This method is transactional and will enlist the entry into ongoing transaction
      * if there is one.
@@ -310,13 +311,13 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * the value will be loaded from the primary node, which in its turn may load the value
      * from the swap storage, and consecutively, if it's not in swap,
      * from the underlying persistent storage. If value has to be loaded from persistent
-     * storage,  {@link org.gridgain.grid.cache.store.GridCacheStore#load(IgniteTx, Object)} method will be used.
+     * storage,  {@link GridCacheStore#load(IgniteTx, Object)} method will be used.
      * <p>
      * If the returned value is not needed, method {@link #putIf(Object, Object, IgnitePredicate)} should
      * always be used instead of this one to avoid the overhead associated with returning of the previous value.
      * <p>
-     * If write-through is enabled, the stored value will be persisted to {@link org.gridgain.grid.cache.store.GridCacheStore}
-     * via {@link org.gridgain.grid.cache.store.GridCacheStore#put(IgniteTx, Object, Object)} method.
+     * If write-through is enabled, the stored value will be persisted to {@link GridCacheStore}
+     * via {@link GridCacheStore#put(IgniteTx, Object, Object)} method.
      * <h2 class="header">Transactions</h2>
      * This method is transactional and will enlist the entry into ongoing transaction
      * if there is one.
@@ -347,8 +348,8 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * value and, therefore, does not have any overhead associated with returning a value. It
      * should be used whenever return value is not required.
      * <p>
-     * If write-through is enabled, the stored value will be persisted to {@link org.gridgain.grid.cache.store.GridCacheStore}
-     * via {@link org.gridgain.grid.cache.store.GridCacheStore#put(IgniteTx, Object, Object)} method.
+     * If write-through is enabled, the stored value will be persisted to {@link GridCacheStore}
+     * via {@link GridCacheStore#put(IgniteTx, Object, Object)} method.
      * <h2 class="header">Transactions</h2>
      * This method is transactional and will enlist the entry into ongoing transaction
      * if there is one.
@@ -375,14 +376,14 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * caches, the value will be loaded from the primary node, which in its turn may load the value
      * from the disk-based swap storage, and consecutively, if it's not in swap,
      * from the underlying persistent storage. If value has to be loaded from persistent
-     * storage, {@link org.gridgain.grid.cache.store.GridCacheStore#load(IgniteTx, Object)} method will be used.
+     * storage, {@link GridCacheStore#load(IgniteTx, Object)} method will be used.
      * <p>
      * If the returned value is not needed, method {@link #removeIf(Object, IgnitePredicate)} should
      * always be used instead of this one to avoid the overhead associated with returning of the
      * previous value.
      * <p>
-     * If write-through is enabled, the value will be removed from {@link org.gridgain.grid.cache.store.GridCacheStore}
-     * via {@link org.gridgain.grid.cache.store.GridCacheStore#remove(IgniteTx, Object)} method.
+     * If write-through is enabled, the value will be removed from {@link GridCacheStore}
+     * via {@link GridCacheStore#remove(IgniteTx, Object)} method.
      * <h2 class="header">Transactions</h2>
      * This method is transactional and will enlist the entry into ongoing transaction
      * if there is one.
@@ -407,8 +408,8 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * This method will return {@code true} if remove did occur, which means that all optionally
      * provided filters have passed and there was something to remove, {@code false} otherwise.
      * <p>
-     * If write-through is enabled, the value will be removed from {@link org.gridgain.grid.cache.store.GridCacheStore}
-     * via {@link org.gridgain.grid.cache.store.GridCacheStore#remove(IgniteTx, Object)} method.
+     * If write-through is enabled, the value will be removed from {@link GridCacheStore}
+     * via {@link GridCacheStore#remove(IgniteTx, Object)} method.
      * <h2 class="header">Transactions</h2>
      * This method is transactional and will enlist the entry into ongoing transaction
      * if there is one.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/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 a985fde..31940ec 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
@@ -12,7 +12,9 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -28,6 +30,7 @@ import javax.cache.integration.*;
 import javax.cache.processor.*;
 import java.io.*;
 import java.util.*;
+import java.util.concurrent.*;
 import java.util.concurrent.locks.*;
 
 /**
@@ -115,16 +118,41 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
     }
 
     /** {@inheritDoc} */
-    @Override public void loadCache(@Nullable IgniteBiPredicate p, @Nullable Object... args) throws CacheException {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+    @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) {
+        try {
+            GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
+
+            try {
+                ClusterGroup nodes = ctx.kernalContext().grid().cluster().forCache(ctx.name());
+
+                IgniteCompute comp = ctx.kernalContext().grid().compute(nodes).withNoFailover();
+
+                comp.broadcast(new LoadCacheClosure<>(ctx.name(), p, args));
+            }
+            finally {
+                gate.leave(prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void localLoadCache(@Nullable IgniteBiPredicate p, @Nullable Object... args)
-        throws CacheException {
-        // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+    @Override public void localLoadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) {
+        try {
+            GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
+
+            try {
+                delegate.<K, V>cache().loadCache(p, 0, args);
+            }
+            finally {
+                gate.leave(prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -320,7 +348,17 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
     /** {@inheritDoc} */
     @Override public int size(CachePeekMode... peekModes) throws CacheException {
         // TODO IGNITE-1.
-        throw new UnsupportedOperationException();
+        if (peekModes.length != 0)
+            throw new UnsupportedOperationException();
+
+        GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
+
+        try {
+            return delegate.size();
+        }
+        finally {
+            gate.leave(prev);
+        }
     }
 
     /** {@inheritDoc} */
@@ -922,4 +960,74 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter implements
     @Override public String toString() {
         return S.toString(IgniteCacheProxy.class, this);
     }
+
+    /**
+     *
+     */
+    private static class LoadCacheClosure<K, V> implements Callable<Void>, Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private String cacheName;
+
+        /** */
+        private IgniteBiPredicate<K, V> p;
+
+        /** */
+        private Object[] args;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * Required by {@link Externalizable}.
+         */
+        public LoadCacheClosure() {
+            // No-op.
+        }
+
+        /**
+         * @param cacheName Cache name.
+         * @param p Predicate.
+         * @param args Arguments.
+         */
+        private LoadCacheClosure(String cacheName, IgniteBiPredicate<K, V> p, Object[] args) {
+            this.cacheName = cacheName;
+            this.p = p;
+            this.args = args;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void call() throws Exception {
+            IgniteCache<K, V> cache = ignite.jcache(cacheName);
+
+            assert cache != null : cacheName;
+
+            cache.localLoadCache(p, args);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(p);
+
+            out.writeObject(args);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            p = (IgniteBiPredicate<K, V>)in.readObject();
+
+            args = (Object[])in.readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(LoadCacheClosure.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
index 855fdeb..2e91b92 100644
--- a/modules/core/src/test/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/cache/store/GridCacheLoadOnlyStoreAdapterSelfTest.java
@@ -46,7 +46,7 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
      * @throws Exception If failed.
      */
     public void testStore() throws Exception {
-        cache().loadCache(null, 0, 1, 2, 3);
+        jcache().localLoadCache(null, 1, 2, 3);
 
         int cnt = 0;
 
@@ -56,6 +56,9 @@ public class GridCacheLoadOnlyStoreAdapterSelfTest extends GridCacheAbstractSelf
         assertEquals(INPUT_SIZE - (INPUT_SIZE/10), cnt);
     }
 
+    /**
+     *
+     */
     private static class TestStore extends GridCacheLoadOnlyStoreAdapter<Integer, String, String> {
         /** {@inheritDoc} */
         @Override protected Iterator<String> inputIterator(@Nullable Object... args)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 54397cc..8b6f777 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -2341,7 +2341,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             putToStore(key, Integer.parseInt(key));
 
         for (int g = 0; g < gridCount(); g++)
-            grid(g).cache(null).loadCache(null, 0);
+            grid(g).jcache(null).localLoadCache(null);
 
         for (int g = 0; g < gridCount(); g++) {
             for (int i = 0; i < cnt; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
index b4cd689..c88736f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractSelfTest.java
@@ -350,7 +350,8 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     /**
      * @return Default cache instance.
      */
-    protected IgniteCache<String, Integer> jcache() {
+    @SuppressWarnings({"unchecked"})
+    @Override protected IgniteCache<String, Integer> jcache() {
         return jcache(0);
     }
 
@@ -358,6 +359,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      * @param idx Index of grid.
      * @return Default cache.
      */
+    @SuppressWarnings({"unchecked"})
     protected IgniteCache<String, Integer> jcache(int idx) {
         return ignite(idx).jcache(null);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGlobalLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGlobalLoadTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGlobalLoadTest.java
new file mode 100644
index 0000000..6027986
--- /dev/null
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheGlobalLoadTest.java
@@ -0,0 +1,163 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.kernal.processors.cache.distributed.dht;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.transactions.*;
+import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.store.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+import org.junit.*;
+
+import java.util.concurrent.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ * Load cache test.
+ */
+public class GridCacheGlobalLoadTest extends IgniteCacheAbstractTest {
+    /** */
+    private static ConcurrentMap<String, Object[]> map;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheDistributionMode distributionMode() {
+        return NEAR_PARTITIONED;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCache() throws Exception {
+        IgniteCache<Integer, Integer> cache = jcache();
+
+        map = new ConcurrentHashMap8<>();
+
+        cache.loadCache(null, 1, 2, 3);
+
+        assertEquals(3, map.size());
+
+        Object[] expArgs = {1, 2, 3};
+
+        for (int i = 0; i < gridCount(); i++) {
+            Object[] args = map.get(getTestGridName(i));
+
+            Assert.assertArrayEquals(expArgs, args);
+        }
+
+        assertEquals(cache.get(1), (Integer)1);
+        assertEquals(cache.get(2), (Integer)2);
+        assertEquals(cache.get(3), (Integer)3);
+
+        map = new ConcurrentHashMap8<>();
+
+        cache.loadCache(new IgniteBiPredicate<Integer, Integer>() {
+            @Override public boolean apply(Integer key, Integer val) {
+                assertNotNull(key);
+                assertNotNull(val);
+
+                return key % 2 == 0;
+            }
+        }, 1, 2, 3, 4, 5, 6);
+
+        assertEquals(3, map.size());
+
+        expArgs = new Object[]{1, 2, 3, 4, 5, 6};
+
+        for (int i = 0; i < gridCount(); i++) {
+            Object[] args = map.get(getTestGridName(i));
+
+            Assert.assertArrayEquals(expArgs, args);
+        }
+
+        assertEquals(cache.get(1), (Integer)1);
+        assertEquals(cache.get(2), (Integer)2);
+        assertEquals(cache.get(3), (Integer)3);
+        assertEquals(cache.get(4), (Integer)4);
+        assertEquals(cache.get(6), (Integer)6);
+        assertNull(cache.get(5));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        map = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheStore<?, ?> cacheStore() {
+        return new TestStore();
+    }
+
+    /**
+     * Test store.
+     */
+    private static class TestStore extends GridCacheStoreAdapter<Integer, Integer> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo,
+            @Nullable Object... args) throws IgniteCheckedException {
+            assertNotNull(ignite);
+            assertNotNull(clo);
+            assertNotNull(map);
+            assertNotNull(args);
+
+            assertNull(map.put(ignite.name(), args));
+
+            for (Object arg : args) {
+                Integer key = (Integer)arg;
+
+                clo.apply(key, key);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(IgniteTx tx, Integer key) throws IgniteCheckedException {
+            assertEquals((Integer)5, key);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void put(IgniteTx tx, Integer key, Integer val) throws IgniteCheckedException {
+            fail();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove(IgniteTx tx, Integer key) throws IgniteCheckedException {
+            fail();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
index e991574..a96515c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedLoadCacheSelfTest.java
@@ -17,6 +17,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.transactions.*;
 import org.gridgain.grid.cache.*;
+import org.gridgain.grid.cache.affinity.*;
 import org.gridgain.grid.cache.store.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;
@@ -63,27 +64,32 @@ public class GridCachePartitionedLoadCacheSelfTest extends GridCommonAbstractTes
     /**
      * @throws Exception If failed.
      */
-    public void testLoadCache() throws Exception {
+    public void testLocalLoadCache() throws Exception {
         try {
             startGridsMultiThreaded(GRID_CNT);
 
-            GridCache<Integer, String> cache = cache(0);
+            IgniteCache<Integer, String> cache = jcache(0);
 
-            cache.loadCache(null, 0, PUT_CNT);
+            cache.localLoadCache(null, PUT_CNT);
 
-            int[] parts = cache.affinity().allPartitions(grid(0).localNode());
+            GridCache<Integer, String> cache0 = cache(0);
+
+            GridCacheAffinity aff = cache0.affinity();
+
+            int[] parts = aff.allPartitions(grid(0).localNode());
 
             int cnt1 = 0;
 
-            for (int i = 0; i < PUT_CNT; i++)
-                if (U.containsIntArray(parts,  cache.affinity().partition(i)))
+            for (int i = 0; i < PUT_CNT; i++) {
+                if (U.containsIntArray(parts, aff.partition(i)))
                     cnt1++;
+            }
 
             info("Number of keys to load: " + cnt1);
 
             int cnt2 = 0;
 
-            for (GridCacheEntry<Integer, String> e : cache.entrySet()) {
+            for (GridCacheEntry<Integer, String> e : cache0.entrySet()) {
                 assert e.primary() || e.backup();
 
                 cnt2++;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/gridgain/testframework/junits/common/GridCommonAbstractTest.java
index f40d941..33e88e6 100644
--- a/modules/core/src/test/java/org/gridgain/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/testframework/junits/common/GridCommonAbstractTest.java
@@ -86,6 +86,13 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     /**
      * @return Cache.
      */
+    protected <K, V> IgniteCache<K, V> jcache() {
+        return grid().jcache(null);
+    }
+
+    /**
+     * @return Cache.
+     */
     protected <K, V> GridLocalCache<K, V> local() {
         return (GridLocalCache<K, V>)((GridKernal)grid()).<K, V>internalCache();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb32d468/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
index d9a5e22..8082d15 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
@@ -104,6 +104,7 @@ public class GridDataGridTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheOffHeapTieredEvictionSelfTest.class);
         suite.addTestSuite(GridCacheOffHeapTieredAtomicSelfTest.class);
         suite.addTestSuite(GridCacheOffHeapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheGlobalLoadTest.class);
 
         // Local cache.
         suite.addTestSuite(GridCacheLocalProjectionSelfTest.class);