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

[1/2] incubator-ignite git commit: IGNITE-941 - Read-only mode for transactional cache

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-941 [created] e9cea3a31


IGNITE-941 - Read-only mode for transactional cache


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

Branch: refs/heads/ignite-941
Commit: 54c72c3e7fb89a3f9e6457dd82aa83af1af4ba1c
Parents: 97d0bc1
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 3 20:06:33 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 3 20:06:33 2015 -0700

----------------------------------------------------------------------
 .../cache/DynamicCacheDescriptor.java           |  17 ++
 .../processors/cache/GridCacheContext.java      |  16 +-
 .../processors/cache/GridCacheProcessor.java    |  37 ++-
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +
 .../transactions/IgniteTxLocalAdapter.java      |   7 +
 ...acheReadOnlyTransactionalClientSelfTest.java | 289 +++++++++++++++++++
 .../loadtests/hashmap/GridCacheTestContext.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 8 files changed, 362 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index a27ebd4..9c6cc43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -61,6 +61,9 @@ public class DynamicCacheDescriptor {
     /** Cache plugin manager. */
     private final CachePluginManager pluginMgr;
 
+    /** */
+    private boolean updatesAllowed = true;
+
     /**
      * @param ctx Context.
      * @param cacheCfg Cache configuration.
@@ -208,6 +211,20 @@ public class DynamicCacheDescriptor {
         rmtCfgs = null;
     }
 
+    /**
+     * @return Updates allowed flag.
+     */
+    public boolean updatesAllowed() {
+        return updatesAllowed;
+    }
+
+    /**
+     * @param updatesAllowed Updates allowed flag.
+     */
+    public void updatesAllowed(boolean updatesAllowed) {
+        this.updatesAllowed = updatesAllowed;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheDescriptor.class, this, "cacheName", U.maskName(cacheCfg.getName()));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index b20e59d..8a4e3b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -196,6 +196,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Dynamic cache deployment ID. */
     private IgniteUuid dynamicDeploymentId;
 
+    /** Updates allowed flag. */
+    private boolean updatesAllowed;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -209,6 +212,7 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param cacheCfg Cache configuration.
      * @param cacheType Cache type.
      * @param affNode {@code True} if local node is affinity node.
+     * @param updatesAllowed Updates allowed flag.
      * @param evtMgr Cache event manager.
      * @param swapMgr Cache swap manager.
      * @param storeMgr Store manager.
@@ -230,6 +234,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         CacheConfiguration cacheCfg,
         CacheType cacheType,
         boolean affNode,
+        boolean updatesAllowed,
 
         /*
          * Managers in starting order!
@@ -271,6 +276,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.cacheCfg = cacheCfg;
         this.cacheType = cacheType;
         this.affNode = affNode;
+        this.updatesAllowed = updatesAllowed;
 
         /*
          * Managers in starting order!
@@ -1469,9 +1475,6 @@ public class GridCacheContext<K, V> implements Externalizable {
             Collection<ClusterNode> dhtNodeIds = new ArrayList<>(dhtRemoteNodes);
             Collection<ClusterNode> nearNodeIds = F.isEmpty(nearRemoteNodes) ? null : new ArrayList<>(nearRemoteNodes);
 
-            if (!F.isEmpty(nearNodeIds))
-                U.dumpStack("Added near mapped nodes: " + entry + ", " + nearNodeIds);
-
             entry.mappings(explicitLockVer, dhtNodeIds, nearNodeIds);
         }
 
@@ -1809,6 +1812,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Updates allowed.
+     */
+    public boolean updatesAllowed() {
+        return updatesAllowed;
+    }
+
+    /**
      * Nulling references to potentially leak-prone objects.
      */
     public void cleanup() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/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 24e2439..bbd5fe9 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
@@ -674,6 +674,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
                 for (ClusterNode n : ctx.discovery().remoteNodes()) {
+                    if (Boolean.valueOf(n.<String>attribute(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)))
+                        continue;
+
                     checkTransactionConfiguration(n);
 
                     DeploymentMode locDepMode = ctx.config().getDeploymentMode();
@@ -688,7 +691,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         if (rmtCfg != null) {
                             CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                            checkCache(locCfg, rmtCfg, n);
+                            checkCache(locCfg, rmtCfg, n, desc);
 
                             // Check plugin cache configurations.
                             CachePluginManager pluginMgr = desc.pluginManager();
@@ -718,7 +721,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     CachePluginManager pluginMgr = desc.pluginManager();
 
-                    GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
+                    GridCacheContext ctx = createCache(
+                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
 
                     ctx.dynamicDeploymentId(desc.deploymentId());
 
@@ -1068,7 +1072,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private GridCacheContext createCache(CacheConfiguration<?, ?> cfg,
         @Nullable CachePluginManager pluginMgr,
         CacheType cacheType,
-        CacheObjectContext cacheObjCtx)
+        CacheObjectContext cacheObjCtx,
+        boolean updatesAllowed)
         throws IgniteCheckedException
     {
         assert cfg != null;
@@ -1126,6 +1131,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cfg,
             cacheType,
             ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),
+            updatesAllowed,
 
             /*
              * Managers in starting order!
@@ -1255,6 +1261,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cfg,
                 cacheType,
                 ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),
+                true,
 
                 /*
                  * Managers in starting order!
@@ -1458,7 +1465,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-            GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx);
+            GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx, true);
 
             cacheCtx.startTopologyVersion(topVer);
 
@@ -2229,10 +2236,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param locCfg Local configuration.
      * @param rmtCfg Remote configuration.
      * @param rmtNode Remote node.
+     * @param desc Cache descriptor.
      * @throws IgniteCheckedException If check failed.
      */
-    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode)
-        throws IgniteCheckedException {
+    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode,
+        DynamicCacheDescriptor desc) throws IgniteCheckedException {
         ClusterNode locNode = ctx.discovery().localNode();
 
         UUID rmt = rmtNode.id();
@@ -2240,6 +2248,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheAttributes rmtAttr = new GridCacheAttributes(rmtCfg);
         GridCacheAttributes locAttr = new GridCacheAttributes(locCfg);
 
+        boolean isLocAff = CU.affinityNode(locNode, locCfg.getNodeFilter());
+        boolean isRmtAff = CU.affinityNode(rmtNode, rmtCfg.getNodeFilter());
+
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheMode", "Cache mode",
             locAttr.cacheMode(), rmtAttr.cacheMode(), true);
 
@@ -2253,8 +2264,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
                 "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
 
-            if (locCfg.getAtomicityMode() == TRANSACTIONAL ||
-                (CU.affinityNode(rmtNode, rmtCfg.getNodeFilter()) && CU.affinityNode(locNode, locCfg.getNodeFilter())))
+            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;
+
+            if (checkStore)
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                     locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 9ca80f9..3208aeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -737,6 +737,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable final CacheEntryPredicate[] filter,
         final boolean waitTopFut
     ) {
+        assert ctx.updatesAllowed();
+
         if (map != null && keyCheck)
             validateCacheKeys(map.keySet());
 
@@ -793,6 +795,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         boolean rawRetval,
         @Nullable final CacheEntryPredicate[] filter
     ) {
+        assert ctx.updatesAllowed();
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/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 3e5a63c..4063be2 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
@@ -40,6 +40,7 @@ import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
 import javax.cache.expiry.*;
 import javax.cache.processor.*;
 import java.io.*;
@@ -2560,6 +2561,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     ) {
         assert filter == null || invokeMap == null;
 
+        if (!cacheCtx.updatesAllowed())
+            throw new CacheException("Updates are not allowed for cache: " + cacheCtx.name());
+
         cacheCtx.checkSecurity(SecurityPermission.CACHE_PUT);
 
         if (retval)
@@ -2780,6 +2784,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         @Nullable GridCacheEntryEx cached,
         final boolean retval,
         @Nullable final CacheEntryPredicate[] filter) {
+        if (!cacheCtx.updatesAllowed())
+            throw new CacheException("Updates are not allowed for cache: " + cacheCtx.name());
+
         cacheCtx.checkSecurity(SecurityPermission.CACHE_REMOVE);
 
         if (retval)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/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
new file mode 100644
index 0000000..e2a1def
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
@@ -0,0 +1,289 @@
+/*
+ * 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.*;
+
+/**
+ * Tests for read-only transactional cache client.
+ */
+public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    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.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(null);
+
+        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 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(null);
+
+        cache.get(0);
+        cache.getAll(F.asSet(0, 1));
+
+        try {
+            cache.getAndPut(0, 0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.getAndPutIfAbsent(0, 0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.getAndRemove(0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.getAndReplace(0, 0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.put(0, 0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.putAll(F.asMap(0, 0, 1, 1));
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.putIfAbsent(0, 0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.remove(0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.remove(0, 0);
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.removeAll(F.asSet(0, 1));
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.removeAll();
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.invoke(0, new EP());
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", e.getMessage());
+        }
+
+        try {
+            cache.invokeAll(F.asSet(0, 1), new EP());
+        }
+        catch (CacheException e) {
+            assertEquals("Updates are not allowed for cache: null", 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/54c72c3e/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 1c85ed3..77d3905 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -60,6 +60,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
             defaultCacheConfiguration(),
             CacheType.USER,
             true,
+            true,
             new GridCacheEventManager(),
             new GridCacheSwapManager(false),
             new CacheOsStoreManager(null, new CacheConfiguration()),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54c72c3e/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 a8019d2..ab304fb 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
@@ -135,6 +135,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class);
 
+        suite.addTestSuite(CacheReadOnlyTransactionalClientSelfTest.class);
+
         return suite;
     }
 }


[2/2] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-941

Posted by vk...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-941


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

Branch: refs/heads/ignite-941
Commit: e9cea3a311cd1ad9728a0ae14de9ba91f3859d1b
Parents: 54c72c3 bd3abbc
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 3 20:06:43 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 3 20:06:43 2015 -0700

----------------------------------------------------------------------

----------------------------------------------------------------------