You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dm...@apache.org on 2015/11/04 15:28:02 UTC

[22/37] ignite git commit: ignite-1820 Deployment service hangs when node is being stopped

ignite-1820 Deployment service hangs when node is being stopped

Signed-off-by: agura <ag...@gridgain.com>


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

Branch: refs/heads/ignite-1790
Commit: 195cd57fb6362220d2ea664046bcdd92d024d871
Parents: fff4e45
Author: agura <ag...@gridgain.com>
Authored: Mon Nov 2 13:26:27 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Nov 2 13:26:27 2015 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           |  31 +++---
 .../GridServiceProcessorStopSelfTest.java       | 103 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 3 files changed, 124 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/195cd57f/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 7e8669a..ed54f00 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -242,28 +242,35 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
         U.shutdownNow(GridServiceProcessor.class, depExe, log);
 
+        Exception err = new IgniteCheckedException("Operation has been cancelled (node is stopping).");
+
+        cancelFutures(depFuts, err);
+        cancelFutures(undepFuts, err);
+
         if (log.isDebugEnabled())
             log.debug("Stopped service processor.");
     }
 
     /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
-        for (Map.Entry<String, GridServiceDeploymentFuture> e : depFuts.entrySet()) {
-            GridServiceDeploymentFuture fut = e.getValue();
-
-            fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
-                "Failed to deploy service, client node disconnected."));
+        cancelFutures(depFuts, new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+            "Failed to deploy service, client node disconnected."));
 
-            depFuts.remove(e.getKey(), fut);
-        }
+        cancelFutures(undepFuts, new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+            "Failed to undeploy service, client node disconnected."));
+    }
 
-        for (Map.Entry<String, GridFutureAdapter<?>> e : undepFuts.entrySet()) {
-            GridFutureAdapter fut = e.getValue();
+    /**
+     * @param futs Futs.
+     * @param err Exception.
+     */
+    private void cancelFutures(ConcurrentMap<String, ? extends GridFutureAdapter<?>> futs, Exception err) {
+        for (Map.Entry<String, ? extends GridFutureAdapter<?>> entry : futs.entrySet()) {
+            GridFutureAdapter fut = entry.getValue();
 
-            fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
-                "Failed to undeploy service, client node disconnected."));
+            fut.onDone(err);
 
-            undepFuts.remove(e.getKey(), fut);
+            futs.remove(entry.getKey(), fut);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/195cd57f/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java
new file mode 100644
index 0000000..16ea5e4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorStopSelfTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.service;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests that {@link GridServiceProcessor} completes deploy/undeploy futures during node stop.
+ */
+public class GridServiceProcessorStopSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStopDuringDeployment() throws Exception {
+        final CountDownLatch depLatch = new CountDownLatch(1);
+
+        final CountDownLatch finishLatch = new CountDownLatch(1);
+
+        final Ignite ignite = startGrid(0);
+
+        Thread t = new Thread(new Runnable() {
+            @Override public void run() {
+                IgniteServices svcs = ignite.services();
+
+                IgniteServices services = svcs.withAsync();
+
+                services.deployClusterSingleton("myClusterSingletonService", new TestServiceImpl());
+
+                depLatch.countDown();
+
+                try {
+                    services.future().get();
+                }
+                catch (IgniteException e) {
+                    finishLatch.countDown();
+                }
+                catch (Throwable e) {
+                    log.error("Service deployment error: ", e);
+                }
+            }
+        });
+
+        t.start();
+
+        depLatch.await();
+
+        Ignition.stopAll(true);
+
+        assertTrue("Deploy future isn't completed", finishLatch.await(15, TimeUnit.SECONDS));
+    }
+
+    /**
+     * Simple map service.
+     */
+    public interface TestService {
+    }
+
+    /**
+     *
+     */
+    public class TestServiceImpl implements Service, TestService {
+        /** Serial version UID. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public void cancel(ServiceContext ctx) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void init(ServiceContext ctx) throws Exception {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute(ServiceContext ctx) throws Exception {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/195cd57f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index 2a24c66..a41859e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNo
 import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceProcessorProxySelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceProcessorSingleNodeSelfTest;
+import org.apache.ignite.internal.processors.service.GridServiceProcessorStopSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceReassignmentSelfTest;
 import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest;
 import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
@@ -115,6 +116,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridServiceProcessorProxySelfTest.class);
         suite.addTestSuite(GridServiceReassignmentSelfTest.class);
         suite.addTestSuite(GridServiceClientNodeTest.class);
+        suite.addTestSuite(GridServiceProcessorStopSelfTest.class);
 
         return suite;
     }