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/12 15:16:09 UTC

incubator-ignite git commit: ignite-1009-v4 decided to completely remove check for store on clients

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1009-v4 [created] 7f97ec757


ignite-1009-v4 decided to completely remove check for store on clients


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

Branch: refs/heads/ignite-1009-v4
Commit: 7f97ec757c9948628678f311ece52978fd53eb5a
Parents: b087aca
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 12 14:12:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 12 16:15:51 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |   6 +-
 .../cache/CacheClientStoreSelfTest.java         | 228 +++++++++++++
 ...acheReadOnlyTransactionalClientSelfTest.java | 327 -------------------
 ...CacheClientWriteBehindStoreAbstractTest.java | 104 ++++++
 ...teCacheClientWriteBehindStoreAtomicTest.java |  38 +++
 .../IgnteCacheClientWriteBehindStoreTxTest.java |  32 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +-
 .../IgniteCacheWriteBehindTestSuite.java        |   2 +
 11 files changed, 413 insertions(+), 341 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 4fdec33..4428b0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -2278,16 +2278,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
                 "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
 
-            boolean checkStore;
-
-            if (!isLocAff && isRmtAff && locCfg.getAtomicityMode() == TRANSACTIONAL) {
-                checkStore = locAttr.storeFactoryClassName() != null;
-
-                if (locAttr.storeFactoryClassName() == null && rmtAttr.storeFactoryClassName() != null)
-                    desc.updatesAllowed(false);
-            }
-            else
-                checkStore = isLocAff && isRmtAff;
+            boolean checkStore = isLocAff && isRmtAff;
 
             if (checkStore)
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 703daf9..4f081bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -859,7 +859,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                         req.isolation(),
                                         req.timeout(),
                                         req.isInvalidate(),
-                                        false,
+                                        true,
                                         req.txSize(),
                                         null,
                                         req.subjectId(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index e6d71aa..e16f7bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -325,7 +325,7 @@ public class IgniteTxHandler {
                     req.isolation(),
                     req.timeout(),
                     req.isInvalidate(),
-                    false,
+                    true,
                     req.txSize(),
                     req.transactionNodes(),
                     req.subjectId(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 8b5eaec..bc6308b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -503,7 +503,11 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                     boolean skipNear = near() && isWriteToStoreFromDht;
 
                     for (IgniteTxEntry e : writeEntries) {
-                        if ((skipNear && e.cached().isNear()) || e.skipStore())
+                        boolean skip = (skipNear && e.cached().isNear()) ||
+                            e.skipStore() ||
+                            (e.context().store().isLocal() && !e.context().affinityNode());
+
+                        if (skip)
                             continue;
 
                         boolean intercept = e.context().config().getInterceptor() != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
new file mode 100644
index 0000000..44b27be
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheClientStoreSelfTest.java
@@ -0,0 +1,228 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.configuration.*;
+import javax.cache.processor.*;
+
+import static org.apache.ignite.IgniteSystemProperties.*;
+
+/**
+ * Tests for cache client without store.
+ */
+public class CacheClientStoreSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_NAME = "test-cache";
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean nearEnabled;
+
+    /** */
+    private Factory<CacheStore> factory;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration cc = new CacheConfiguration();
+
+        cc.setName(CACHE_NAME);
+        cc.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        cc.setCacheStoreFactory(factory);
+
+        if (client && nearEnabled)
+            cc.setNearConfiguration(new NearCacheConfiguration());
+
+        cfg.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        client = false;
+        factory = new Factory1();
+
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopGrid();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCorrectStore() throws Exception {
+        client = true;
+        nearEnabled = false;
+        factory = new Factory1();
+
+        Ignite ignite = startGrid();
+
+        IgniteCache cache = ignite.cache(CACHE_NAME);
+
+        cache.get(0);
+        cache.getAll(F.asSet(0, 1));
+        cache.getAndPut(0, 0);
+        cache.getAndPutIfAbsent(0, 0);
+        cache.getAndRemove(0);
+        cache.getAndReplace(0, 0);
+        cache.put(0, 0);
+        cache.putAll(F.asMap(0, 0, 1, 1));
+        cache.putIfAbsent(0, 0);
+        cache.remove(0);
+        cache.remove(0, 0);
+        cache.removeAll(F.asSet(0, 1));
+        cache.removeAll();
+        cache.invoke(0, new EP());
+        cache.invokeAll(F.asSet(0, 1), new EP());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvalidStore() throws Exception {
+        client = true;
+        nearEnabled = false;
+        factory = new Factory2();
+
+        startGrid();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDisabledConsistencyCheck() throws Exception {
+        client = false;
+        nearEnabled = false;
+        factory = new Factory2();
+
+        System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, "true");
+
+        startGrid("client-1");
+
+        factory = new Factory1();
+
+        System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
+
+        startGrid("client-2");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoStoreNearDisabled() throws Exception {
+        nearEnabled = false;
+
+        doTestNoStore();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoStoreNearEnabled() throws Exception {
+        nearEnabled = true;
+
+        doTestNoStore();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestNoStore() throws Exception {
+        client = true;
+        factory = null;
+
+        Ignite ignite = startGrid();
+
+        IgniteCache cache = ignite.cache(CACHE_NAME);
+
+        cache.get(0);
+        cache.getAll(F.asSet(0, 1));
+        cache.getAndPut(0, 0);
+        cache.getAndPutIfAbsent(0, 0);
+        cache.getAndRemove(0);
+        cache.getAndReplace(0, 0);
+        cache.put(0, 0);
+        cache.putAll(F.asMap(0, 0, 1, 1));
+        cache.putIfAbsent(0, 0);
+        cache.remove(0);
+        cache.remove(0, 0);
+        cache.removeAll(F.asSet(0, 1));
+        cache.removeAll();
+        cache.invoke(0, new EP());
+        cache.invokeAll(F.asSet(0, 1), new EP());
+    }
+
+    /**
+     */
+    private static class Factory1 implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return null;
+        }
+    }
+
+    /**
+     */
+    private static class Factory2 implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return null;
+        }
+    }
+
+    /**
+     */
+    private static class EP implements CacheEntryProcessor {
+        @Override public Object process(MutableEntry entry, Object... arguments) {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
deleted file mode 100644
index f2c38e1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
+++ /dev/null
@@ -1,327 +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.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import javax.cache.*;
-import javax.cache.configuration.*;
-import javax.cache.processor.*;
-
-import static org.apache.ignite.IgniteSystemProperties.*;
-
-/**
- * Tests for read-only transactional cache client.
- */
-public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final String CACHE_NAME = "test-cache";
-
-    /** */
-    private boolean client;
-
-    /** */
-    private boolean nearEnabled;
-
-    /** */
-    private Factory<CacheStore> factory;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setClientMode(client);
-
-        CacheConfiguration cc = new CacheConfiguration();
-
-        cc.setName(CACHE_NAME);
-        cc.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        cc.setCacheStoreFactory(factory);
-
-        if (client && nearEnabled)
-            cc.setNearConfiguration(new NearCacheConfiguration());
-
-        cfg.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        client = false;
-        factory = new Factory1();
-
-        startGrids(2);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopGrid();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCorrectStore() throws Exception {
-        client = true;
-        nearEnabled = false;
-        factory = new Factory1();
-
-        Ignite ignite = startGrid();
-
-        IgniteCache cache = ignite.cache(CACHE_NAME);
-
-        cache.get(0);
-        cache.getAll(F.asSet(0, 1));
-        cache.getAndPut(0, 0);
-        cache.getAndPutIfAbsent(0, 0);
-        cache.getAndRemove(0);
-        cache.getAndReplace(0, 0);
-        cache.put(0, 0);
-        cache.putAll(F.asMap(0, 0, 1, 1));
-        cache.putIfAbsent(0, 0);
-        cache.remove(0);
-        cache.remove(0, 0);
-        cache.removeAll(F.asSet(0, 1));
-        cache.removeAll();
-        cache.invoke(0, new EP());
-        cache.invokeAll(F.asSet(0, 1), new EP());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testInvalidStore() throws Exception {
-        client = true;
-        nearEnabled = false;
-        factory = new Factory2();
-
-        try {
-            startGrid();
-
-            assert false : "Exception was not thrown.";
-        }
-        catch (Exception e) {
-            assert e.getMessage().startsWith("Store factory mismatch") : e.getMessage();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDisabledConsistencyCheck() throws Exception {
-        client = false;
-        nearEnabled = false;
-        factory = new Factory2();
-
-        System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, "true");
-
-        startGrid("client-1");
-
-        factory = new Factory1();
-
-        System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
-
-        startGrid("client-2");
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoStoreNearDisabled() throws Exception {
-        nearEnabled = false;
-
-        doTestNoStore();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoStoreNearEnabled() throws Exception {
-        nearEnabled = true;
-
-        doTestNoStore();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void doTestNoStore() throws Exception {
-        client = true;
-        factory = null;
-
-        Ignite ignite = startGrid();
-
-        IgniteCache cache = ignite.cache(CACHE_NAME);
-
-        cache.get(0);
-        cache.getAll(F.asSet(0, 1));
-
-        try {
-            cache.getAndPut(0, 0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.getAndPutIfAbsent(0, 0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.getAndRemove(0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.getAndReplace(0, 0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.put(0, 0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.putAll(F.asMap(0, 0, 1, 1));
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.putIfAbsent(0, 0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.remove(0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.remove(0, 0);
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.removeAll(F.asSet(0, 1));
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.removeAll();
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.invoke(0, new EP());
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-
-        try {
-            cache.invokeAll(F.asSet(0, 1), new EP());
-        }
-        catch (CacheException e) {
-            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
-                e.getMessage();
-        }
-    }
-
-    /**
-     */
-    private static class Factory1 implements Factory<CacheStore> {
-        /** {@inheritDoc} */
-        @Override public CacheStore create() {
-            return null;
-        }
-    }
-
-    /**
-     */
-    private static class Factory2 implements Factory<CacheStore> {
-        /** {@inheritDoc} */
-        @Override public CacheStore create() {
-            return null;
-        }
-    }
-
-    /**
-     */
-    private static class EP implements CacheEntryProcessor {
-        @Override public Object process(MutableEntry entry, Object... arguments) {
-            return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAbstractTest.java
new file mode 100644
index 0000000..f7c150d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAbstractTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.store;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.testframework.*;
+
+import javax.cache.configuration.*;
+
+/**
+ * Tests that write behind store is updated if client does not have store.
+ */
+public abstract class IgnteCacheClientWriteBehindStoreAbstractTest extends IgniteCacheAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+        ccfg.setWriteBehindEnabled(true);
+        ccfg.setWriteBehindBatchSize(10);
+
+        if (getTestGridName(2).equals(gridName)) {
+            ccfg.setCacheStoreFactory(null);
+            ccfg.setWriteThrough(false);
+            ccfg.setReadThrough(false);
+            ccfg.setWriteBehindEnabled(false);
+        }
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(2).equals(gridName))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStore> cacheStoreFactory() {
+        return new TestStoreFactory();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientWithoutStore() throws Exception {
+        Ignite client = grid(2);
+
+        assertTrue(client.configuration().isClientMode());
+
+        IgniteCache<Integer, Integer> cache = client.cache(null);
+
+        assertNull(cache.getConfiguration(CacheConfiguration.class).getCacheStoreFactory());
+
+        for (int i = 0; i < 1000; i++)
+            cache.put(i, i);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return storeMap.size() == 1000;
+            }
+        }, 5000);
+
+        assertEquals(1000, storeMap.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAtomicTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAtomicTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAtomicTest.java
new file mode 100644
index 0000000..72ed3d6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreAtomicTest.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.internal.processors.cache.store;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class IgnteCacheClientWriteBehindStoreAtomicTest extends IgnteCacheClientWriteBehindStoreAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreTxTest.java
new file mode 100644
index 0000000..a5ced98
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/IgnteCacheClientWriteBehindStoreTxTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.store;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class IgnteCacheClientWriteBehindStoreTxTest extends IgnteCacheClientWriteBehindStoreAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index c598e38..83a30bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -134,7 +134,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class);
 
-        suite.addTestSuite(CacheReadOnlyTransactionalClientSelfTest.class);
+        suite.addTestSuite(CacheClientStoreSelfTest.class);
 
         suite.addTestSuite(IgniteCacheManyClientsTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f97ec75/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java
index 529b227..5abc8b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheWriteBehindTestSuite.java
@@ -40,6 +40,8 @@ public class IgniteCacheWriteBehindTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheWriteBehindStorePartitionedTest.class));
         suite.addTest(new TestSuite(GridCacheWriteBehindStorePartitionedMultiNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedWritesTest.class));
+        suite.addTest(new TestSuite(IgnteCacheClientWriteBehindStoreAtomicTest.class));
+        suite.addTest(new TestSuite(IgnteCacheClientWriteBehindStoreTxTest.class));
 
         return suite;
     }