You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2018/04/23 10:34:32 UTC

[8/9] ignite git commit: IGNITE-8192 Print out information on how many nodes left until auto-activation

IGNITE-8192 Print out information on how many nodes left until auto-activation

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-6083
Commit: 493cc14844b5ea1d4429f4c0a06f39ff739de2ca
Parents: 244bb75
Author: Aleksey Plekhanov <pl...@gmail.com>
Authored: Mon Apr 23 13:27:51 2018 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Mon Apr 23 13:32:45 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  5 +-
 .../discovery/GridDiscoveryManager.java         | 65 ++++++++++++++++----
 .../ignite/internal/util/lang/GridFunc.java     | 14 +++--
 3 files changed, 65 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/493cc148/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 0e0e655..920f519 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -99,12 +99,12 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.binary.BinaryEnumCache;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.binary.BinaryUtils;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
 import org.apache.ignite.internal.managers.GridManager;
 import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager;
@@ -1301,7 +1301,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ackStart(rtBean);
 
         if (!isDaemon())
-            ctx.discovery().ackTopology(ctx.discovery().localJoin().joinTopologyVersion().topologyVersion());
+            ctx.discovery().ackTopology(ctx.discovery().localJoin().joinTopologyVersion().topologyVersion(),
+                EventType.EVT_NODE_JOINED, localNode());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/493cc148/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 77c9657..6af3293 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
@@ -1372,18 +1372,22 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * Prints the latest topology info into log taking into account logging/verbosity settings.
      *
      * @param topVer Topology version.
+     * @param evtType Event type.
+     * @param evtNode Event node.
      */
-    public void ackTopology(long topVer) {
-        ackTopology(topVer, false);
+    public void ackTopology(long topVer, int evtType, ClusterNode evtNode) {
+        ackTopology(topVer, evtType, evtNode, false);
     }
 
     /**
      * Logs grid size for license compliance.
      *
      * @param topVer Topology version.
+     * @param evtType Event type.
+     * @param evtNode Event node.
      * @param throttle Suppress printing if this topology was already printed.
      */
-    private void ackTopology(long topVer, boolean throttle) {
+    private void ackTopology(long topVer, int evtType, ClusterNode evtNode, boolean throttle) {
         assert !isLocDaemon;
 
         DiscoCache discoCache = discoCacheHist.get(new AffinityTopologyVersion(topVer));
@@ -1430,7 +1434,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     return null;
                 }
-            }, topVer, srvNodes.size(), clientNodes.size(), totalCpus, heap, offheap);
+            }, topVer, discoCache, evtType, evtNode, srvNodes.size(), clientNodes.size(), totalCpus, heap, offheap);
 
         if (log.isDebugEnabled()) {
             String dbg = "";
@@ -1481,7 +1485,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     return null;
                 }
-            }, topVer, srvNodes.size(), clientNodes.size(), totalCpus, heap, offheap);
+            }, topVer, discoCache, evtType, evtNode, srvNodes.size(), clientNodes.size(), totalCpus, heap, offheap);
     }
 
     /**
@@ -1557,14 +1561,18 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * @param clo Wrapper of logger.
      * @param topVer Topology version.
+     * @param discoCache Discovery cache.
+     * @param evtType Event type.
+     * @param evtNode Event node.
      * @param srvNodesNum Server nodes number.
      * @param clientNodesNum Client nodes number.
      * @param totalCpus Total cpu number.
      * @param heap Heap size.
      * @param offheap Offheap size.
      */
-    private void topologySnapshotMessage(IgniteClosure<String, Void> clo, long topVer, int srvNodesNum,
-        int clientNodesNum, int totalCpus, double heap, double offheap) {
+    private void topologySnapshotMessage(IgniteClosure<String, Void> clo, long topVer, DiscoCache discoCache,
+        int evtType, ClusterNode evtNode, int srvNodesNum, int clientNodesNum, int totalCpus, double heap,
+        double offheap) {
         String summary = PREFIX + " [" +
             (discoOrdered ? "ver=" + topVer + ", " : "") +
             "servers=" + srvNodesNum +
@@ -1575,6 +1583,41 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         clo.apply(summary);
 
+        DiscoveryDataClusterState state = discoCache.state();
+
+        clo.apply("  ^-- Node [id=" + discoCache.localNode().id().toString().toUpperCase() + ", clusterState="
+            + (state.active() ? "ACTIVE" : "INACTIVE") + ']');
+
+        BaselineTopology blt = state.baselineTopology();
+
+        if (blt != null && discoCache.baselineNodes() != null) {
+            int bltSize = discoCache.baselineNodes().size();
+            int bltOnline = discoCache.aliveBaselineNodes().size();
+            int bltOffline = bltSize - bltOnline;
+
+            clo.apply("  ^-- Baseline [id=" + blt.id() + ", size=" + bltSize + ", online=" + bltOnline
+                + ", offline=" + bltOffline + ']');
+
+            if (!state.active() && ctx.config().isAutoActivationEnabled()) {
+                String offlineConsistentIds = "";
+
+                if (bltOffline > 0 && bltOffline <= 5) {
+                    Collection<BaselineNode> offlineNodes = new HashSet<>(discoCache.baselineNodes());
+
+                    offlineNodes.removeAll(discoCache.aliveBaselineNodes());
+
+                    offlineConsistentIds = ' ' + F.nodeConsistentIds(offlineNodes).toString();
+                }
+
+                if (bltOffline == 0) {
+                    if (evtType == EVT_NODE_JOINED && discoCache.baselineNode(evtNode))
+                        clo.apply("  ^-- All baseline nodes are online, will start auto-activation");
+                }
+                else
+                    clo.apply("  ^-- " + bltOffline + " nodes left for auto-activation" + offlineConsistentIds);
+            }
+        }
+
         DataStorageConfiguration memCfg = ctx.config().getDataStorageConfiguration();
 
         if (memCfg == null)
@@ -2742,7 +2785,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                             if (log.isInfoEnabled())
                                 log.info("Added new node to topology: " + node);
 
-                            ackTopology(topVer.topologyVersion(), true);
+                            ackTopology(topVer.topologyVersion(), type, node, true);
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Added new node to topology: " + node);
@@ -2763,7 +2806,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                             if (log.isInfoEnabled())
                                 log.info("Node left topology: " + node);
 
-                            ackTopology(topVer.topologyVersion(), true);
+                            ackTopology(topVer.topologyVersion(), type, node, true);
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Node left topology: " + node);
@@ -2785,7 +2828,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         log.info("Client node reconnected to topology: " + node);
 
                     if (!isLocDaemon)
-                        ackTopology(topVer.topologyVersion(), true);
+                        ackTopology(topVer.topologyVersion(), type, node, true);
 
                     break;
                 }
@@ -2799,7 +2842,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         if (!isLocDaemon) {
                             U.warn(log, "Node FAILED: " + node);
 
-                            ackTopology(topVer.topologyVersion(), true);
+                            ackTopology(topVer.topologyVersion(), type, node, true);
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Node FAILED: " + node);

http://git-wip-us.apache.org/repos/asf/ignite/blob/493cc148/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index ef32e31..f9c4d9d 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -43,6 +43,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -164,11 +165,12 @@ public class GridFunc {
     private static final IgniteClosure<ClusterNode, UUID> NODE2ID = new ClusterNodeGetIdClosure();
 
     /** */
-    private static final IgniteClosure<ClusterNode, Object> NODE2CONSISTENTID = new IgniteClosure<ClusterNode, Object>() {
-        @Override public Object apply(ClusterNode node) {
-            return node.consistentId();
-        }
-    };
+    private static final IgniteClosure<BaselineNode, Object> NODE2CONSISTENTID =
+        new IgniteClosure<BaselineNode, Object>() {
+            @Override public Object apply(BaselineNode node) {
+                return node.consistentId();
+            }
+        };
 
     /**
      * Gets predicate that evaluates to {@code true} only for given local node ID.
@@ -335,7 +337,7 @@ public class GridFunc {
      * @param nodes Collection of grid nodes.
      * @return Collection of node consistent IDs for given collection of grid nodes.
      */
-    public static Collection<Object> nodeConsistentIds(@Nullable Collection<? extends ClusterNode> nodes) {
+    public static Collection<Object> nodeConsistentIds(@Nullable Collection<? extends BaselineNode> nodes) {
         if (nodes == null || nodes.isEmpty())
             return Collections.emptyList();