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/05/14 10:51:28 UTC

incubator-ignite git commit: # ignite-709 do not create atomic cache on client nodes

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-709_2 cfcd1d92d -> 4ba5671be


# ignite-709 do not create atomic cache on client nodes


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

Branch: refs/heads/ignite-709_2
Commit: 4ba5671be90d2cdf54d543a60f7dda0b310bf46b
Parents: cfcd1d9
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 14 11:50:33 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 14 11:50:33 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../datastructures/DataStructuresProcessor.java |   4 +-
 .../IgniteClientDataStructuresAbstractTest.java | 211 +++++++++++++++++++
 .../IgniteClientDataStructuresTest.java         |  28 +++
 ...IgniteClientDiscoveryDataStructuresTest.java |  28 +++
 .../IgniteCacheDataStructuresSelfTestSuite.java |   3 +
 6 files changed, 273 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ba5671b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 8d88677..0807d09 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2062,7 +2062,7 @@ public class IgnitionEx {
             ccfg.setRebalanceMode(SYNC);
             ccfg.setWriteSynchronizationMode(FULL_SYNC);
             ccfg.setCacheMode(cfg.getCacheMode());
-            ccfg.setNodeFilter(CacheConfiguration.ALL_NODES);
+            ccfg.setNodeFilter(CacheConfiguration.SERVER_NODES);
             ccfg.setNearConfiguration(new NearCacheConfiguration());
 
             if (cfg.getCacheMode() == PARTITIONED)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ba5671b/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 72911af..b8d5bef 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
@@ -112,7 +112,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() {
+    @Override public void onKernalStart() throws IgniteCheckedException {
         if (ctx.config().isDaemon())
             return;
 
@@ -123,7 +123,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
         assert utilityCache != null;
 
         if (atomicCfg != null) {
-            IgniteInternalCache atomicsCache = ctx.cache().atomicsCache();
+            IgniteInternalCache atomicsCache = ctx.cache().getOrStartCache(CU.ATOMICS_CACHE_NAME);
 
             assert atomicsCache != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ba5671b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
new file mode 100644
index 0000000..175c1e6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -0,0 +1,211 @@
+/*
+ * 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.datastructures;
+
+import org.apache.ignite.*;
+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 java.util.*;
+
+/**
+ *
+ */
+public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODE_CNT = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(NODE_CNT - 1))) {
+            cfg.setClientMode(true);
+
+            if (clientDiscovery()) {
+                TcpDiscoveryVmIpFinder clientFinder = new TcpDiscoveryVmIpFinder();
+
+                String firstSrvAddr = F.first(ipFinder.getRegisteredAddresses()).toString();
+
+                if (firstSrvAddr.startsWith("/"))
+                    firstSrvAddr = firstSrvAddr.substring(1);
+
+                clientFinder.setAddresses(Collections.singletonList(firstSrvAddr));
+
+                TcpClientDiscoverySpi discoverySpi = new TcpClientDiscoverySpi();
+                discoverySpi.setIpFinder(clientFinder);
+
+                cfg.setDiscoverySpi(discoverySpi);
+            }
+        }
+
+        cfg.setLocalHost("127.0.0.1");
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODE_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return {@code True} if use client discovery.
+     */
+    protected abstract boolean clientDiscovery();
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSequence() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        Ignite srvNode = serverNode();
+
+        assertNull(clientNode.atomicSequence("seq1", 1L, false));
+
+        try (IgniteAtomicSequence seq = clientNode.atomicSequence("seq1", 1L, true)) {
+            assertNotNull(seq);
+
+            assertEquals(1L, seq.get());
+
+            assertEquals(1L, seq.getAndAdd(1));
+
+            assertEquals(2L, seq.get());
+
+            IgniteAtomicSequence seq0 = srvNode.atomicSequence("seq1", 1L, false);
+
+            assertNotNull(seq0);
+        }
+
+        assertNull(clientNode.atomicSequence("seq1", 1L, false));
+        assertNull(srvNode.atomicSequence("seq1", 1L, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicLong() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        Ignite srvNode = serverNode();
+
+        assertNull(clientNode.atomicLong("long1", 1L, false));
+
+        try (IgniteAtomicLong cntr = clientNode.atomicLong("long1", 1L, true)) {
+            assertNotNull(cntr);
+
+            assertEquals(1L, cntr.get());
+
+            assertEquals(1L, cntr.getAndAdd(1));
+
+            assertEquals(2L, cntr.get());
+
+            IgniteAtomicLong cntr0 = srvNode.atomicLong("long1", 1L, false);
+
+            assertNotNull(cntr0);
+
+            assertEquals(2L, cntr0.get());
+
+            assertEquals(3L, cntr0.incrementAndGet());
+
+            assertEquals(3L, cntr.get());
+        }
+
+        assertNull(clientNode.atomicLong("long1", 1L, false));
+        assertNull(srvNode.atomicLong("long1", 1L, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSet() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        Ignite srvNode = serverNode();
+
+        assertNull(clientNode.set("set1", null));
+
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        try (IgniteSet<Integer> set = clientNode.set("set1", colCfg)) {
+            assertNotNull(set);
+
+            assertEquals(0, set.size());
+
+            assertFalse(set.contains(1));
+
+            assertTrue(set.add(1));
+
+            assertTrue(set.contains(1));
+
+            IgniteSet<Integer> set0 = srvNode.set("set1", null);
+
+            assertTrue(set0.contains(1));
+
+            assertEquals(1, set0.size());
+
+            assertTrue(set0.remove(1));
+
+            assertFalse(set.contains(1));
+        }
+    }
+
+    /**
+     * @return Client node.
+     */
+    private Ignite clientIgnite() {
+        Ignite ignite = ignite(NODE_CNT - 1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        if (clientDiscovery())
+            assertTrue(ignite.configuration().getDiscoverySpi() instanceof TcpClientDiscoverySpi);
+        else
+            assertTrue(ignite.configuration().getDiscoverySpi() instanceof TcpDiscoverySpi);
+
+        return ignite;
+    }
+
+    /**
+     * @return Server node.
+     */
+    private Ignite serverNode() {
+        Ignite ignite = ignite(0);
+
+        assertFalse(ignite.configuration().isClientMode());
+
+        return ignite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ba5671b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java
new file mode 100644
index 0000000..a228cc2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.datastructures;
+
+/**
+ *
+ */
+public class IgniteClientDataStructuresTest extends IgniteClientDataStructuresAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean clientDiscovery() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ba5671b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java
new file mode 100644
index 0000000..bd5cce8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.datastructures;
+
+/**
+ *
+ */
+public class IgniteClientDiscoveryDataStructuresTest extends IgniteClientDataStructuresAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean clientDiscovery() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ba5671b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index 4c40e75..7fb4563 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -112,6 +112,9 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteDataStructureUniqueNameTest.class));
 
+        suite.addTest(new TestSuite(IgniteClientDataStructuresTest.class));
+        suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
+
         return suite;
     }
 }