You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2020/09/17 08:53:10 UTC

[lucene-solr] branch jira/solr-14749 updated: SOLR-14749: Attempt to load ClusterSingleton-s and the ClusterEventProducer from plugins.

This is an automated email from the ASF dual-hosted git repository.

ab pushed a commit to branch jira/solr-14749
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/solr-14749 by this push:
     new 9bc81f1  SOLR-14749: Attempt to load ClusterSingleton-s and the ClusterEventProducer from plugins.
9bc81f1 is described below

commit 9bc81f1e7b71a712c82f57a710c8fd3a5216aa74
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Sep 17 10:52:23 2020 +0200

    SOLR-14749: Attempt to load ClusterSingleton-s and the ClusterEventProducer from plugins.
---
 .../apache/solr/api/CustomContainerPlugins.java    | 10 ++-
 .../src/java/org/apache/solr/cloud/Overseer.java   | 17 +++-
 .../solr/cluster/events/ClusterEventProducer.java  |  2 +
 .../events/impl/ClusterEventProducerImpl.java      | 19 ++---
 .../impl/CollectionsRepairEventListener.java       |  1 -
 .../solr/cluster/events/impl/package-info.java     | 23 ++++++
 .../apache/solr/cluster/events/package-info.java   | 23 ++++++
 .../java/org/apache/solr/core/CoreContainer.java   | 32 ++++++--
 .../solr/handler/admin/ContainerPluginsApi.java    | 14 ++--
 .../apache/solr/packagemanager/PackageManager.java |  3 +-
 .../cluster/events/ClusterEventProducerTest.java   | 90 ++++++++++++++++++++++
 11 files changed, 201 insertions(+), 33 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
index f24626b..ec67638 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -67,6 +67,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
     refresh();
     return false;
   }
+
   public CustomContainerPlugins(CoreContainer coreContainer, ApiBag apiBag) {
     this.coreContainer = coreContainer;
     this.containerApiBag = apiBag;
@@ -77,6 +78,10 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
     currentPlugins.forEach(ew.getBiConsumer());
   }
 
+  public synchronized ApiInfo getPlugin(String name) {
+    return currentPlugins.get(name);
+  }
+
   public synchronized void refresh() {
     Map<String, Object> pluginInfos = null;
     try {
@@ -221,6 +226,9 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
       return null;
     }
 
+    public Object getInstance() {
+      return instance;
+    }
 
     @SuppressWarnings({"unchecked","rawtypes"})
     public ApiInfo(PluginMeta info, List<String> errs) {
@@ -267,7 +275,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
             errs.add("Only one HTTP method and url supported for each API");
           }
           if (endPoint.method().length != 1 || endPoint.path().length != 1) {
-            errs.add("The @EndPint must have exactly one method and path attributes");
+            errs.add("The @EndPoint must have exactly one method and path attributes");
           }
           List<String> pathSegments = StrUtils.splitSmart(endPoint.path()[0], '/', true);
           PathTrie.replaceTemplates(pathSegments, getTemplateVars(info));
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 733ddaa..b8403f4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -781,8 +781,15 @@ public class Overseer implements SolrCloseable {
   /**
    * Start {@link ClusterSingleton} plugins when we become the leader.
    */
-  private void startClusterSingletons() {
-    getCoreContainer().getClusterSingletons().forEach((name, singleton) -> {
+  public void startClusterSingletons() {
+    Map<String, ClusterSingleton> singletons = getCoreContainer().getClusterSingletons();
+    if (singletons == null) {
+      return;
+    }
+    if (isClosed()) {
+      return;
+    }
+    singletons.forEach((name, singleton) -> {
       try {
         singleton.start();
         if (singleton instanceof ClusterEventListener) {
@@ -798,7 +805,11 @@ public class Overseer implements SolrCloseable {
    * Stop {@link ClusterSingleton} plugins when we lose leadership.
    */
   private void stopClusterSingletons() {
-    getCoreContainer().getClusterSingletons().forEach((name, singleton) -> {
+    Map<String, ClusterSingleton> singletons = getCoreContainer().getClusterSingletons();
+    if (singletons == null) {
+      return;
+    }
+    singletons.forEach((name, singleton) -> {
       if (singleton instanceof ClusterEventListener) {
         getCoreContainer().getClusterEventProducer().unregisterListener((ClusterEventListener) singleton);
       }
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
index a46703d..0d01699 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventProducer.java
@@ -26,6 +26,8 @@ import java.util.concurrent.ConcurrentHashMap;
  */
 public interface ClusterEventProducer {
 
+  String PLUGIN_NAME = "clusterEventProducer";
+
   /**
    * Returns a modifiable map of event types and listeners to process events
    * of a given type.
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java
index 3186b18..a6897e7 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/impl/ClusterEventProducerImpl.java
@@ -55,7 +55,7 @@ public class ClusterEventProducerImpl implements ClusterEventProducer, ClusterSi
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private final Map<ClusterEvent.EventType, Set<ClusterEventListener>> listeners = new HashMap<>();
-  private final CoreContainer cc;
+  private CoreContainer coreContainer;
   private LiveNodesListener liveNodesListener;
   private CloudCollectionsListener cloudCollectionsListener;
   private ClusterPropertiesListener clusterPropertiesListener;
@@ -75,19 +75,19 @@ public class ClusterEventProducerImpl implements ClusterEventProducer, ClusterSi
   private volatile boolean isClosed = false;
 
   public ClusterEventProducerImpl(CoreContainer coreContainer) {
-    this.cc = coreContainer;
+    this.coreContainer = coreContainer;
   }
 
   // ClusterSingleton lifecycle methods
   @Override
   public void start() {
-    this.zkController = this.cc.getZkController();
-    if (zkController == null) {
+    if (coreContainer == null) {
       liveNodesListener = null;
       cloudCollectionsListener = null;
       clusterPropertiesListener = null;
       return;
     }
+    this.zkController = this.coreContainer.getZkController();
 
     // clean up any previous instances
     doStop();
@@ -136,6 +136,7 @@ public class ClusterEventProducerImpl implements ClusterEventProducer, ClusterSi
       }
       return false;
     };
+    zkController.zkStateReader.registerLiveNodesListener(liveNodesListener);
 
     cloudCollectionsListener = ((oldCollections, newCollections) -> {
       if (oldCollections.equals(newCollections)) {
@@ -231,19 +232,12 @@ public class ClusterEventProducerImpl implements ClusterEventProducer, ClusterSi
     clusterPropertiesListener = null;
   }
 
-  private void ensureRunning() {
-    if (isClosed || !running) {
-      throw new RuntimeException("ClusterEventProducerImpl is not running.");
-    }
-  }
-
   @Override
   public void registerListener(ClusterEventListener listener) throws Exception {
-    ensureRunning();
     try {
       listener.getEventTypes().forEach(type -> {
         if (!supportedEvents.contains(type)) {
-          throw new RuntimeException("event type " + type + " not supported yet");
+          log.warn("event type {} not supported yet.", type);
         }
       });
     } catch (Throwable e) {
@@ -261,7 +255,6 @@ public class ClusterEventProducerImpl implements ClusterEventProducer, ClusterSi
 
   @Override
   public Map<ClusterEvent.EventType, Set<ClusterEventListener>> getEventListeners() {
-    ensureRunning();
     return listeners;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
index fc2ac25..773362a 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
@@ -32,7 +32,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * XXX nocommit.
  * This is an (incomplete) illustration how to re-implement the combination of 8x
  * NodeLostTrigger and AutoAddReplicasPlanAction to maintain the collection's replication factor.
  */
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/impl/package-info.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/package-info.java
new file mode 100644
index 0000000..2c115b6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/events/impl/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Default implementation of {@link org.apache.solr.cluster.events.ClusterEventProducer}.
+ */
+package org.apache.solr.cluster.events.impl;
+
+
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/package-info.java b/solr/core/src/java/org/apache/solr/cluster/events/package-info.java
new file mode 100644
index 0000000..a334a00
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cluster/events/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Interfaces defining support for cluster-level event generation and processing.
+ */
+package org.apache.solr.cluster.events;
+
+
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 54c113d..555827e 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -69,6 +69,7 @@ import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ClusterSingleton;
+import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.OverseerTaskQueue;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cluster.events.ClusterEventProducer;
@@ -177,7 +178,6 @@ public class CoreContainer {
   public final Supplier<SolrZkClient> zkClientSupplier = () -> getZkController().getZkClient();
 
   private final CustomContainerPlugins customContainerPlugins =  new CustomContainerPlugins(this, containerHandlers.getApiBag());
-  private final Map<String, ClusterSingleton> clusterSingletons = new ConcurrentHashMap<>();
 
   protected final Map<String, CoreLoadFailure> coreInitFailures = new ConcurrentHashMap<>();
 
@@ -251,6 +251,7 @@ public class CoreContainer {
 
   private PackageStoreAPI packageStoreAPI;
   private PackageLoader packageLoader;
+  private Map<String, ClusterSingleton> clusterSingletons = null;
 
   private Set<Path> allowPaths;
 
@@ -891,20 +892,37 @@ public class CoreContainer {
       ContainerPluginsApi containerPluginsApi = new ContainerPluginsApi(this);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
       containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
-      // create the default ClusterEventProducer
-      // XXX can we load it as a pluggable component? or configurable?
-      clusterEventProducer = new ClusterEventProducerImpl(this);
+
+      // create the ClusterEventProducer
+      CustomContainerPlugins.ApiInfo clusterEventProducerInfo = customContainerPlugins.getPlugin(ClusterEventProducer.PLUGIN_NAME);
+      if (clusterEventProducerInfo != null) {
+        clusterEventProducer = (ClusterEventProducer) clusterEventProducerInfo.getInstance();
+      } else {
+        clusterEventProducer = new ClusterEventProducerImpl(this);
+      }
+      // init ClusterSingleton-s
+      Map<String, ClusterSingleton> singletons = new ConcurrentHashMap<>();
+      if (clusterEventProducer instanceof ClusterSingleton) {
+        singletons.put(ClusterEventProducer.PLUGIN_NAME, (ClusterSingleton) clusterEventProducer);
+      }
+
       // register ClusterSingleton handlers
       // XXX register also other ClusterSingleton-s from packages - how?
       containerHandlers.keySet().forEach(handlerName -> {
         SolrRequestHandler handler = containerHandlers.get(handlerName);
         if (handler instanceof ClusterSingleton) {
-          clusterSingletons.put(handlerName, (ClusterSingleton) handler);
+          singletons.put(handlerName, (ClusterSingleton) handler);
         }
       });
-      // our default clusterEventProducer is also a ClusterSingleton
-      clusterSingletons.put("clusterEventProducer", (ClusterSingleton) clusterEventProducer);
       zkSys.getZkController().checkOverseerDesignate();
+
+      // XXX note that these ClusterSingleton components are registered too late -
+      // XXX the Overseer leader may be already started
+      clusterSingletons = singletons;
+      Overseer overseer = zkSys.getZkController().getOverseer();
+      if (!overseer.isClosed()) { // we are the leader
+        overseer.startClusterSingletons();
+      }
     }
     // This is a bit redundant but these are two distinct concepts for all they're accomplished at the same time.
     status |= LOAD_COMPLETE | INITIAL_CORE_LOAD_COMPLETE;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
index 0c7a487..9f2f459 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ContainerPluginsApi.java
@@ -51,7 +51,7 @@ import static org.apache.lucene.util.IOUtils.closeWhileHandlingException;
 public class ContainerPluginsApi {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String PLUGIN = "plugin";
+  public static final String PLUGINS = "plugins";
   private final Supplier<SolrZkClient> zkClientSupplier;
   private final CoreContainer coreContainer;
   public final Read readAPI = new Read();
@@ -64,15 +64,15 @@ public class ContainerPluginsApi {
 
   public class Read {
     @EndPoint(method = METHOD.GET,
-        path = "/cluster/plugin",
+        path = "/cluster/plugins",
         permission = PermissionNameProvider.Name.COLL_READ_PERM)
     public void list(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
-      rsp.add(PLUGIN, plugins(zkClientSupplier));
+      rsp.add(PLUGINS, plugins(zkClientSupplier));
     }
   }
 
   @EndPoint(method = METHOD.POST,
-      path = "/cluster/plugin",
+      path = "/cluster/plugins",
       permission = PermissionNameProvider.Name.COLL_EDIT_PERM)
   public class Edit {
 
@@ -151,7 +151,7 @@ public class ContainerPluginsApi {
     SolrZkClient zkClient = zkClientSupplier.get();
     try {
       Map<String, Object> clusterPropsJson = (Map<String, Object>) Utils.fromJSON(zkClient.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
-      return (Map<String, Object>) clusterPropsJson.computeIfAbsent(PLUGIN, o -> new LinkedHashMap<>());
+      return (Map<String, Object>) clusterPropsJson.computeIfAbsent(PLUGINS, o -> new LinkedHashMap<>());
     } catch (KeeperException.NoNodeException e) {
       return new LinkedHashMap<>();
     } catch (KeeperException | InterruptedException e) {
@@ -165,9 +165,9 @@ public class ContainerPluginsApi {
       zkClientSupplier.get().atomicUpdate(ZkStateReader.CLUSTER_PROPS, bytes -> {
         Map rawJson = bytes == null ? new LinkedHashMap() :
             (Map) Utils.fromJSON(bytes);
-        Map pluginsModified = modifier.apply((Map) rawJson.computeIfAbsent(PLUGIN, o -> new LinkedHashMap<>()));
+        Map pluginsModified = modifier.apply((Map) rawJson.computeIfAbsent(PLUGINS, o -> new LinkedHashMap<>()));
         if (pluginsModified == null) return null;
-        rawJson.put(PLUGIN, pluginsModified);
+        rawJson.put(PLUGINS, pluginsModified);
         return Utils.toJSON(rawJson);
       });
     } catch (KeeperException | InterruptedException e) {
diff --git a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
index 424f604..044cf8c 100644
--- a/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
+++ b/solr/core/src/java/org/apache/solr/packagemanager/PackageManager.java
@@ -52,6 +52,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.filestore.DistribPackageStore;
+import org.apache.solr.handler.admin.ContainerPluginsApi;
 import org.apache.solr.packagemanager.SolrPackage.Command;
 import org.apache.solr.packagemanager.SolrPackage.Manifest;
 import org.apache.solr.packagemanager.SolrPackage.Plugin;
@@ -231,7 +232,7 @@ public class PackageManager implements Closeable {
       }
     }
     @SuppressWarnings({"unchecked"})
-    Map<String, Object> clusterPlugins = (Map<String, Object>) result.getOrDefault("plugin", Collections.emptyMap());
+    Map<String, Object> clusterPlugins = (Map<String, Object>) result.getOrDefault(ContainerPluginsApi.PLUGINS, Collections.emptyMap());
     for (String key : clusterPlugins.keySet()) {
       // Map<String, String> pluginMeta = (Map<String, String>) clusterPlugins.get(key);
       PluginMeta pluginMeta;
diff --git a/solr/core/src/test/org/apache/solr/cluster/events/ClusterEventProducerTest.java b/solr/core/src/test/org/apache/solr/cluster/events/ClusterEventProducerTest.java
new file mode 100644
index 0000000..bde8ca4
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/events/ClusterEventProducerTest.java
@@ -0,0 +1,90 @@
+package org.apache.solr.cluster.events;
+
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ *
+ */
+public class ClusterEventProducerTest extends SolrCloudTestCase {
+
+  public static class AllEventsListener implements ClusterEventListener {
+    Map<ClusterEvent.EventType, List<ClusterEvent>> events = new HashMap<>();
+
+    @Override
+    public Set<ClusterEvent.EventType> getEventTypes() {
+      return new HashSet<>(Arrays.asList(ClusterEvent.EventType.values()));
+    }
+
+    @Override
+    public void onEvent(ClusterEvent event) {
+      events.computeIfAbsent(event.getType(), type -> new ArrayList<>()).add(event);
+    }
+  }
+
+  private static AllEventsListener eventsListener = new AllEventsListener();
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(3)
+        .addConfig("conf", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
+        .configure();
+    cluster.getOpenOverseer().getCoreContainer().getClusterEventProducer().registerListener(eventsListener);
+  }
+
+  @Before
+  public void setUp() throws Exception  {
+    super.setUp();
+    cluster.deleteAllCollections();
+  }
+
+  @Test
+  public void testNodesEvent() throws Exception {
+
+    // NODES_DOWN
+
+    // don't kill Overseer
+    JettySolrRunner nonOverseerJetty = null;
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      if (cluster.getOpenOverseer().getCoreContainer().getZkController().getNodeName().equals(jetty.getNodeName())) {
+        continue;
+      }
+      nonOverseerJetty = jetty;
+      break;
+    }
+    String nodeName = nonOverseerJetty.getNodeName();
+    cluster.stopJettySolrRunner(nonOverseerJetty);
+    cluster.waitForJettyToStop(nonOverseerJetty);
+    assertNotNull("should be NODES_DOWN events", eventsListener.events.get(ClusterEvent.EventType.NODES_DOWN));
+    List<ClusterEvent> events = eventsListener.events.get(ClusterEvent.EventType.NODES_DOWN);
+    assertEquals("should be one NODES_DOWN event", 1, events.size());
+    ClusterEvent event = events.get(0);
+    assertEquals("should be NODES_DOWN event type", ClusterEvent.EventType.NODES_DOWN, event.getType());
+    NodesDownEvent nodesDown = (NodesDownEvent) event;
+    assertEquals("should be node " + nodeName, nodeName, nodesDown.getNodeNames().iterator().next());
+
+    // NODES_UP
+    JettySolrRunner newNode = cluster.startJettySolrRunner();
+    cluster.waitForNode(newNode, 60);
+    assertNotNull("should be NODES_UP events", eventsListener.events.get(ClusterEvent.EventType.NODES_UP));
+    events = eventsListener.events.get(ClusterEvent.EventType.NODES_UP);
+    assertEquals("should be one NODES_UP event", 1, events.size());
+    event = events.get(0);
+    assertEquals("should be NODES_UP event type", ClusterEvent.EventType.NODES_UP, event.getType());
+    NodesUpEvent nodesUp = (NodesUpEvent) event;
+    assertEquals("should be node " + newNode.getNodeName(), newNode.getNodeName(), nodesUp.getNodeNames().iterator().next());
+
+
+  }
+}