You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/06/16 10:21:18 UTC

[08/50] incubator-ignite git commit: # ignite-sprint-5 increased affinity history size

# ignite-sprint-5 increased affinity history size


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

Branch: refs/heads/ignite-gg-10411
Commit: 2454eb58ae60718f8fcf55eccb7a4fc7016e0bcf
Parents: addc91b
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 10 17:43:02 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 10 17:43:02 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../affinity/GridAffinityAssignmentCache.java   |   5 +-
 .../GridCachePartitionExchangeManager.java      |   2 +-
 .../distributed/IgniteCacheManyClientsTest.java | 169 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 5 files changed, 178 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 439ea2d..b166f39 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -337,6 +337,9 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_SQL_MERGE_TABLE_MAX_SIZE = "IGNITE_SQL_MERGE_TABLE_MAX_SIZE";
 
+    /** Maximum size for affinity assignment history. */
+    public static final String IGNITE_AFFINITY_HISTORY_SIZE = "IGNITE_AFFINITY_HISTORY_SIZE";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 47f222e..6989385 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -408,9 +408,10 @@ public class GridAffinityAssignmentCache {
                 throw new IllegalStateException("Getting affinity for topology version earlier than affinity is " +
                     "calculated [locNodeId=" + ctx.localNodeId() +
                     ", cache=" + cacheName +
-                    ", history=" + affCache.keySet() +
                     ", topVer=" + topVer +
-                    ", head=" + head.get().topologyVersion() + ']');
+                    ", head=" + head.get().topologyVersion() +
+                    ", history=" + affCache.keySet() +
+                    ']');
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 3236bb5..3df45cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -59,7 +59,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     private static final int EXCHANGE_HISTORY_SIZE = 1000;
 
     /** Cleanup history size. */
-    public static final int EXCH_FUT_CLEANUP_HISTORY_SIZE = 10;
+    public static final int EXCH_FUT_CLEANUP_HISTORY_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 100);
 
     /** Atomic reference for pending timeout object. */
     private AtomicReference<ResendTimeoutObject> pendingResend = new AtomicReference<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
new file mode 100644
index 0000000..24ebb7c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheManyClientsTest.java
@@ -0,0 +1,169 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+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.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int SRVS = 4;
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean clientDiscovery;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (!clientDiscovery)
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(SRVS);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testManyClients() throws Exception {
+        manyClientsPutGet();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testManyClientsClientDiscovery() throws Exception {
+        clientDiscovery = true;
+
+        manyClientsPutGet();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void manyClientsPutGet() throws Exception {
+        client = true;
+
+        final AtomicInteger idx = new AtomicInteger(SRVS);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final int THREADS = 30;
+
+        final CountDownLatch latch = new CountDownLatch(THREADS);
+
+        try {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    try (Ignite ignite = startGrid(idx.getAndIncrement())) {
+                        log.info("Started node: " + ignite.name());
+
+                        assertTrue(ignite.configuration().isClientMode());
+
+                        IgniteCache<Object, Object> cache = ignite.cache(null);
+
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        int iter = 0;
+
+                        Integer key = rnd.nextInt(0, 1000);
+
+                        cache.put(key, iter++);
+
+                        assertNotNull(cache.get(key));
+
+                        latch.countDown();
+
+                        while (!stop.get()) {
+                            key = rnd.nextInt(0, 1000);
+
+                            cache.put(key, iter++);
+
+                            assertNotNull(cache.get(key));
+                        }
+
+                        log.info("Stopping node: " + ignite.name());
+                    }
+
+                    return null;
+                }
+            }, THREADS, "client-thread");
+
+            latch.await();
+
+            Thread.sleep(10_000);
+
+            log.info("Stop clients.");
+
+            stop.set(true);
+
+            fut.get();
+        }
+        finally {
+            stop.set(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2454eb58/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 713c5e5..ed9fc9a 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
@@ -138,6 +138,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheReadOnlyTransactionalClientSelfTest.class);
 
+        suite.addTestSuite(IgniteCacheManyClientsTest.class);
+
         return suite;
     }
 }