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 2017/11/22 10:23:18 UTC

[02/15] ignite git commit: IGNITE-6437: DataStructure can not be obtained on client if it is created on server node

IGNITE-6437: DataStructure can not be obtained on client if it is created on server node


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

Branch: refs/heads/ignite-zk
Commit: ed4616e2d39b21c3ba5899a8eb112496dcd28ef5
Parents: 6508067
Author: Evgeny Stanilovskiy <es...@gridgain.com>
Authored: Tue Nov 21 13:12:21 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Nov 21 13:12:21 2017 +0300

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java |  18 ++-
 .../IgniteClientReconnectCollectionsTest.java   | 131 ++++++++-----------
 2 files changed, 69 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ed4616e2/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 33a2fd2..b26acdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.CacheType;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheInternal;
@@ -952,6 +953,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
 
         assert name != null;
         assert type.isCollection() : type;
+        assert !create || cfg != null;
 
         if (grpName == null) {
             if (cfg != null && cfg.getGroupName() != null)
@@ -960,17 +962,23 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
                 grpName = DEFAULT_DS_GROUP_NAME;
         }
 
-        assert !create || cfg != null;
-
         final String metaCacheName = ATOMICS_CACHE_NAME + "@" + grpName;
 
         IgniteInternalCache<GridCacheInternalKey, AtomicDataStructureValue> metaCache0 = ctx.cache().cache(metaCacheName);
 
         if (metaCache0 == null) {
-            if (!create)
-                return null;
+            CacheConfiguration ccfg = null;
+
+            if (!create) {
+                DynamicCacheDescriptor desc = ctx.cache().cacheDescriptor(metaCacheName);
+
+                if (desc == null)
+                    return null;
+            }
+            else
+                ccfg = metaCacheConfiguration(cfg, metaCacheName, grpName);
 
-            ctx.cache().dynamicStartCache(metaCacheConfiguration(cfg, metaCacheName, grpName),
+            ctx.cache().dynamicStartCache(ccfg,
                 metaCacheName,
                 null,
                 CacheType.DATA_STRUCTURES,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed4616e2/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
index 8ee669c..3f0e33d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
@@ -38,6 +38,20 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  *
  */
 public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectAbstractTest {
+    /** */
+    private static final CollectionConfiguration TX_CFGS = new CollectionConfiguration();
+
+    /** */
+    private static final CollectionConfiguration ATOMIC_CONF = new CollectionConfiguration();
+
+    static {
+        TX_CFGS.setCacheMode(PARTITIONED);
+        TX_CFGS.setAtomicityMode(TRANSACTIONAL);
+
+        ATOMIC_CONF.setCacheMode(PARTITIONED);
+        ATOMIC_CONF.setAtomicityMode(ATOMIC);
+    }
+
     /** {@inheritDoc} */
     @Override protected int serverCount() {
         return 1;
@@ -52,17 +66,12 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
      * @throws Exception If failed.
      */
     public void testCollectionsReconnectClusterRestart() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
-
         Ignite client = grid(serverCount());
 
         assertTrue(client.cluster().localNode().isClient());
 
-        final IgniteQueue<Object> queue = client.queue("q", 0, colCfg);
-        final IgniteSet<Object> set = client.set("s", colCfg);
+        final IgniteQueue<Object> queue = client.queue("q", 0, TX_CFGS);
+        final IgniteSet<Object> set = client.set("s", TX_CFGS);
 
         Ignite srv = grid(0);
 
@@ -88,12 +97,16 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
             }
         }, IllegalStateException.class, null);
 
-        try (IgniteQueue<Object> queue2 = client.queue("q", 0, colCfg)) {
+        try (IgniteQueue<Object> queue2 = client.queue("q", 0, TX_CFGS)) {
             queue2.add(1);
+
+            assert queue2.size() == 1 : queue2.size();
         }
 
-        try (IgniteSet<Object> set2 = client.set("s", colCfg)) {
+        try (IgniteSet<Object> set2 = client.set("s", TX_CFGS)) {
             set2.add(1);
+
+            assert set2.size() == 1 : set2.size();
         }
     }
 
@@ -101,114 +114,82 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
      * @throws Exception If failed.
      */
     public void testQueueReconnect() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
+        queueReconnect(TX_CFGS);
 
-        queueReconnect(colCfg);
-
-        colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(ATOMIC);
-
-        queueReconnect(colCfg);
+        queueReconnect(ATOMIC_CONF);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testQueueReconnectRemoved() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
-
-        queueReconnectRemoved(colCfg);
+        queueReconnectRemoved(TX_CFGS);
 
-        colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(ATOMIC);
-
-        queueReconnectRemoved(colCfg);
+        queueReconnectRemoved(ATOMIC_CONF);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testQueueReconnectInProgress() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
-
-        queueReconnectInProgress(colCfg);
+        queueReconnectInProgress(TX_CFGS);
 
-        colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(ATOMIC);
-
-        queueReconnectInProgress(colCfg);
+        queueReconnectInProgress(ATOMIC_CONF);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testSetReconnect() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
-
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
-
-        setReconnect(colCfg);
-
-        colCfg = new CollectionConfiguration();
+        setReconnect(TX_CFGS);
 
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(ATOMIC);
-
-        setReconnect(colCfg);
+        setReconnect(ATOMIC_CONF);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testSetReconnectRemoved() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
+        setReconnectRemove(TX_CFGS);
 
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(ATOMIC);
+        setReconnectRemove(ATOMIC_CONF);
+    }
 
-        setReconnectRemove(colCfg);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetReconnectInProgress() throws Exception {
+        setReconnectInProgress(TX_CFGS);
 
-        colCfg = new CollectionConfiguration();
+        setReconnectInProgress(ATOMIC_CONF);
+    }
 
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerReconnect() throws Exception {
+        serverNodeReconnect(TX_CFGS);
 
-        setReconnectRemove(colCfg);
+        serverNodeReconnect(ATOMIC_CONF);
     }
 
     /**
+     * @param colCfg Collection configuration.
      * @throws Exception If failed.
      */
-    public void testSetReconnectInProgress() throws Exception {
-        CollectionConfiguration colCfg = new CollectionConfiguration();
+    private void serverNodeReconnect(CollectionConfiguration colCfg) throws Exception {
+        final Ignite client = grid(serverCount());
 
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(ATOMIC);
+        final Ignite srv = clientRouter(client);
 
-        setReconnectInProgress(colCfg);
+        assertNotNull(srv.queue("q", 0, colCfg));
+        assertNotNull(srv.set("s", colCfg));
 
-        colCfg = new CollectionConfiguration();
+        reconnectClientNode(client, srv, null);
 
-        colCfg.setCacheMode(PARTITIONED);
-        colCfg.setAtomicityMode(TRANSACTIONAL);
+        IgniteQueue<Object> q = client.queue("q", 0, null);
 
-        setReconnectInProgress(colCfg);
+        assertNotNull(q);
     }
 
     /**
@@ -495,4 +476,4 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
 
         assertEquals("1", clientQueue.poll());
     }
-}
\ No newline at end of file
+}