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/14 15:03:40 UTC

[61/63] [abbrv] incubator-ignite git commit: # ignite-42

# ignite-42


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

Branch: refs/heads/ignite-42
Commit: bafb441173d10ec01e39501bab18080ae4a9e2eb
Parents: cbe46e1
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jan 14 15:52:09 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jan 14 16:57:04 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/store/CacheStoreSession.java   |  38 +++++
 .../resources/IgniteCacheSessionResource.java   |  29 ++++
 .../grid/cache/GridCacheConfiguration.java      |   2 +
 .../grid/cache/store/GridCacheStoreAdapter.java |  13 ++
 .../cache/GridCacheLoaderWriterStore.java       |  22 ++-
 .../processors/cache/GridCacheProcessor.java    |   4 +-
 .../processors/cache/GridCacheStoreManager.java | 142 +++++++++++++++++--
 .../cache/IgniteCacheAbstractTest.java          |  21 +++
 .../IgniteCacheAtomicLoaderWriterTest.java      |  55 +++++++
 .../IgniteCacheLoaderWriterAbstractTest.java    |  82 +++++++++++
 .../loadtests/hashmap/GridCacheTestContext.java |   6 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |  14 +-
 12 files changed, 406 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
new file mode 100644
index 0000000..2331e02
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
@@ -0,0 +1,38 @@
+/*
+ * 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.cache.store;
+
+import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public interface CacheStoreSession {
+    /**
+     * @return Current cache transaction.
+     */
+    @Nullable public IgniteTx transaction();
+
+    /**
+     * @return Session properties.
+     */
+    public Map<Object, Object> properties();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/apache/ignite/resources/IgniteCacheSessionResource.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/resources/IgniteCacheSessionResource.java b/modules/core/src/main/java/org/apache/ignite/resources/IgniteCacheSessionResource.java
new file mode 100644
index 0000000..c892dd7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/resources/IgniteCacheSessionResource.java
@@ -0,0 +1,29 @@
+/*
+ * 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.resources;
+
+import java.lang.annotation.*;
+
+/**
+ * TODO
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD, ElementType.FIELD})
+public @interface IgniteCacheSessionResource {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
index 5df8b75..3118552 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
@@ -370,6 +370,8 @@ public class GridCacheConfiguration extends MutableConfiguration {
         indexingSpiName = cc.getIndexingSpiName();
         interceptor = cc.getInterceptor();
         invalidate = cc.isInvalidate();
+        isReadThrough = cc.isReadThrough();
+        isWriteThrough = cc.isWriteThrough();
         keepPortableInStore = cc.isKeepPortableInStore();
         offHeapMaxMem = cc.getOffHeapMaxMemory();
         maxConcurrentAsyncOps = cc.getMaxConcurrentAsyncOperations();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
index e07a1f1..2d3767b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/store/GridCacheStoreAdapter.java
@@ -10,7 +10,9 @@
 package org.gridgain.grid.cache.store;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
 import org.gridgain.grid.cache.*;
 import org.jetbrains.annotations.*;
@@ -32,6 +34,17 @@ import java.util.*;
  * specific arguments.
  */
 public abstract class GridCacheStoreAdapter<K, V> implements GridCacheStore<K, V> {
+    /** */
+    @IgniteCacheSessionResource
+    private CacheStoreSession ses;
+
+    /**
+     * @return Current session.
+     */
+    protected CacheStoreSession session() {
+        return ses;
+    }
+
     /**
      * Default empty implementation. This method needs to be overridden only if
      * {@link GridCache#loadCache(org.apache.ignite.lang.IgniteBiPredicate, long, Object...)} method

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheLoaderWriterStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheLoaderWriterStore.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheLoaderWriterStore.java
index 0e7bd01..d787c94 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheLoaderWriterStore.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheLoaderWriterStore.java
@@ -1,10 +1,18 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+/*
+ * 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.gridgain.grid.kernal.processors.cache;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
index c7f9c48..953929c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheProcessor.java
@@ -223,7 +223,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheWriter writer = null;
 
-            if (cfg.isWriteBehindEnabled() && writerFactory != null)
+            if (cfg.isWriteThrough() && writerFactory != null)
                 writer = writerFactory.create();
 
             if (ldr != null || writer != null)
@@ -595,7 +595,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             GridCacheStore store = cacheStore(ctx.gridName(), cfg);
 
-            GridCacheStoreManager storeMgr = new GridCacheStoreManager(store);
+            GridCacheStoreManager storeMgr = new GridCacheStoreManager(ctx, store);
 
             GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
                 ctx,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
index eab631d..1e699bd 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheStoreManager.java
@@ -10,10 +10,13 @@
 package org.gridgain.grid.kernal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
 import org.gridgain.grid.cache.store.*;
+import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.interop.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.typedef.*;
@@ -27,22 +30,31 @@ import java.util.*;
  */
 public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
     /** */
+    private static final String SES_ATTR = "STORE_SES";
+
+    /** */
     private final GridCacheStore<K, Object> store;
 
     /** */
     private final GridCacheStoreBalancingWrapper<K, Object> singleThreadGate;
 
     /** */
+    private final ThreadLocal<SessionData> sesHolder = new ThreadLocal<>();
+
+    /** */
     private final boolean locStore;
 
     /** */
     private boolean convertPortable;
 
     /**
+     * @param ctx Kernal context.
      * @param store Store.
+     * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings("unchecked")
-    public GridCacheStoreManager(@Nullable GridCacheStore<K, Object> store) {
+    public GridCacheStoreManager(GridKernalContext ctx, @Nullable GridCacheStore<K, Object> store)
+        throws IgniteCheckedException {
         this.store = store;
 
         singleThreadGate = store == null ? null : new GridCacheStoreBalancingWrapper<>(store);
@@ -50,6 +62,9 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
         if (store instanceof GridCacheWriteBehindStore)
             store = ((GridCacheWriteBehindStore)store).store();
 
+        if (store != null)
+            ctx.resource().injectBasicResource(store, IgniteCacheSessionResource.class, new ThreadLocalSession());
+
         locStore = U.hasAnnotation(store, GridCacheLocalStore.class);
     }
 
@@ -136,12 +151,18 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
 
             V val = null;
 
+            boolean ses = initSession(tx);
+
             try {
                 val = convert(singleThreadGate.load(tx, key));
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
             }
+            finally {
+                if (ses)
+                    sesHolder.set(null);
+            }
 
             if (log.isDebugEnabled())
                 log.debug("Loaded value from store [key=" + key + ", val=" + val + ']');
@@ -204,6 +225,8 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
                 if (log.isDebugEnabled())
                     log.debug("Loading values from store for keys: " + keys0);
 
+                boolean ses = initSession(tx);
+
                 try {
                     singleThreadGate.loadAll(tx, keys0, new CI2<K, Object>() {
                         @Override public void apply(K k, Object o) {
@@ -224,6 +247,9 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
                 catch (IgniteException e) {
                     throw U.cast(e);
                 }
+                finally {
+                    sesHolder.set(null);
+                }
 
                 if (log.isDebugEnabled())
                     log.debug("Loaded values from store for keys: " + keys0);
@@ -317,12 +343,18 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
             if (log.isDebugEnabled())
                 log.debug("Storing value in cache store [key=" + key + ", val=" + val + ']');
 
+            boolean ses = initSession(tx);
+
             try {
                 store.put(tx, key, locStore ? F.t(val, ver) : val);
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
             }
+            finally {
+                if (ses)
+                    sesHolder.set(null);
+            }
 
             if (log.isDebugEnabled())
                 log.debug("Stored value in cache store [key=" + key + ", val=" + val + ']');
@@ -371,6 +403,8 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
                 if (log.isDebugEnabled())
                     log.debug("Storing values in cache store [map=" + map0 + ']');
 
+                boolean ses = initSession(tx);
+
                 try {
                     store.putAll(tx, locStore ? map0 : F.viewReadOnly(map0,
                         new C1<IgniteBiTuple<V, GridCacheVersion>, Object>() {
@@ -382,6 +416,9 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
                 catch (ClassCastException e) {
                     handleClassCastException(e);
                 }
+                finally {
+                    sesHolder.set(null);
+                }
 
                 if (log.isDebugEnabled())
                     log.debug("Stored value in cache store [map=" + map0 + ']');
@@ -411,13 +448,17 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
             if (log.isDebugEnabled())
                 log.debug("Removing value from cache store [key=" + key + ']');
 
-            else {
-                try {
-                    store.remove(tx, key);
-                }
-                catch (ClassCastException e) {
-                    handleClassCastException(e);
-                }
+            boolean ses = initSession(tx);
+
+            try {
+                store.remove(tx, key);
+            }
+            catch (ClassCastException e) {
+                handleClassCastException(e);
+            }
+            finally {
+                if (ses)
+                    sesHolder.set(null);
             }
 
             if (log.isDebugEnabled())
@@ -457,12 +498,18 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
             if (log.isDebugEnabled())
                 log.debug("Removing values from cache store [keys=" + keys0 + ']');
 
+            boolean ses = initSession(tx);
+
             try {
                 store.removeAll(tx, keys0);
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
             }
+            finally {
+                if (ses)
+                    sesHolder.set(null);
+            }
 
             if (log.isDebugEnabled())
                 log.debug("Removed values from cache store [keys=" + keys0 + ']');
@@ -494,6 +541,10 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
      * @throws IgniteCheckedException If failed.
      */
     public void txEnd(IgniteTx tx, boolean commit) throws IgniteCheckedException {
+        assert store != null;
+
+        tx.removeMeta(SES_ATTR);
+
         store.txEnd(tx, commit);
     }
 
@@ -512,4 +563,79 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
         else
             throw e;
     }
+
+    /**
+     * @param tx Current transaction.
+     * @return {@code True} if
+     */
+    private boolean initSession(@Nullable IgniteTx tx) {
+        if (tx == null)
+            return false;
+
+        SessionData ses = tx.meta(SES_ATTR);
+
+        if (ses == null) {
+            ses = new SessionData(tx);
+
+            tx.addMeta(SES_ATTR, ses);
+        }
+
+        sesHolder.set(ses);
+
+        return true;
+    }
+
+    /**
+     *
+     */
+    private static class SessionData {
+        /** */
+        private final IgniteTx tx;
+
+        /** */
+        private Map<Object, Object> props;
+
+        /**
+         * @param tx Current transaction.
+         */
+        private SessionData(@Nullable IgniteTx tx) {
+            this.tx = tx;
+        }
+
+        /**
+         * @return Transaction.
+         */
+        @Nullable private IgniteTx transaction() {
+            return tx;
+        }
+
+        /**
+         * @return Properties.
+         */
+        private Map<Object, Object> properties() {
+            if (props == null)
+                props = U.newHashMap(1);
+
+            return props;
+        }
+    }
+
+    /**
+     *
+     */
+    private class ThreadLocalSession implements CacheStoreSession {
+        /** {@inheritDoc} */
+        @Nullable @Override public IgniteTx transaction() {
+            SessionData ses0 = sesHolder.get();
+
+            return ses0 != null ? ses0.transaction() : null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<Object, Object> properties() {
+            SessionData ses0 = sesHolder.get();
+
+            return ses0 != null ? ses0.properties() : null;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
index ac5514d..04390bc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
@@ -23,6 +23,8 @@ import org.gridgain.testframework.junits.common.*;
 import org.jdk8.backport.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
 import java.util.*;
 
 import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
@@ -96,6 +98,7 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
      * @return Cache configuration.
      * @throws Exception In case of error.
      */
+    @SuppressWarnings("unchecked")
     protected GridCacheConfiguration cacheConfiguration(String gridName) throws Exception {
         GridCacheConfiguration cfg = defaultCacheConfiguration();
 
@@ -113,6 +116,10 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
         cfg.setDistributionMode(distributionMode());
         cfg.setPortableEnabled(portableEnabled());
 
+        cfg.setCacheLoaderFactory(loaderFactory());
+
+        cfg.setCacheWriterFactory(writerFactory());
+
         cfg.setStore(cacheStore());
 
         if (cacheMode() == PARTITIONED)
@@ -129,6 +136,20 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @return Cache loader factory.
+     */
+    protected Factory<? extends CacheLoader> loaderFactory() {
+        return null;
+    }
+
+    /**
+     * @return Cache writer factory.
+     */
+    protected Factory<? extends CacheWriter> writerFactory() {
+        return null;
+    }
+
+    /**
      * @return Default cache mode.
      */
     protected abstract GridCacheMode cacheMode();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLoaderWriterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLoaderWriterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLoaderWriterTest.java
new file mode 100644
index 0000000..a63133a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheAtomicLoaderWriterTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.integration;
+
+import org.gridgain.grid.cache.*;
+
+import static org.gridgain.grid.cache.GridCacheAtomicWriteOrderMode.*;
+import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
+import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicLoaderWriterTest extends IgniteCacheLoaderWriterAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected GridCacheDistributionMode distributionMode() {
+        return PARTITIONED_ONLY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
new file mode 100644
index 0000000..c9f5634
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheLoaderWriterAbstractTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.integration;
+
+import org.apache.ignite.internal.processors.cache.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.*;
+
+/**
+ *
+ */
+public abstract class IgniteCacheLoaderWriterAbstractTest extends IgniteCacheAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheLoader> loaderFactory() {
+        return super.loaderFactory();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheWriter> writerFactory() {
+        return super.writerFactory();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoad() throws Exception {
+    }
+
+    /**
+     *
+     */
+    static class TestLoader implements CacheLoader<Integer, Integer> {
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<Integer, Integer> loadAll(Iterable<? extends Integer> keys) {
+            return null;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestWriter implements CacheWriter<Integer, Integer> {
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry) {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeAll(Collection<Cache.Entry<? extends Integer, ? extends Integer>> entries) {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void deleteAll(Collection<?> keys) {
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridCacheTestContext.java
index adae929..8521f82 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridCacheTestContext.java
@@ -25,9 +25,11 @@ import static org.gridgain.testframework.junits.GridAbstractTest.*;
  */
 public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
     /**
+     * @param ctx Context.
+     * @throws Exception If failed.
      */
     @SuppressWarnings("NullableProblems")
-    public GridCacheTestContext(GridTestKernalContext ctx) {
+    public GridCacheTestContext(GridTestKernalContext ctx) throws Exception {
         super(
             ctx,
             new GridCacheSharedContext<>(
@@ -42,7 +44,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             defaultCacheConfiguration(),
             new GridCacheEventManager<K, V>(),
             new GridCacheSwapManager<K, V>(false),
-            new GridCacheStoreManager<K, V>(null),
+            new GridCacheStoreManager<K, V>(null, null),
             new GridCacheEvictionManager<K, V>(),
             new GridCacheLocalQueryManager<K, V>(),
             new GridCacheContinuousQueryManager<K, V>(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bafb4411/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridHashMapLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridHashMapLoadTest.java
index a64a733..2c8e9b8 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridHashMapLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/hashmap/GridHashMapLoadTest.java
@@ -23,6 +23,9 @@ import java.util.concurrent.*;
  */
 @SuppressWarnings("InfiniteLoopStatement")
 public class GridHashMapLoadTest extends GridCommonAbstractTest {
+    /**
+     *
+     */
     public void testHashMapLoad() {
         Map<Integer, Integer> map = new HashMap<>(5 * 1024 * 1024);
 
@@ -36,6 +39,9 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     *
+     */
     public void testConcurrentHashMapLoad() {
         Map<Integer, Integer> map = new ConcurrentHashMap<>(5 * 1024 * 1024);
 
@@ -49,9 +55,11 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
         }
     }
 
-    public void testMapEntry() {
-        Map<Integer, GridCacheMapEntry<Integer, Integer>> map =
-            new HashMap<>(5 * 1024 * 1024);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapEntry() throws Exception {
+        Map<Integer, GridCacheMapEntry<Integer, Integer>> map = new HashMap<>(5 * 1024 * 1024);
 
         int i = 0;