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/07/03 08:38:11 UTC

[20/50] incubator-ignite git commit: IGNITE-1062 Split in topology snapshot server and client nodes.

IGNITE-1062 Split in topology snapshot server and 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/8006a843
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8006a843
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8006a843

Branch: refs/heads/ignite-901
Commit: 8006a84344118f39e9190643420512fbee06bc73
Parents: bade9f1
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 30 16:06:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 30 16:06:28 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  46 +++-
 .../IgniteTopologyPrintFormatSelfTest.java      | 233 +++++++++++++++++++
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 3 files changed, 274 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8006a843/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 5e7600f..1d555e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -98,6 +98,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
     };
 
+    /** Predicate filtering client nodes. */
+    private static final IgnitePredicate<ClusterNode> clientFilter = new P1<ClusterNode>() {
+        @Override public boolean apply(ClusterNode n) {
+            return n.isClient();
+        }
+    };
+
     /** Disco history entries comparator. */
     private static final Comparator<Map.Entry<AffinityTopologyVersion, DiscoCache>> histCmp =
         new Comparator<Map.Entry<AffinityTopologyVersion, DiscoCache>>() {
@@ -933,6 +940,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         Collection<ClusterNode> rmtNodes = discoCache.remoteNodes();
 
+        Collection<ClusterNode> serverNodes = discoCache.serverNodes();
+
+        Collection<ClusterNode> clientNodes = discoCache.clientNodes();
+
         ClusterNode locNode = discoCache.localNode();
 
         Collection<ClusterNode> allNodes = discoCache.allNodes();
@@ -949,7 +960,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         double heap = U.heapSize(allNodes, 2);
 
         if (log.isQuiet())
-            U.quiet(false, topologySnapshotMessage(rmtNodes.size(), totalCpus, heap));
+            U.quiet(false, topologySnapshotMessage(serverNodes.size(), clientNodes.size(), totalCpus, heap));
 
         if (log.isDebugEnabled()) {
             String dbg = "";
@@ -959,7 +970,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 ">>> " + PREFIX + "." + U.nl() +
                 ">>> +----------------+" + U.nl() +
                 ">>> Grid name: " + (ctx.gridName() == null ? "default" : ctx.gridName()) + U.nl() +
-                ">>> Number of nodes: " + (rmtNodes.size() + 1) + U.nl() +
+                ">>> Number of server nodes: " + serverNodes.size() + U.nl() +
+                ">>> Number of client nodes: " + clientNodes.size() + U.nl() +
                 (discoOrdered ? ">>> Topology version: " + topVer + U.nl() : "") +
                 ">>> Topology hash: 0x" + Long.toHexString(hash).toUpperCase() + U.nl();
 
@@ -992,19 +1004,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             log.debug(dbg);
         }
         else if (log.isInfoEnabled())
-            log.info(topologySnapshotMessage(rmtNodes.size(), totalCpus, heap));
+            log.info(topologySnapshotMessage(serverNodes.size(), clientNodes.size(), totalCpus, heap));
     }
 
     /**
-     * @param rmtNodesNum Remote nodes number.
+     * @param serverNodesNum Server nodes number.
+     * @param clientNodesNum Client nodes number.
      * @param totalCpus Total cpu number.
      * @param heap Heap size.
      * @return Topology snapshot message.
      */
-    private String topologySnapshotMessage(int rmtNodesNum, int totalCpus, double heap) {
+    private String topologySnapshotMessage(int serverNodesNum, int clientNodesNum, int totalCpus, double heap) {
         return PREFIX + " [" +
             (discoOrdered ? "ver=" + topSnap.get().topVer.topologyVersion() + ", " : "") +
-            "nodes=" + (rmtNodesNum + 1) +
+            "server nodes=" + serverNodesNum +
+            ", client nodes=" + clientNodesNum +
             ", CPUs=" + totalCpus +
             ", heap=" + heap + "GB" +
             ']';
@@ -2108,6 +2122,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /** Remote nodes. */
         private final List<ClusterNode> rmtNodes;
 
+        /** Client nodes. */
+        private final List<ClusterNode> clientNodes;
+
+        /** Server nodes. */
+        private final List<ClusterNode> serverNodes;
+
         /** All nodes. */
         private final List<ClusterNode> allNodes;
 
@@ -2196,6 +2216,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             all.addAll(rmtNodes);
 
+            clientNodes = Collections.unmodifiableList(new ArrayList<>(F.view(all, clientFilter)));
+
+            serverNodes = Collections.unmodifiableList(new ArrayList<>(F.view(all, F.not(clientFilter))));
+
             Collections.sort(all, GridNodeOrderComparator.INSTANCE);
 
             allNodes = Collections.unmodifiableList(all);
@@ -2346,6 +2370,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             return rmtNodes;
         }
 
+        /** @return Server nodes. */
+        Collection<ClusterNode> serverNodes() {
+            return serverNodes;
+        }
+
+        /** @return Client nodes. */
+        Collection<ClusterNode> clientNodes() {
+            return clientNodes;
+        }
+
         /** @return All nodes. */
         Collection<ClusterNode> allNodes() {
             return allNodes;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8006a843/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
new file mode 100644
index 0000000..efbc431
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteTopologyPrintFormatSelfTest.java
@@ -0,0 +1,233 @@
+/*
+ * 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.managers.discovery;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.*;
+import org.apache.ignite.internal.util.typedef.*;
+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 org.apache.ignite.testframework.junits.logger.*;
+import org.apache.log4j.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteTopologyPrintFormatSelfTest extends GridCommonAbstractTest {
+    /** */
+    public static final String TOPOLOGY_SNAPSHOT = "Topology snapshot";
+
+    /** */
+    public static final String SERV_NODE = ">>> Number of server nodes";
+
+    /** */
+    public static final String CLIENT_NODE = ">>> Number of client nodes";
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disc = new TcpDiscoverySpi();
+        disc.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disc);
+
+        if (gridName.endsWith("client"))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        if (log instanceof MockLogger)
+            ((MockLogger)log).clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.INFO);
+
+        doServerLogTest(log);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerDebugLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.DEBUG);
+
+        doServerLogTest(log);
+    }
+
+    /**
+     * @param log Logger.
+     * @throws Exception If failed.
+     */
+    private void doServerLogTest(MockLogger log) throws Exception {
+        try {
+            Ignite server = startGrid("server");
+
+            setLogger(log, server);
+
+            Ignite server1 = startGrid("server1");
+
+            waitForDiscovery(server, server1);
+        }
+        finally {
+            stopAllGrids();
+        }
+
+        assertTrue(F.forAny(log.logs(), new IgnitePredicate<String>() {
+            @Override public boolean apply(String s) {
+                return s.contains("Topology snapshot [ver=2, server nodes=2, client nodes=0,")
+                    || (s.contains(">>> Number of server nodes: 2") && s.contains(">>> Number of client nodes: 0"));
+            }
+        }));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerAndClientLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.INFO);
+
+        doServerAndClientTest(log);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerAndClientDebugLogs() throws Exception {
+        MockLogger log = new MockLogger();
+
+        log.setLevel(Level.DEBUG);
+
+        doServerAndClientTest(log);
+    }
+
+    /**
+     * @param log Log.
+     * @throws Exception If failed.
+     */
+    private void doServerAndClientTest(MockLogger log) throws Exception {
+        try {
+            Ignite server = startGrid("server");
+
+            setLogger(log, server);
+
+            Ignite server1 = startGrid("server1");
+            Ignite client1 = startGrid("first client");
+            Ignite client2 = startGrid("second client");
+
+            waitForDiscovery(server, server1, client1, client2);
+        }
+        finally {
+            stopAllGrids();
+        }
+
+        assertTrue(F.forAny(log.logs(), new IgnitePredicate<String>() {
+            @Override public boolean apply(String s) {
+                return s.contains("Topology snapshot [ver=4, server nodes=2, client nodes=2,")
+                    || (s.contains(">>> Number of server nodes: 2") && s.contains(">>> Number of client nodes: 2"));
+            }
+        }));
+    }
+
+    /**
+     * Set log.
+     *
+     * @param log Log.
+     * @param server Ignite.
+     */
+    private void setLogger(MockLogger log, Ignite server) {
+        IgniteKernal server0 = (IgniteKernal)server;
+
+        GridDiscoveryManager discovery = server0.context().discovery();
+
+        GridTestUtils.setFieldValue(discovery, GridManagerAdapter.class, "log", log);
+    }
+
+    /**
+     *
+     */
+    private static class MockLogger extends GridTestLog4jLogger {
+        /** */
+        private List<String> logs = new ArrayList<>();
+
+        /**  {@inheritDoc} */
+        @Override public void debug(String msg) {
+            if ((msg != null && !msg.isEmpty()) && (
+                msg.contains(TOPOLOGY_SNAPSHOT)
+                    || msg.contains(SERV_NODE)
+                    || msg.contains(CLIENT_NODE)))
+                logs.add(msg);
+
+            super.debug(msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void info(String msg) {
+            if ((msg != null && !msg.isEmpty()) && (
+                msg.contains(TOPOLOGY_SNAPSHOT)
+                || msg.contains(SERV_NODE)
+                || msg.contains(CLIENT_NODE)))
+                logs.add(msg);
+
+            super.info(msg);
+        }
+
+        /**
+         * @return Logs.
+         */
+        public List<String> logs() {
+            return logs;
+        }
+
+        /** */
+        public void clear() {
+            logs.clear();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8006a843/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 e8c89ec..575f1fa 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
@@ -83,6 +83,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridKernalConcurrentAccessStopSelfTest.class);
         suite.addTestSuite(GridUpdateNotifierSelfTest.class);
         suite.addTestSuite(GridLocalEventListenerSelfTest.class);
+        suite.addTestSuite(IgniteTopologyPrintFormatSelfTest.class);
 
         // Managed Services.
         suite.addTestSuite(GridServiceProcessorSingleNodeSelfTest.class);