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/24 11:29:26 UTC
[lucene-solr] 01/02: SOLR-14749: Improve registration of singletons
and listeners.
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
commit a772b0e7059c4b035e484f8f898e4af74f3fb466
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Sep 24 13:18:54 2020 +0200
SOLR-14749: Improve registration of singletons and listeners.
---
.../apache/solr/api/CustomContainerPlugins.java | 46 ++++++++++++++++++++++
.../org/apache/solr/cloud/ClusterSingleton.java | 5 +++
.../solr/cluster/events/ClusterEventListener.java | 4 +-
.../solr/cluster/events/ClusterEventProducer.java | 5 ++-
.../impl/CollectionsRepairEventListener.java | 11 ++++--
.../java/org/apache/solr/core/CoreContainer.java | 29 ++++++--------
.../solr/cluster/events/AllEventsListener.java | 20 ++++++++++
.../impl/CollectionsRepairEventListenerTest.java | 22 ++++++++++-
8 files changed, 120 insertions(+), 22 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 a8f2d07..63db89c 100644
--- a/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
+++ b/solr/core/src/java/org/apache/solr/api/CustomContainerPlugins.java
@@ -33,6 +33,9 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.lucene.analysis.util.ResourceLoaderAware;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.request.beans.PluginMeta;
+import org.apache.solr.cloud.ClusterSingleton;
+import org.apache.solr.cluster.events.ClusterEvent;
+import org.apache.solr.cluster.events.ClusterEventListener;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.annotation.JsonProperty;
@@ -112,6 +115,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
if (e.getValue() == Diff.REMOVED) {
ApiInfo apiInfo = currentPlugins.remove(e.getKey());
if (apiInfo == null) continue;
+ handleClusterSingleton(null, apiInfo);
for (ApiHolder holder : apiInfo.holders) {
Api old = containerApiBag.unregister(holder.api.getEndPoint().method()[0],
getActualPath(apiInfo, holder.api.getEndPoint().path()[0]));
@@ -141,6 +145,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
containerApiBag.register(holder, getTemplateVars(apiInfo.info));
}
currentPlugins.put(e.getKey(), apiInfo);
+ handleClusterSingleton(apiInfo, null);
} else {
//this plugin is being updated
ApiInfo old = currentPlugins.put(e.getKey(), apiInfo);
@@ -148,6 +153,7 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
//register all new paths
containerApiBag.register(holder, getTemplateVars(apiInfo.info));
}
+ handleClusterSingleton(apiInfo, old);
if (old != null) {
//this is an update of the plugin. But, it is possible that
// some paths are remved in the newer version of the plugin
@@ -168,6 +174,46 @@ public class CustomContainerPlugins implements ClusterPropertiesListener, MapWri
}
}
+ private void handleClusterSingleton(ApiInfo newApiInfo, ApiInfo oldApiInfo) {
+ if (newApiInfo != null) {
+ // register new api
+ Object instance = newApiInfo.getInstance();
+ if (instance instanceof ClusterSingleton) {
+ ClusterSingleton singleton = (ClusterSingleton) instance;
+ coreContainer.getClusterSingletons().put(singleton.getName(), singleton);
+ // easy check to see if we should immediately start this singleton
+ if (coreContainer.getClusterEventProducer().isRunning()) {
+ try {
+ singleton.start();
+ } catch (Exception exc) {
+ log.warn("Exception starting ClusterSingleton {}: {}", newApiInfo, exc);
+ }
+ }
+ }
+ if (instance instanceof ClusterEventListener) {
+ // XXX nocommit obtain a list of supported event types from the config
+ ClusterEvent.EventType[] types = ClusterEvent.EventType.values();
+ try {
+ coreContainer.getClusterEventProducer().registerListener((ClusterEventListener) instance, types);
+ } catch (Exception exc) {
+ log.warn("Exception adding ClusterEventListener {}: {}", newApiInfo, exc);
+ }
+ }
+ }
+ if (oldApiInfo != null) {
+ // stop & unregister the old api
+ Object instance = oldApiInfo.getInstance();
+ if (instance instanceof ClusterSingleton) {
+ ClusterSingleton singleton = (ClusterSingleton) instance;
+ singleton.stop();
+ coreContainer.getClusterSingletons().remove(singleton.getName());
+ }
+ if (instance instanceof ClusterEventListener) {
+ coreContainer.getClusterEventProducer().unregisterListener((ClusterEventListener) instance);
+ }
+ }
+ }
+
private static String getActualPath(ApiInfo apiInfo, String path) {
path = path.replaceAll("\\$path-prefix", apiInfo.info.pathPrefix);
path = path.replaceAll("\\$plugin-name", apiInfo.info.name);
diff --git a/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java b/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
index 3aa7ec1..1ae1eed 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ClusterSingleton.java
@@ -34,6 +34,11 @@ package org.apache.solr.cloud;
public interface ClusterSingleton {
/**
+ * Unique name of this singleton. Used for registration.
+ */
+ String getName();
+
+ /**
* Start the operation of the component. On return the component is assumed
* to be in the RUNNING state.
* @throws Exception on startup errors. The component should revert to the
diff --git a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java
index 6f84457..592f118 100644
--- a/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java
+++ b/solr/core/src/java/org/apache/solr/cluster/events/ClusterEventListener.java
@@ -16,13 +16,15 @@
*/
package org.apache.solr.cluster.events;
+import org.apache.solr.cloud.ClusterSingleton;
+
/**
* Components that want to be notified of cluster-wide events should use this.
*
* XXX should this work only for ClusterSingleton-s? some types of events may be
* XXX difficult (or pointless) to propagate to every node.
*/
-public interface ClusterEventListener {
+public interface ClusterEventListener extends ClusterSingleton {
/**
* Handle the event. Implementations should be non-blocking - if any long
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 cd47d0e..1c2327b 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
@@ -18,7 +18,6 @@ package org.apache.solr.cluster.events;
import org.apache.solr.cloud.ClusterSingleton;
-import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
@@ -32,6 +31,10 @@ public interface ClusterEventProducer extends ClusterSingleton {
String PLUGIN_NAME = "clusterEventProducer";
+ default String getName() {
+ return PLUGIN_NAME;
+ }
+
/**
* 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/CollectionsRepairEventListener.java b/solr/core/src/java/org/apache/solr/cluster/events/impl/CollectionsRepairEventListener.java
index dc4cdb5..88df58f 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
@@ -34,7 +34,6 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.cloud.api.collections.Assign;
import org.apache.solr.cluster.events.ClusterEvent;
import org.apache.solr.cluster.events.ClusterEventListener;
-import org.apache.solr.cloud.ClusterSingleton;
import org.apache.solr.cluster.events.NodesDownEvent;
import org.apache.solr.cluster.events.ReplicasDownEvent;
import org.apache.solr.common.cloud.ClusterState;
@@ -51,10 +50,11 @@ import org.slf4j.LoggerFactory;
* <p>NOTE 2: this functionality would be probably more reliable when executed also as a
* periodically scheduled check - both as a reactive (listener) and proactive (scheduled) measure.</p>
*/
-public class CollectionsRepairEventListener implements ClusterSingleton, ClusterEventListener {
+public class CollectionsRepairEventListener implements ClusterEventListener {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
- private static final String ASYNC_ID_PREFIX = "_col_repair_";
+ public static final String PLUGIN_NAME = "collectionsRepairListener";
+ private static final String ASYNC_ID_PREFIX = "_async_" + PLUGIN_NAME;
private static final AtomicInteger counter = new AtomicInteger();
private final SolrClient solrClient;
@@ -68,6 +68,11 @@ public class CollectionsRepairEventListener implements ClusterSingleton, Cluster
}
@Override
+ public String getName() {
+ return PLUGIN_NAME;
+ }
+
+ @Override
public void onEvent(ClusterEvent event) {
if (!isRunning()) {
// ignore the event
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 0a106cb..52627a9 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -250,9 +250,10 @@ public class CoreContainer {
private final ObjectCache objectCache = new ObjectCache();
+ private final Map<String, ClusterSingleton> clusterSingletons = new ConcurrentHashMap<>();
+
private PackageStoreAPI packageStoreAPI;
private PackageLoader packageLoader;
- private Map<String, ClusterSingleton> clusterSingletons = null;
private Set<Path> allowPaths;
@@ -897,32 +898,28 @@ public class CoreContainer {
containerHandlers.getApiBag().registerObject(containerPluginsApi.readAPI);
containerHandlers.getApiBag().registerObject(containerPluginsApi.editAPI);
+ // init ClusterSingleton-s
+
+ // register handlers that are also ClusterSingleton
+ containerHandlers.keySet().forEach(handlerName -> {
+ SolrRequestHandler handler = containerHandlers.get(handlerName);
+ if (handler instanceof ClusterSingleton) {
+ clusterSingletons.put(handlerName, (ClusterSingleton) handler);
+ }
+ });
// 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);
+ clusterSingletons.put(ClusterEventProducer.PLUGIN_NAME, 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) {
- singletons.put(handlerName, (ClusterSingleton) handler);
- }
- });
zkSys.getZkController().checkOverseerDesignate();
- // XXX note that these ClusterSingleton components are registered too late -
+ // XXX note that 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();
diff --git a/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java b/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java
index a8eae86..9d03c8d 100644
--- a/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java
+++ b/solr/core/src/test/org/apache/solr/cluster/events/AllEventsListener.java
@@ -38,6 +38,11 @@ public class AllEventsListener implements ClusterEventListener {
Map<ClusterEvent.EventType, List<ClusterEvent>> events = new HashMap<>();
@Override
+ public String getName() {
+ return "allEventsListener";
+ }
+
+ @Override
public void onEvent(ClusterEvent event) {
events.computeIfAbsent(event.getType(), type -> new ArrayList<>()).add(event);
if (event.getType() == expectedType) {
@@ -56,4 +61,19 @@ public class AllEventsListener implements ClusterEventListener {
Assert.fail("Timed out waiting for expected event " + expectedType);
}
}
+
+ @Override
+ public void start() throws Exception {
+
+ }
+
+ @Override
+ public boolean isRunning() {
+ return false;
+ }
+
+ @Override
+ public void stop() {
+
+ }
}
diff --git a/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java b/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java
index 6213dac..a764051 100644
--- a/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/events/impl/CollectionsRepairEventListenerTest.java
@@ -43,7 +43,6 @@ public class CollectionsRepairEventListenerTest extends SolrCloudTestCase {
CollectionsRepairWrapperListener(CoreContainer cc) throws Exception {
delegate = new CollectionsRepairEventListener(cc);
- delegate.start();
}
@Override
@@ -51,6 +50,26 @@ public class CollectionsRepairEventListenerTest extends SolrCloudTestCase {
delegate.onEvent(event);
completed.countDown();
}
+
+ @Override
+ public String getName() {
+ return "wrapperListener";
+ }
+
+ @Override
+ public void start() throws Exception {
+ delegate.start();
+ }
+
+ @Override
+ public boolean isRunning() {
+ return delegate.isRunning();
+ }
+
+ @Override
+ public void stop() {
+ delegate.stop();
+ }
}
private static AllEventsListener eventsListener = new AllEventsListener();
@@ -69,6 +88,7 @@ public class CollectionsRepairEventListenerTest extends SolrCloudTestCase {
repairListener = new CollectionsRepairWrapperListener(cc);
cc.getClusterEventProducer()
.registerListener(repairListener, ClusterEvent.EventType.NODES_DOWN);
+ repairListener.start();
}
@Before