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/07/10 12:47:16 UTC

[25/50] [abbrv] incubator-ignite git commit: # Fixed unmarshalling error handling for cache 'get'

# Fixed unmarshalling error handling for cache 'get'


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

Branch: refs/heads/ignite-1056
Commit: 6d6ec778b43d730dd19001011aaaa49ec86c5d20
Parents: 94a42a4
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 8 15:08:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 8 15:08:33 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  8 ++-
 .../distributed/near/GridNearGetFuture.java     |  4 +-
 .../cache/CacheFutureExceptionSelfTest.java     | 72 +++++++++++++++++---
 3 files changed, 71 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d6ec778/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 0707096..29e3551 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -445,8 +445,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
             case 50: {
                 GridNearGetResponse res = (GridNearGetResponse)msg;
 
-                GridPartitionedGetFuture fut = (GridPartitionedGetFuture)ctx.mvcc().future(
-                    res.version(), res.futureId());
+                GridCacheFuture fut = ctx.mvcc().future(res.version(), res.futureId());
 
                 if (fut == null) {
                     if (log.isDebugEnabled())
@@ -457,7 +456,10 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
                 res.error(res.classError());
 
-                fut.onResult(nodeId, res);
+                if (fut instanceof GridNearGetFuture)
+                    ((GridNearGetFuture)fut).onResult(nodeId, res);
+                else
+                    ((GridPartitionedGetFuture)fut).onResult(nodeId, res);
             }
 
             break;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d6ec778/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 74438bb..58f6fe5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -223,7 +223,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param nodeId Sender.
      * @param res Result.
      */
-    void onResult(UUID nodeId, GridNearGetResponse res) {
+    public void onResult(UUID nodeId, GridNearGetResponse res) {
         for (IgniteInternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
@@ -649,7 +649,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     if (log.isDebugEnabled())
                         log.debug("Got removed entry while processing get response (will not retry).");
                 }
-                catch (IgniteCheckedException e) {
+                catch (Exception e) {
                     // Fail.
                     onDone(e);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6d6ec778/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
index 34d2daa..372c859 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheFutureExceptionSelfTest.java
@@ -20,6 +20,9 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
+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.*;
@@ -32,31 +35,82 @@ import static java.util.concurrent.TimeUnit.*;
  * Cache future self test.
  */
 public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static volatile boolean fail;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = new IgniteConfiguration();
+
         cfg.setGridName(gridName);
 
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(spi);
+
         if (gridName.equals(getTestGridName(1)))
             cfg.setClientMode(true);
 
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testAsyncCacheFuture() throws Exception {
-        Ignite srv = startGrid(0);
+        startGrid(0);
+
+        startGrid(1);
+
+        testGet(false, false);
+
+        testGet(false, true);
+
+        testGet(true, false);
+
+        testGet(true, true);
+    }
+
+    /**
+     * @param nearCache If {@code true} creates near cache on client.
+     * @param cpyOnRead Cache copy on read flag.
+     * @throws Exception If failed.
+     */
+    private void testGet(boolean nearCache, boolean cpyOnRead) throws Exception {
+        fail = false;
+
+        Ignite srv = grid(0);
+
+        Ignite client = grid(1);
+
+        final String cacheName = nearCache ? ("NEAR-CACHE-" + cpyOnRead) : ("CACHE-" + cpyOnRead);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setCopyOnRead(cpyOnRead);
+
+        ccfg.setName(cacheName);
+
+        IgniteCache<Object, Object> cache = srv.createCache(ccfg);
 
-        IgniteCache<String, NotSerializableClass> cache = srv.getOrCreateCache("CACHE");
         cache.put("key", new NotSerializableClass());
 
-        Ignite client = startGrid(1);
+        IgniteCache<Object, Object> clientCache = nearCache ? client.createNearCache(cacheName,
+            new NearCacheConfiguration<>()) : client.cache(cacheName);
 
-        IgniteCache<String, NotSerializableClass> asyncCache = client.<String, NotSerializableClass>cache("CACHE").withAsync();
+        IgniteCache<Object, Object> asyncCache = clientCache.withAsync();
 
-        System.setProperty("FAIL", "true");
+        fail = true;
 
         asyncCache.get("key");
 
@@ -79,7 +133,9 @@ public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        assertTrue(futLatch.await(60, SECONDS));
+        assertTrue(futLatch.await(5, SECONDS));
+
+        srv.destroyCache(cache.getName());
     }
 
     /**
@@ -93,10 +149,10 @@ public class CacheFutureExceptionSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc}*/
         private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-            if (System.getProperty("FAIL") != null)
+            if (fail)
                 throw new RuntimeException("Deserialization failed.");
 
             in.readObject();
         }
     }
-}	
\ No newline at end of file
+}
\ No newline at end of file