You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2016/03/03 00:39:12 UTC

ignite git commit: IGNITE-2748 - Do not make a remote call when java.lang.Object method is called on service proxy

Repository: ignite
Updated Branches:
  refs/heads/master d2d5d246b -> ee01b6153


IGNITE-2748 - Do not make a remote call when java.lang.Object method is called on service proxy


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

Branch: refs/heads/master
Commit: ee01b6153a2d73c54dde0ccbacc1ac879f009b62
Parents: d2d5d24
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Mar 2 15:39:04 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Mar 2 15:39:04 2016 -0800

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           |   2 +-
 .../processors/service/GridServiceProxy.java    | 110 +++++++++++--------
 .../ignite/internal/util/IgniteUtils.java       |  39 +++++++
 .../GridServiceProxyNodeStopSelfTest.java       | 105 ++++++++++++++++++
 .../processors/service/inner/MyService.java     |  11 ++
 .../processors/service/inner/MyServiceImpl.java |  10 ++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   4 +-
 7 files changed, 231 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/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 8db6d27..2d8488b 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
@@ -685,7 +685,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             if (ctx != null) {
                 if (!svcItf.isAssignableFrom(ctx.service().getClass()))
                     throw new IgniteException("Service does not implement specified interface [svcItf=" +
-                        svcItf.getSimpleName() + ", svcCls=" + ctx.service().getClass() + ']');
+                        svcItf.getName() + ", svcCls=" + ctx.service().getClass().getName() + ']');
 
                 return (T)ctx.service();
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
index 6bec8ec..f29d239 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
@@ -147,61 +147,75 @@ class GridServiceProxy<T> implements Serializable {
         /** {@inheritDoc} */
         @SuppressWarnings("BusyWait")
         @Override public Object invoke(Object proxy, final Method mtd, final Object[] args) {
-            while (true) {
-                ClusterNode node = null;
-
-                try {
-                    node = nodeForService(name, sticky);
-
-                    if (node == null)
-                        throw new IgniteException("Failed to find deployed service: " + name);
-
-                    // If service is deployed locally, then execute locally.
-                    if (node.isLocal()) {
-                        ServiceContextImpl svcCtx = ctx.service().serviceContext(name);
-
-                        if (svcCtx != null)
-                            return mtd.invoke(svcCtx.service(), args);
+            if (U.isHashCodeMethod(mtd))
+                return System.identityHashCode(proxy);
+            else if (U.isEqualsMethod(mtd))
+                return proxy == args[0];
+            else if (U.isToStringMethod(mtd))
+                return GridServiceProxy.class.getSimpleName() + " [name=" + name + ", sticky=" + sticky + ']';
+
+            ctx.gateway().readLock();
+
+            try {
+                while (true) {
+                    ClusterNode node = null;
+
+                    try {
+                        node = nodeForService(name, sticky);
+
+                        if (node == null)
+                            throw new IgniteException("Failed to find deployed service: " + name);
+
+                        // If service is deployed locally, then execute locally.
+                        if (node.isLocal()) {
+                            ServiceContextImpl svcCtx = ctx.service().serviceContext(name);
+
+                            if (svcCtx != null)
+                                return mtd.invoke(svcCtx.service(), args);
+                        }
+                        else {
+                            // Execute service remotely.
+                            return ctx.closure().callAsyncNoFailover(
+                                BALANCE,
+                                new ServiceProxyCallable(mtd.getName(), name, mtd.getParameterTypes(), args),
+                                Collections.singleton(node),
+                                false
+                            ).get();
+                        }
                     }
-                    else {
-                        // Execute service remotely.
-                        return ctx.closure().callAsyncNoFailover(
-                            BALANCE,
-                            new ServiceProxyCallable(mtd.getName(), name, mtd.getParameterTypes(), args),
-                            Collections.singleton(node),
-                            false
-                        ).get();
+                    catch (GridServiceNotFoundException | ClusterTopologyCheckedException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Service was not found or topology changed (will retry): " + e.getMessage());
+                    }
+                    catch (RuntimeException | Error e) {
+                        throw e;
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw U.convertException(e);
+                    }
+                    catch (Exception e) {
+                        throw new IgniteException(e);
                     }
-                }
-                catch (GridServiceNotFoundException | ClusterTopologyCheckedException e) {
-                    if (log.isDebugEnabled())
-                        log.debug("Service was not found or topology changed (will retry): " + e.getMessage());
-                }
-                catch (RuntimeException | Error e) {
-                    throw e;
-                }
-                catch (IgniteCheckedException e) {
-                    throw U.convertException(e);
-                }
-                catch (Exception e) {
-                    throw new IgniteException(e);
-                }
 
-                // If we are here, that means that service was not found
-                // or topology was changed. In this case, we erase the
-                // previous sticky node and try again.
-                rmtNode.compareAndSet(node, null);
+                    // If we are here, that means that service was not found
+                    // or topology was changed. In this case, we erase the
+                    // previous sticky node and try again.
+                    rmtNode.compareAndSet(node, null);
 
-                // Add sleep between retries to avoid busy-wait loops.
-                try {
-                    Thread.sleep(10);
-                }
-                catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
+                    // Add sleep between retries to avoid busy-wait loops.
+                    try {
+                        Thread.sleep(10);
+                    }
+                    catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
 
-                    throw new IgniteException(e);
+                        throw new IgniteException(e);
+                    }
                 }
             }
+            finally {
+                ctx.gateway().readUnlock();
+            }
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index db2960d..35722e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -475,6 +475,15 @@ public abstract class IgniteUtils {
     /** */
     private static volatile Boolean hasShmem;
 
+    /** Object.hashCode() */
+    private static Method hashCodeMtd;
+
+    /** Object.equals(...) */
+    private static Method equalsMtd;
+
+    /** Object.toString() */
+    private static Method toStringMtd;
+
     /**
      * Initializes enterprise check.
      */
@@ -695,6 +704,15 @@ public abstract class IgniteUtils {
 
         // Set the http.strictPostRedirect property to prevent redirected POST from being mapped to a GET.
         System.setProperty("http.strictPostRedirect", "true");
+
+        for (Method mtd : Object.class.getMethods()) {
+            if ("hashCode".equals(mtd.getName()))
+                hashCodeMtd = mtd;
+            else if ("equals".equals(mtd.getName()))
+                equalsMtd = mtd;
+            else if ("toString".equals(mtd.getName()))
+                toStringMtd = mtd;
+        }
     }
 
     /**
@@ -9442,4 +9460,25 @@ public abstract class IgniteUtils {
         else
             return GridIoManager.DIRECT_PROTO_VER;
     }
+
+    /**
+     * @return Whether provided method is {@code Object.hashCode()}.
+     */
+    public static boolean isHashCodeMethod(Method mtd) {
+        return hashCodeMtd.equals(mtd);
+    }
+
+    /**
+     * @return Whether provided method is {@code Object.equals(...)}.
+     */
+    public static boolean isEqualsMethod(Method mtd) {
+        return equalsMtd.equals(mtd);
+    }
+
+    /**
+     * @return Whether provided method is {@code Object.toString()}.
+     */
+    public static boolean isToStringMethod(Method mtd) {
+        return toStringMtd.equals(mtd);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyNodeStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyNodeStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyNodeStopSelfTest.java
new file mode 100644
index 0000000..3afe04b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProxyNodeStopSelfTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.Callable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.service.inner.MyService;
+import org.apache.ignite.internal.processors.service.inner.MyServiceFactory;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for service proxy after client node stopped.
+ */
+public class GridServiceProxyNodeStopSelfTest extends GridCommonAbstractTest {
+    /** */
+    private final static TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testProxyHashCode() throws Exception {
+        Ignite server = startGrid("server");
+
+        server.services().deployClusterSingleton("my-service", MyServiceFactory.create());
+
+        Ignition.setClientMode(true);
+
+        Ignite client = startGrid("client");
+
+        final MyService proxy = client.services().serviceProxy("my-service", MyService.class, false);
+
+        assertEquals("GridServiceProxy [name=my-service, sticky=false]", proxy.toString());
+
+        assertEquals(42, proxy.hello());
+        assertEquals(MyService.HASH, proxy.hashCode(null));
+
+        MyService proxy0 = proxy;
+
+        assertTrue(proxy0.equals(proxy));
+
+        proxy0 = client.services().serviceProxy("my-service", MyService.class, false);
+
+        assertFalse(proxy0.equals(proxy));
+
+        int hash = proxy.hashCode();
+
+        assertFalse(hash == MyService.HASH);
+
+        client.close();
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    proxy.hello();
+
+                    return null;
+                }
+            },
+            IllegalStateException.class,
+            null
+        );
+
+        int hash0 = proxy.hashCode();
+
+        assertFalse(hash0 == MyService.HASH);
+
+        assertEquals(hash, hash0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyService.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyService.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyService.java
index b558207..251b438 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyService.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyService.java
@@ -23,8 +23,19 @@ import org.apache.ignite.services.Service;
  * Service.
  */
 public interface MyService extends Service {
+    /** Custom hash code. */
+    public static int HASH = 12345;
+
     /**
      * @return Some value.
      */
     int hello();
+
+    /**
+     * hashCode() method with a dummy argument.
+     *
+     * @param dummy Argument.
+     * @return Hash code.
+     */
+    int hashCode(Object dummy);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyServiceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyServiceImpl.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyServiceImpl.java
index cb2e0cf..fefa697 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyServiceImpl.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/inner/MyServiceImpl.java
@@ -29,6 +29,11 @@ class MyServiceImpl implements MyService {
     }
 
     /** {@inheritDoc} */
+    @Override public int hashCode(Object dummy) {
+        return HASH;
+    }
+
+    /** {@inheritDoc} */
     @Override public void cancel(ServiceContext ctx) {
         // No-op.
     }
@@ -42,4 +47,9 @@ class MyServiceImpl implements MyService {
     @Override public void execute(ServiceContext ctx) throws Exception {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return HASH;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee01b615/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 214d375..6233bab 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
@@ -33,7 +33,6 @@ import org.apache.ignite.internal.GridNodeVisorAttributesSelfTest;
 import org.apache.ignite.internal.GridRuntimeExceptionSelfTest;
 import org.apache.ignite.internal.GridSameVmStartupSelfTest;
 import org.apache.ignite.internal.GridSpiExceptionSelfTest;
-import org.apache.ignite.internal.processors.cluster.GridUpdateNotifierSelfTest;
 import org.apache.ignite.internal.GridVersionSelfTest;
 import org.apache.ignite.internal.IgniteUpdateNotifierPerClusterSettingSelfTest;
 import org.apache.ignite.internal.managers.GridManagerStopSelfTest;
@@ -45,6 +44,7 @@ import org.apache.ignite.internal.managers.discovery.GridDiscoveryManagerSelfTes
 import org.apache.ignite.internal.managers.discovery.IgniteTopologyPrintFormatSelfTest;
 import org.apache.ignite.internal.managers.events.GridEventStorageManagerSelfTest;
 import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManagerSelfTest;
+import org.apache.ignite.internal.processors.cluster.GridUpdateNotifierSelfTest;
 import org.apache.ignite.internal.processors.port.GridPortProcessorSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceClientNodeTest;
 import org.apache.ignite.internal.processors.service.GridServicePackagePrivateSelfTest;
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessorMultiNo
 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.GridServiceProxyNodeStopSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceReassignmentSelfTest;
 import org.apache.ignite.internal.processors.service.GridServiceSerializationSelfTest;
 import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
@@ -125,6 +126,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(ServicePredicateAccessCacheTest.class);
         suite.addTestSuite(GridServicePackagePrivateSelfTest.class);
         suite.addTestSuite(GridServiceSerializationSelfTest.class);
+        suite.addTestSuite(GridServiceProxyNodeStopSelfTest.class);
 
         return suite;
     }