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/19 11:37:49 UTC

[45/50] incubator-ignite git commit: # ignite-sprint-7 do not run TaskDiscoveryListener in discovery thread

# ignite-sprint-7 do not run TaskDiscoveryListener in discovery thread


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

Branch: refs/heads/ignite-gg-10411
Commit: 044f17deae5a16c0d463633b75640d04a125f5d2
Parents: c6a0f24
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 15:36:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 15:36:44 2015 +0300

----------------------------------------------------------------------
 .../processors/task/GridTaskProcessor.java      |  22 ++-
 .../GridTaskFailoverAffinityRunTest.java        | 170 +++++++++++++++++++
 .../testsuites/IgniteComputeGridTestSuite.java  |   1 +
 3 files changed, 184 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/044f17de/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
index 88713a1..bb9ff50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
@@ -1171,17 +1171,21 @@ public class GridTaskProcessor extends GridProcessorAdapter {
         @Override public void onEvent(Event evt) {
             assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
 
-            UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+            final UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
-            lock.readLock();
+            ctx.closure().runLocalSafe(new Runnable() {
+                @Override public void run() {
+                    lock.readLock();
 
-            try {
-                for (GridTaskWorker<?, ?> task : tasks.values())
-                    task.onNodeLeft(nodeId);
-            }
-            finally {
-                lock.readUnlock();
-            }
+                    try {
+                        for (GridTaskWorker<?, ?> task : tasks.values())
+                            task.onNodeLeft(nodeId);
+                    }
+                    finally {
+                        lock.readUnlock();
+                    }
+                }
+            }, false);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/044f17de/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java
new file mode 100644
index 0000000..7ddd966
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+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.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+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.CacheRebalanceMode.*;
+
+/**
+ *
+ */
+public class GridTaskFailoverAffinityRunTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean clientMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        boolean client = clientMode && gridName.equals(getTestGridName(0));
+
+        if (client) {
+            cfg.setClientMode(true);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        }
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeRestart() throws Exception {
+        clientMode = false;
+
+        nodeRestart();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeRestartClient() throws Exception {
+        clientMode = true;
+
+        nodeRestart();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void nodeRestart() throws Exception {
+        startGridsMultiThreaded(4);
+
+        assertEquals((Boolean)clientMode, grid(0).configuration().isClientMode());
+
+        IgniteCompute comp = grid(0).compute().withAsync();
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final AtomicInteger gridIdx = new AtomicInteger(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int grid = gridIdx.getAndIncrement();
+
+                while (!stop.get()) {
+                    stopGrid(grid);
+
+                    startGrid(grid);
+                }
+
+                return null;
+            }
+        }, 2, "restart-thread");
+
+        try {
+            long stopTime = System.currentTimeMillis() + 60_000;
+
+            while (System.currentTimeMillis() < stopTime) {
+                Collection<IgniteFuture<?>> futs = new ArrayList<>(1000);
+
+                for (int i = 0; i < 1000; i++) {
+                    comp.affinityCall(null, i, new TestJob());
+
+                    IgniteFuture<?> fut0 = comp.future();
+
+                    assertNotNull(fut0);
+
+                    futs.add(fut0);
+                }
+
+                for (IgniteFuture<?> fut0 : futs) {
+                    try {
+                        fut0.get();
+                    }
+                    catch (IgniteException ignore) {
+                        // No-op.
+                    }
+                }
+            }
+        }
+        finally {
+            stop.set(true);
+
+            fut.get();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestJob implements IgniteCallable<Object> {
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            Thread.sleep(1000);
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/044f17de/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 82fc5e0..baf425c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -93,6 +93,7 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(GridMultinodeRedeployIsolatedModeSelfTest.class);
         suite.addTestSuite(IgniteComputeEmptyClusterGroupTest.class);
         suite.addTestSuite(IgniteComputeTopologyExceptionTest.class);
+        suite.addTestSuite(GridTaskFailoverAffinityRunTest.class);
 
         return suite;
     }