mirror of https://github.com/apache/lucene.git
SOLR-7408: Let SolrCore be the only thing which registers/unregisters a config directory listener
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1675274 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ede96a0dd1
commit
5afa59c004
|
@ -156,6 +156,9 @@ Bug Fixes
|
|||
* SOLR-7440: DebugComponent does not return the right requestPurpose for pivot facet refinements.
|
||||
(shalin)
|
||||
|
||||
* SOLR-7408: Listeners set by SolrCores on config directories in ZK could be removed if collections
|
||||
are created/deleted in paralle against the same config set. (Shai Erera, Anshum Gupta)
|
||||
|
||||
Optimizations
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -96,6 +96,8 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
import org.slf4j.MDC;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
|
||||
/**
|
||||
* Handle ZooKeeper interactions.
|
||||
* <p>
|
||||
|
@ -1209,14 +1211,13 @@ public final class ZkController {
|
|||
return true;
|
||||
}
|
||||
|
||||
public void unregister(String coreName, CoreDescriptor cd, String configLocation)
|
||||
throws InterruptedException, KeeperException {
|
||||
public void unregister(String coreName, CoreDescriptor cd) throws InterruptedException, KeeperException {
|
||||
final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
|
||||
final String collection = cd.getCloudDescriptor().getCollectionName();
|
||||
assert collection != null;
|
||||
|
||||
if (collection == null || collection.trim().length() == 0) {
|
||||
if (Strings.isNullOrEmpty(collection)) {
|
||||
log.error("No collection was specified.");
|
||||
assert false : "No collection was specified [" + collection + "]";
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -1226,37 +1227,28 @@ public final class ZkController {
|
|||
context.cancelElection();
|
||||
}
|
||||
|
||||
final Collection<SolrCore> cores = cc.getCores();
|
||||
|
||||
// if there is no SolrCore which is a member of this collection, remove the watch
|
||||
CloudDescriptor cloudDescriptor = cd.getCloudDescriptor();
|
||||
boolean removeWatch = true;
|
||||
// if there is no SolrCore which is a member of this collection, remove the watch
|
||||
for (SolrCore solrCore : cc.getCores()) {
|
||||
if (((ZkSolrResourceLoader) solrCore.getResourceLoader()).getConfigSetZkPath().equals(configLocation))
|
||||
configLocation = null; //if a core uses this config dir , then set it to null
|
||||
|
||||
|
||||
CloudDescriptor cloudDesc = solrCore.getCoreDescriptor()
|
||||
.getCloudDescriptor();
|
||||
if (cloudDesc != null
|
||||
&& cloudDescriptor.getCollectionName().equals(
|
||||
cloudDesc.getCollectionName())) {
|
||||
for (SolrCore solrCore : cores) {
|
||||
final CloudDescriptor cloudDesc = solrCore.getCoreDescriptor().getCloudDescriptor();
|
||||
if (cloudDesc != null && cloudDescriptor.getCollectionName().equals(cloudDesc.getCollectionName())) {
|
||||
removeWatch = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (removeWatch) zkStateReader.removeZKWatch(collection);
|
||||
|
||||
if (removeWatch) {
|
||||
zkStateReader.removeZKWatch(collection);
|
||||
}
|
||||
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
|
||||
OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
|
||||
ZkStateReader.NODE_NAME_PROP, getNodeName(),
|
||||
ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
|
||||
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
||||
overseerJobQueue.offer(ZkStateReader.toJSON(m));
|
||||
|
||||
if (configLocation != null) {
|
||||
synchronized (confDirectoryListeners) {
|
||||
log.info("This conf directory is no more watched {}", configLocation);
|
||||
confDirectoryListeners.remove(configLocation);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void createCollection(String collection) throws KeeperException,
|
||||
|
@ -2254,18 +2246,19 @@ public final class ZkController {
|
|||
}
|
||||
}
|
||||
|
||||
public void unRegisterConfListener(Runnable listener) {
|
||||
if (listener == null) return;
|
||||
private void unregisterConfListener(String confDir, Runnable listener) {
|
||||
synchronized (confDirectoryListeners) {
|
||||
for (Set<Runnable> listeners : confDirectoryListeners.values()) {
|
||||
if (listeners != null) {
|
||||
if (listeners.remove(listener)) {
|
||||
log.info(" a listener was removed because of core close");
|
||||
}
|
||||
}
|
||||
final Set<Runnable> listeners = confDirectoryListeners.get(confDir);
|
||||
assert listeners != null : confDir + " has no more registered listeners, but a live one attempts to unregister!";
|
||||
if (listeners.remove(listener)) {
|
||||
log.info("removed listener for config directory [{}]", confDir);
|
||||
}
|
||||
if (listeners.isEmpty()) {
|
||||
// no more listeners for this confDir, remove it from the map
|
||||
log.info("No more listeners for config directory [{}]", confDir);
|
||||
confDirectoryListeners.remove(confDir);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2274,37 +2267,41 @@ public final class ZkController {
|
|||
* item of interest has been modified. When the last core which was interested in
|
||||
* this conf directory is gone the listeners will be removed automatically.
|
||||
*/
|
||||
public void registerConfListenerForCore(String confDir, SolrCore core, final Runnable listener) {
|
||||
if (listener == null) throw new NullPointerException("listener cannot be null");
|
||||
synchronized (confDirectoryListeners) {
|
||||
if (confDirectoryListeners.containsKey(confDir)) {
|
||||
confDirectoryListeners.get(confDir).add(listener);
|
||||
core.addCloseHook(new CloseHook() {
|
||||
@Override
|
||||
public void preClose(SolrCore core) {
|
||||
unRegisterConfListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postClose(SolrCore core) {
|
||||
}
|
||||
});
|
||||
} else {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "This conf directory is not valid "+ confDir);
|
||||
}
|
||||
public void registerConfListenerForCore(final String confDir, SolrCore core, final Runnable listener) {
|
||||
if (listener == null) {
|
||||
throw new NullPointerException("listener cannot be null");
|
||||
}
|
||||
synchronized (confDirectoryListeners) {
|
||||
final Set<Runnable> confDirListeners = getConfDirListeners(confDir);
|
||||
confDirListeners.add(listener);
|
||||
core.addCloseHook(new CloseHook() {
|
||||
@Override
|
||||
public void preClose(SolrCore core) {
|
||||
unregisterConfListener(confDir, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postClose(SolrCore core) {
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
// this method is called in a protected confDirListeners block
|
||||
private Set<Runnable> getConfDirListeners(final String confDir) {
|
||||
assert Thread.holdsLock(confDirectoryListeners) : "confDirListeners lock not held by thread";
|
||||
Set<Runnable> confDirListeners = confDirectoryListeners.get(confDir);
|
||||
if (confDirListeners == null) {
|
||||
log.info("watch zkdir {}" , confDir);
|
||||
confDirListeners = new HashSet<>();
|
||||
confDirectoryListeners.put(confDir, confDirListeners);
|
||||
setConfWatcher(confDir, new WatcherImpl(confDir), null);
|
||||
}
|
||||
return confDirListeners;
|
||||
}
|
||||
|
||||
private final Map<String, Set<Runnable>> confDirectoryListeners = new HashMap<>();
|
||||
|
||||
void watchZKConfDir(final String zkDir) {
|
||||
log.info("watch zkdir {}" , zkDir);
|
||||
if (!confDirectoryListeners.containsKey(zkDir)) {
|
||||
confDirectoryListeners.put(zkDir, new HashSet<>());
|
||||
setConfWatcher(zkDir, new WatcherImpl(zkDir), null);
|
||||
}
|
||||
}
|
||||
|
||||
private class WatcherImpl implements Watcher {
|
||||
private final String zkDir;
|
||||
|
||||
|
@ -2335,7 +2332,6 @@ public final class ZkController {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private boolean fireEventListeners(String zkDir) {
|
||||
|
@ -2345,11 +2341,12 @@ public final class ZkController {
|
|||
log.info("Watcher on {} is removed ", zkDir);
|
||||
return false;
|
||||
}
|
||||
Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
|
||||
final Set<Runnable> listeners = confDirectoryListeners.get(zkDir);
|
||||
if (listeners != null && !listeners.isEmpty()) {
|
||||
final Set<Runnable> listenersCopy = new HashSet<>(listeners);
|
||||
new Thread() {
|
||||
//run these in a separate thread because this can be long running
|
||||
// run these in a separate thread because this can be long running
|
||||
@Override
|
||||
public void run() {
|
||||
log.info("Running listeners for {}", zkDir);
|
||||
for (final Runnable listener : listenersCopy) {
|
||||
|
@ -2362,7 +2359,6 @@ public final class ZkController {
|
|||
}
|
||||
}.start();
|
||||
}
|
||||
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -2371,7 +2367,7 @@ public final class ZkController {
|
|||
try {
|
||||
Stat newStat = zkClient.exists(zkDir, watcher, true);
|
||||
if (stat != null && newStat.getVersion() > stat.getVersion()) {
|
||||
//a race condition where a we missed an even fired
|
||||
//a race condition where a we missed an event fired
|
||||
//so fire the event listeners
|
||||
fireEventListeners(zkDir);
|
||||
}
|
||||
|
@ -2389,7 +2385,7 @@ public final class ZkController {
|
|||
public void command() {
|
||||
synchronized (confDirectoryListeners) {
|
||||
for (String s : confDirectoryListeners.keySet()) {
|
||||
watchZKConfDir(s);
|
||||
setConfWatcher(s, new WatcherImpl(s), null);
|
||||
fireEventListeners(s);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,7 +48,6 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
|
|||
super(instanceDir);
|
||||
this.zkController = zooKeeperController;
|
||||
configSetZkPath = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet;
|
||||
zkController.watchZKConfDir(configSetZkPath);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -63,7 +62,6 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
|
|||
super(instanceDir, parent, coreProperties);
|
||||
this.zkController = zooKeeperController;
|
||||
configSetZkPath = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet;
|
||||
zkController.watchZKConfDir(configSetZkPath);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,13 +17,25 @@
|
|||
|
||||
package org.apache.solr.core;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import static com.google.common.base.Preconditions.*;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import org.apache.solr.cloud.ZkController;
|
||||
import org.apache.solr.cloud.ZkSolrResourceLoader;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.IOUtils;
|
||||
import org.apache.solr.handler.RequestHandlerBase;
|
||||
import org.apache.solr.handler.admin.CollectionsHandler;
|
||||
import org.apache.solr.handler.admin.CoreAdminHandler;
|
||||
|
@ -37,21 +49,9 @@ import org.apache.solr.util.FileUtils;
|
|||
import org.apache.zookeeper.KeeperException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.slf4j.MDC;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import static com.google.common.base.Preconditions.checkNotNull;
|
||||
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -508,6 +508,7 @@ public class CoreContainer {
|
|||
throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Solr has close.");
|
||||
}
|
||||
|
||||
SolrCore core = null;
|
||||
try {
|
||||
|
||||
if (zkSys.getZkController() != null) {
|
||||
|
@ -516,7 +517,7 @@ public class CoreContainer {
|
|||
|
||||
ConfigSet coreConfig = coreConfigService.getConfig(dcore);
|
||||
log.info("Creating SolrCore '{}' using configuration from {}", dcore.getName(), coreConfig.getName());
|
||||
SolrCore core = new SolrCore(dcore, coreConfig);
|
||||
core = new SolrCore(dcore, coreConfig);
|
||||
solrCores.addCreated(core);
|
||||
|
||||
// always kick off recovery if we are in non-Cloud mode
|
||||
|
@ -527,15 +528,17 @@ public class CoreContainer {
|
|||
registerCore(dcore.getName(), core, publishState);
|
||||
|
||||
return core;
|
||||
|
||||
} catch (Exception e) {
|
||||
coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
|
||||
log.error("Error creating core [{}]: {}", dcore.getName(), e.getMessage(), e);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to create core [" + dcore.getName() + "]", e);
|
||||
final SolrException solrException = new SolrException(ErrorCode.SERVER_ERROR, "Unable to create core [" + dcore.getName() + "]", e);
|
||||
IOUtils.closeQuietly(core);
|
||||
throw solrException;
|
||||
} catch (Throwable t) {
|
||||
SolrException e = new SolrException(ErrorCode.SERVER_ERROR, "JVM Error creating core [" + dcore.getName() + "]: " + t.getMessage(), t);
|
||||
log.error("Error creating core [{}]: {}", dcore.getName(), t.getMessage(), t);
|
||||
coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
|
||||
IOUtils.closeQuietly(core);
|
||||
throw t;
|
||||
}
|
||||
|
||||
|
@ -694,7 +697,7 @@ public class CoreContainer {
|
|||
|
||||
if (zkSys.getZkController() != null) {
|
||||
try {
|
||||
zkSys.getZkController().unregister(name, cd, configSetZkPath);
|
||||
zkSys.getZkController().unregister(name, cd);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state");
|
||||
|
|
|
@ -17,7 +17,9 @@
|
|||
|
||||
package org.apache.solr.core;
|
||||
|
||||
import javax.xml.parsers.ParserConfigurationException;
|
||||
import static com.google.common.base.Preconditions.*;
|
||||
import static org.apache.solr.common.params.CommonParams.*;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
|
@ -50,7 +52,6 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
@ -143,9 +144,6 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.apache.zookeeper.data.Stat;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
import static org.apache.solr.common.params.CommonParams.PATH;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -184,8 +182,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
private final PluginBag<UpdateRequestProcessorFactory> updateProcessors = new PluginBag<>(UpdateRequestProcessorFactory.class, this);
|
||||
private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
|
||||
private final Map<String, SolrInfoMBean> infoRegistry;
|
||||
private IndexDeletionPolicyWrapper solrDelPolicy;
|
||||
private DirectoryFactory directoryFactory;
|
||||
private final IndexDeletionPolicyWrapper solrDelPolicy;
|
||||
private final DirectoryFactory directoryFactory;
|
||||
private IndexReaderFactory indexReaderFactory;
|
||||
private final Codec codec;
|
||||
private final MemClassLoader memClassLoader;
|
||||
|
@ -196,13 +194,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
|
||||
public long getStartTime() { return startTime; }
|
||||
|
||||
private RestManager restManager;
|
||||
private final RestManager restManager;
|
||||
|
||||
public RestManager getRestManager() {
|
||||
return restManager;
|
||||
}
|
||||
|
||||
static int boolean_query_max_clause_count = Integer.MIN_VALUE;
|
||||
|
||||
// only change the BooleanQuery maxClauseCount once for ALL cores...
|
||||
void booleanQueryMaxClauseCount() {
|
||||
synchronized(SolrCore.class) {
|
||||
|
@ -210,7 +209,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
boolean_query_max_clause_count = solrConfig.booleanQueryMaxClauseCount;
|
||||
BooleanQuery.setMaxClauseCount(boolean_query_max_clause_count);
|
||||
} else if (boolean_query_max_clause_count != solrConfig.booleanQueryMaxClauseCount ) {
|
||||
log.debug("BooleanQuery.maxClauseCount= " +boolean_query_max_clause_count+ ", ignoring " +solrConfig.booleanQueryMaxClauseCount);
|
||||
log.debug("BooleanQuery.maxClauseCount={}, ignoring {}", boolean_query_max_clause_count, solrConfig.booleanQueryMaxClauseCount);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -367,33 +366,37 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
return infoRegistry;
|
||||
}
|
||||
|
||||
private void initDeletionPolicy() {
|
||||
PluginInfo info = solrConfig.getPluginInfo(IndexDeletionPolicy.class.getName());
|
||||
IndexDeletionPolicy delPolicy = null;
|
||||
if(info != null){
|
||||
delPolicy = createInstance(info.className, IndexDeletionPolicy.class, "Deletion Policy for SOLR", this, getResourceLoader());
|
||||
if (delPolicy instanceof NamedListInitializedPlugin) {
|
||||
((NamedListInitializedPlugin) delPolicy).init(info.initArgs);
|
||||
}
|
||||
} else {
|
||||
delPolicy = new SolrDeletionPolicy();
|
||||
}
|
||||
solrDelPolicy = new IndexDeletionPolicyWrapper(delPolicy);
|
||||
}
|
||||
private IndexDeletionPolicyWrapper initDeletionPolicy(IndexDeletionPolicyWrapper delPolicyWrapper) {
|
||||
if (delPolicyWrapper != null) {
|
||||
return delPolicyWrapper;
|
||||
}
|
||||
|
||||
final PluginInfo info = solrConfig.getPluginInfo(IndexDeletionPolicy.class.getName());
|
||||
final IndexDeletionPolicy delPolicy;
|
||||
if (info != null) {
|
||||
delPolicy = createInstance(info.className, IndexDeletionPolicy.class, "Deletion Policy for SOLR", this, getResourceLoader());
|
||||
if (delPolicy instanceof NamedListInitializedPlugin) {
|
||||
((NamedListInitializedPlugin) delPolicy).init(info.initArgs);
|
||||
}
|
||||
} else {
|
||||
delPolicy = new SolrDeletionPolicy();
|
||||
}
|
||||
return new IndexDeletionPolicyWrapper(delPolicy);
|
||||
}
|
||||
|
||||
private void initListeners() {
|
||||
final Class<SolrEventListener> clazz = SolrEventListener.class;
|
||||
final String label = "Event Listener";
|
||||
for (PluginInfo info : solrConfig.getPluginInfos(SolrEventListener.class.getName())) {
|
||||
String event = info.attributes.get("event");
|
||||
if("firstSearcher".equals(event) ){
|
||||
SolrEventListener obj = createInitInstance(info,clazz,label,null);
|
||||
final String event = info.attributes.get("event");
|
||||
if ("firstSearcher".equals(event)) {
|
||||
SolrEventListener obj = createInitInstance(info, clazz, label, null);
|
||||
firstSearcherListeners.add(obj);
|
||||
log.info(logid + "Added SolrEventListener for firstSearcher: " + obj);
|
||||
} else if("newSearcher".equals(event) ){
|
||||
SolrEventListener obj = createInitInstance(info,clazz,label,null);
|
||||
log.info("[{}] Added SolrEventListener for firstSearcher: [{}]", logid, obj);
|
||||
} else if ("newSearcher".equals(event)) {
|
||||
SolrEventListener obj = createInitInstance(info, clazz, label, null);
|
||||
newSearcherListeners.add(obj);
|
||||
log.info(logid + "Added SolrEventListener for newSearcher: " + obj);
|
||||
log.info("[{}] Added SolrEventListener for newSearcher: [{}]", logid, obj);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -436,36 +439,39 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
return responseWriters.put(name, responseWriter);
|
||||
}
|
||||
|
||||
public SolrCore reload(ConfigSet coreConfig) throws IOException,
|
||||
ParserConfigurationException, SAXException {
|
||||
|
||||
public SolrCore reload(ConfigSet coreConfig) throws IOException {
|
||||
solrCoreState.increfSolrCoreState();
|
||||
SolrCore currentCore;
|
||||
boolean indexDirChange = !getNewIndexDir().equals(getIndexDir());
|
||||
if (indexDirChange) {
|
||||
final SolrCore currentCore;
|
||||
if (!getNewIndexDir().equals(getIndexDir())) {
|
||||
// the directory is changing, don't pass on state
|
||||
currentCore = null;
|
||||
} else {
|
||||
currentCore = this;
|
||||
}
|
||||
|
||||
SolrCore core = new SolrCore(getName(), getDataDir(), coreConfig.getSolrConfig(),
|
||||
coreConfig.getIndexSchema(), coreDescriptor, updateHandler, this.solrDelPolicy, currentCore);
|
||||
core.solrDelPolicy = this.solrDelPolicy;
|
||||
boolean success = false;
|
||||
SolrCore core = null;
|
||||
try {
|
||||
core = new SolrCore(getName(), getDataDir(), coreConfig.getSolrConfig(),
|
||||
coreConfig.getIndexSchema(), coreDescriptor, updateHandler, solrDelPolicy, currentCore);
|
||||
|
||||
// we open a new IndexWriter to pick up the latest config
|
||||
core.getUpdateHandler().getSolrCoreState().newIndexWriter(core, false);
|
||||
|
||||
// we open a new indexwriter to pick up the latest config
|
||||
core.getUpdateHandler().getSolrCoreState().newIndexWriter(core, false);
|
||||
|
||||
core.getSearcher(true, false, null, true);
|
||||
|
||||
return core;
|
||||
core.getSearcher(true, false, null, true);
|
||||
success = true;
|
||||
return core;
|
||||
} finally {
|
||||
// close the new core on any errors that have occurred.
|
||||
if (!success) {
|
||||
IOUtils.closeQuietly(core);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void initDirectoryFactory() {
|
||||
DirectoryFactory dirFactory;
|
||||
PluginInfo info = solrConfig.getPluginInfo(DirectoryFactory.class.getName());
|
||||
private DirectoryFactory initDirectoryFactory() {
|
||||
final PluginInfo info = solrConfig.getPluginInfo(DirectoryFactory.class.getName());
|
||||
final DirectoryFactory dirFactory;
|
||||
if (info != null) {
|
||||
log.info(info.className);
|
||||
dirFactory = getResourceLoader().newInstance(info.className, DirectoryFactory.class);
|
||||
|
@ -474,8 +480,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
log.info("solr.NRTCachingDirectoryFactory");
|
||||
dirFactory = new NRTCachingDirectoryFactory();
|
||||
}
|
||||
// And set it
|
||||
directoryFactory = dirFactory;
|
||||
return dirFactory;
|
||||
}
|
||||
|
||||
private void initIndexReaderFactory() {
|
||||
|
@ -645,6 +650,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
* @param schema a solr schema instance
|
||||
*
|
||||
* @since solr 1.3
|
||||
* @deprecated will be removed in the next release
|
||||
*/
|
||||
public SolrCore(String name, String dataDir, SolrConfig config, IndexSchema schema, CoreDescriptor cd) {
|
||||
this(name, dataDir, config, schema, cd, null, null, null);
|
||||
|
@ -654,13 +660,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
this(cd.getName(), null, coreConfig.getSolrConfig(), coreConfig.getIndexSchema(), cd, null, null, null);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a new core that is to be loaded lazily. i.e. lazyLoad="true" in solr.xml
|
||||
*
|
||||
* @since solr 4.1
|
||||
* @deprecated will be removed in the next release
|
||||
*/
|
||||
public SolrCore(String name, CoreDescriptor cd) {
|
||||
coreDescriptor = cd;
|
||||
public SolrCore(String name, CoreDescriptor coreDescriptor) {
|
||||
this.coreDescriptor = coreDescriptor;
|
||||
this.setName(name);
|
||||
this.schema = null;
|
||||
this.dataDir = null;
|
||||
|
@ -678,97 +685,59 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
this.codec = null;
|
||||
this.ruleExpiryLock = null;
|
||||
this.memClassLoader = null;
|
||||
|
||||
solrCoreState = null;
|
||||
this.directoryFactory = null;
|
||||
this.solrCoreState = null;
|
||||
this.restManager = null;
|
||||
this.solrDelPolicy = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new core and register it in the list of cores.
|
||||
* If a core with the same name already exists, it will be stopped and replaced by this one.
|
||||
*@param dataDir the index directory
|
||||
*@param config a solr config instance
|
||||
*@param schema a solr schema instance
|
||||
* Creates a new core and register it in the list of cores. If a core with the
|
||||
* same name already exists, it will be stopped and replaced by this one.
|
||||
*
|
||||
*@since solr 1.3
|
||||
* @param dataDir
|
||||
* the index directory
|
||||
* @param config
|
||||
* a solr config instance
|
||||
* @param schema
|
||||
* a solr schema instance
|
||||
*
|
||||
* @since solr 1.3
|
||||
*/
|
||||
public SolrCore(String name, String dataDir, SolrConfig config, IndexSchema schema, CoreDescriptor cd, UpdateHandler updateHandler, IndexDeletionPolicyWrapper delPolicy, SolrCore prev) {
|
||||
coreDescriptor = cd;
|
||||
this.setName( name );
|
||||
public SolrCore(String name, String dataDir, SolrConfig config,
|
||||
IndexSchema schema, CoreDescriptor coreDescriptor, UpdateHandler updateHandler,
|
||||
IndexDeletionPolicyWrapper delPolicy, SolrCore prev) {
|
||||
checkNotNull(coreDescriptor, "coreDescriptor cannot be null");
|
||||
|
||||
this.coreDescriptor = coreDescriptor;
|
||||
setName(name);
|
||||
MDCUtils.setCore(name); // show the core name in the error logs
|
||||
|
||||
resourceLoader = config.getResourceLoader();
|
||||
this.solrConfig = config;
|
||||
|
||||
if (updateHandler == null) {
|
||||
initDirectoryFactory();
|
||||
directoryFactory = initDirectoryFactory();
|
||||
solrCoreState = new DefaultSolrCoreState(directoryFactory);
|
||||
} else {
|
||||
solrCoreState = updateHandler.getSolrCoreState();
|
||||
directoryFactory = solrCoreState.getDirectoryFactory();
|
||||
isReloaded = true;
|
||||
}
|
||||
|
||||
if (dataDir == null) {
|
||||
if (cd.usingDefaultDataDir()) dataDir = config.getDataDir();
|
||||
if (dataDir == null) {
|
||||
try {
|
||||
dataDir = cd.getDataDir();
|
||||
if (!directoryFactory.isAbsolute(dataDir)) {
|
||||
dataDir = directoryFactory.getDataHome(cd);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
dataDir = SolrResourceLoader.normalizeDir(dataDir);
|
||||
this.dataDir = initDataDir(dataDir, config, coreDescriptor);
|
||||
this.ulogDir = initUpdateLogDir(coreDescriptor);
|
||||
|
||||
String updateLogDir = cd.getUlogDir();
|
||||
if (updateLogDir == null) {
|
||||
updateLogDir = dataDir;
|
||||
if (new File(updateLogDir).isAbsolute() == false) {
|
||||
updateLogDir = SolrResourceLoader.normalizeDir(cd.getInstanceDir()) + updateLogDir;
|
||||
}
|
||||
}
|
||||
ulogDir = updateLogDir;
|
||||
log.info("[{}] Opening new SolrCore at [{}], dataDir=[{}]", logid, resourceLoader.getInstanceDir(), dataDir);
|
||||
|
||||
checkVersionFieldExistsInSchema(schema, coreDescriptor);
|
||||
|
||||
log.info(logid+"Opening new SolrCore at " + resourceLoader.getInstanceDir() + ", dataDir="+dataDir);
|
||||
|
||||
if (null != cd && null != cd.getCloudDescriptor()) {
|
||||
// we are evidently running in cloud mode.
|
||||
//
|
||||
// In cloud mode, version field is required for correct consistency
|
||||
// ideally this check would be more fine grained, and individual features
|
||||
// would assert it when they initialize, but DistributedUpdateProcessor
|
||||
// is currently a big ball of wax that does more then just distributing
|
||||
// updates (ie: partial document updates), so it needs to work in no cloud
|
||||
// mode as well, and can't assert version field support on init.
|
||||
|
||||
try {
|
||||
VersionInfo.getAndCheckVersionField(schema);
|
||||
} catch (SolrException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Schema will not work with SolrCloud mode: " +
|
||||
e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
//Initialize JMX
|
||||
if (config.jmxConfig.enabled) {
|
||||
infoRegistry = new JmxMonitoredMap<String, SolrInfoMBean>(name, String.valueOf(this.hashCode()), config.jmxConfig);
|
||||
} else {
|
||||
log.info("JMX monitoring not detected for core: " + name);
|
||||
infoRegistry = new ConcurrentHashMap<>();
|
||||
}
|
||||
|
||||
// Initialize JMX
|
||||
this.infoRegistry = initInfoRegistry(name, config);
|
||||
infoRegistry.put("fieldCache", new SolrFieldCacheMBean());
|
||||
|
||||
if (schema==null) {
|
||||
schema = IndexSchemaFactory.buildIndexSchema(IndexSchema.DEFAULT_SCHEMA_FILE, config);
|
||||
}
|
||||
this.schema = schema;
|
||||
final SimilarityFactory similarityFactory = schema.getSimilarityFactory();
|
||||
if (similarityFactory instanceof SolrCoreAware) {
|
||||
// Similarity needs SolrCore before inform() is called on all registered SolrCoreAware listeners below
|
||||
((SolrCoreAware)similarityFactory).inform(this);
|
||||
}
|
||||
this.schema = initSchema(config, schema);
|
||||
|
||||
this.dataDir = dataDir;
|
||||
this.startTime = System.currentTimeMillis();
|
||||
this.maxWarmingSearchers = config.maxWarmingSearchers;
|
||||
this.slowQueryThresholdMillis = config.slowQueryThresholdMillis;
|
||||
|
@ -781,21 +750,10 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
|
||||
initListeners();
|
||||
|
||||
if (delPolicy == null) {
|
||||
initDeletionPolicy();
|
||||
} else {
|
||||
this.solrDelPolicy = delPolicy;
|
||||
}
|
||||
this.solrDelPolicy = initDeletionPolicy(delPolicy);
|
||||
|
||||
this.codec = initCodec(solrConfig, schema);
|
||||
this.codec = initCodec(solrConfig, this.schema);
|
||||
|
||||
if (updateHandler == null) {
|
||||
solrCoreState = new DefaultSolrCoreState(getDirectoryFactory());
|
||||
} else {
|
||||
solrCoreState = updateHandler.getSolrCoreState();
|
||||
directoryFactory = solrCoreState.getDirectoryFactory();
|
||||
this.isReloaded = true;
|
||||
}
|
||||
memClassLoader = new MemClassLoader(PluginBag.RuntimeLib.getLibObjects(this, solrConfig.getPluginInfos(PluginBag.RuntimeLib.class.getName())), getResourceLoader());
|
||||
initIndex(prev != null);
|
||||
|
||||
|
@ -804,7 +762,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
valueSourceParsers.init(ValueSourceParser.standardValueSourceParsers, this);
|
||||
transformerFactories.init(TransformerFactory.defaultFactories, this);
|
||||
loadSearchComponents();
|
||||
updateProcessors.init(Collections.EMPTY_MAP, this);
|
||||
updateProcessors.init(Collections.emptyMap(), this);
|
||||
|
||||
// Processors initialized before the handlers
|
||||
updateProcessorChains = loadUpdateProcessorChains();
|
||||
|
@ -827,42 +785,9 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
}
|
||||
});
|
||||
|
||||
// use the (old) writer to open the first searcher
|
||||
RefCounted<IndexWriter> iwRef = null;
|
||||
if (prev != null) {
|
||||
iwRef = prev.getUpdateHandler().getSolrCoreState().getIndexWriter(null);
|
||||
if (iwRef != null) {
|
||||
final IndexWriter iw = iwRef.get();
|
||||
final SolrCore core = this;
|
||||
newReaderCreator = new Callable<DirectoryReader>() {
|
||||
// this is used during a core reload
|
||||
this.updateHandler = initUpdateHandler(updateHandler);
|
||||
|
||||
@Override
|
||||
public DirectoryReader call() throws Exception {
|
||||
return indexReaderFactory.newReader(iw, core);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
String updateHandlerClass = solrConfig.getUpdateHandlerInfo().className;
|
||||
|
||||
if (updateHandler == null) {
|
||||
this.updateHandler = createUpdateHandler(updateHandlerClass == null ? DirectUpdateHandler2.class
|
||||
.getName() : updateHandlerClass);
|
||||
} else {
|
||||
this.updateHandler = createUpdateHandler(
|
||||
updateHandlerClass == null ? DirectUpdateHandler2.class.getName()
|
||||
: updateHandlerClass, updateHandler);
|
||||
}
|
||||
infoRegistry.put("updateHandler", this.updateHandler);
|
||||
|
||||
try {
|
||||
getSearcher(false, false, null, true);
|
||||
} finally {
|
||||
newReaderCreator = null;
|
||||
if (iwRef != null) iwRef.decref();
|
||||
}
|
||||
initSearcher(prev);
|
||||
|
||||
// Initialize the RestManager
|
||||
restManager = initRestManager();
|
||||
|
@ -871,23 +796,25 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
resourceLoader.inform(resourceLoader);
|
||||
resourceLoader.inform(this); // last call before the latch is released.
|
||||
} catch (Throwable e) {
|
||||
latch.countDown();//release the latch, otherwise we block trying to do the close. This should be fine, since counting down on a latch of 0 is still fine
|
||||
//close down the searcher and any other resources, if it exists, as this is not recoverable
|
||||
// release the latch, otherwise we block trying to do the close. This
|
||||
// should be fine, since counting down on a latch of 0 is still fine
|
||||
latch.countDown();
|
||||
if (e instanceof OutOfMemoryError) {
|
||||
throw (OutOfMemoryError)e;
|
||||
}
|
||||
|
||||
try {
|
||||
this.close();
|
||||
// close down the searcher and any other resources, if it exists, as this
|
||||
// is not recoverable
|
||||
close();
|
||||
} catch (Throwable t) {
|
||||
if (t instanceof OutOfMemoryError) {
|
||||
throw (OutOfMemoryError)t;
|
||||
throw (OutOfMemoryError) t;
|
||||
}
|
||||
log.error("Error while closing", t);
|
||||
}
|
||||
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
e.getMessage(), e);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e.getMessage(), e);
|
||||
} finally {
|
||||
// allow firstSearcher events to fire and make sure it is released
|
||||
latch.countDown();
|
||||
|
@ -903,30 +830,151 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
// from the core.
|
||||
resourceLoader.inform(infoRegistry);
|
||||
|
||||
CoreContainer cc = cd.getCoreContainer();
|
||||
bufferUpdatesIfConstructing(coreDescriptor);
|
||||
|
||||
// For debugging
|
||||
// numOpens.incrementAndGet();
|
||||
// openHandles.put(this, new RuntimeException("unclosed core - name:" + getName() + " refs: " + refCount.get()));
|
||||
|
||||
this.ruleExpiryLock = new ReentrantLock();
|
||||
registerConfListener();
|
||||
}
|
||||
|
||||
/** Set UpdateLog to buffer updates if the slice is in construction. */
|
||||
private void bufferUpdatesIfConstructing(CoreDescriptor coreDescriptor) {
|
||||
final CoreContainer cc = coreDescriptor.getCoreContainer();
|
||||
if (cc != null && cc.isZooKeeperAware()) {
|
||||
SolrRequestHandler realtimeGetHandler = reqHandlers.get("/get");
|
||||
if (realtimeGetHandler == null) {
|
||||
if (reqHandlers.get("/get") == null) {
|
||||
log.warn("WARNING: RealTimeGetHandler is not registered at /get. " +
|
||||
"SolrCloud will always use full index replication instead of the more efficient PeerSync method.");
|
||||
}
|
||||
|
||||
// ZK pre-Register would have already happened so we read slice properties now
|
||||
ClusterState clusterState = cc.getZkController().getClusterState();
|
||||
Slice slice = clusterState.getSlice(cd.getCloudDescriptor().getCollectionName(),
|
||||
cd.getCloudDescriptor().getShardId());
|
||||
// ZK pre-register would have already happened so we read slice properties now
|
||||
final ClusterState clusterState = cc.getZkController().getClusterState();
|
||||
final Slice slice = clusterState.getSlice(coreDescriptor.getCloudDescriptor().getCollectionName(),
|
||||
coreDescriptor.getCloudDescriptor().getShardId());
|
||||
if (slice.getState() == Slice.State.CONSTRUCTION) {
|
||||
// set update log to buffer before publishing the core
|
||||
getUpdateHandler().getUpdateLog().bufferUpdates();
|
||||
}
|
||||
}
|
||||
// For debugging
|
||||
// numOpens.incrementAndGet();
|
||||
// openHandles.put(this, new RuntimeException("unclosed core - name:" + getName() + " refs: " + refCount.get()));
|
||||
}
|
||||
|
||||
ruleExpiryLock = new ReentrantLock();
|
||||
registerConfListener();
|
||||
private void initSearcher(SolrCore prev) throws IOException {
|
||||
// use the (old) writer to open the first searcher
|
||||
RefCounted<IndexWriter> iwRef = null;
|
||||
if (prev != null) {
|
||||
iwRef = prev.getUpdateHandler().getSolrCoreState().getIndexWriter(null);
|
||||
if (iwRef != null) {
|
||||
final IndexWriter iw = iwRef.get();
|
||||
final SolrCore core = this;
|
||||
newReaderCreator = new Callable<DirectoryReader>() {
|
||||
// this is used during a core reload
|
||||
|
||||
@Override
|
||||
public DirectoryReader call() throws Exception {
|
||||
return indexReaderFactory.newReader(iw, core);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
getSearcher(false, false, null, true);
|
||||
} finally {
|
||||
newReaderCreator = null;
|
||||
if (iwRef != null) {
|
||||
iwRef.decref();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private UpdateHandler initUpdateHandler(UpdateHandler updateHandler) {
|
||||
String updateHandlerClass = solrConfig.getUpdateHandlerInfo().className;
|
||||
if (updateHandlerClass == null) {
|
||||
updateHandlerClass = DirectUpdateHandler2.class.getName();
|
||||
}
|
||||
|
||||
final UpdateHandler newUpdateHandler;
|
||||
if (updateHandler == null) {
|
||||
newUpdateHandler = createUpdateHandler(updateHandlerClass);
|
||||
} else {
|
||||
newUpdateHandler = createUpdateHandler(updateHandlerClass, updateHandler);
|
||||
}
|
||||
infoRegistry.put("updateHandler", newUpdateHandler);
|
||||
return newUpdateHandler;
|
||||
}
|
||||
|
||||
private IndexSchema initSchema(SolrConfig config, IndexSchema schema) {
|
||||
if (schema == null) {
|
||||
schema = IndexSchemaFactory.buildIndexSchema(IndexSchema.DEFAULT_SCHEMA_FILE, config);
|
||||
}
|
||||
final SimilarityFactory similarityFactory = schema.getSimilarityFactory();
|
||||
if (similarityFactory instanceof SolrCoreAware) {
|
||||
// Similarity needs SolrCore before inform() is called on all registered SolrCoreAware listeners below
|
||||
((SolrCoreAware) similarityFactory).inform(this);
|
||||
}
|
||||
return schema;
|
||||
}
|
||||
|
||||
private Map<String,SolrInfoMBean> initInfoRegistry(String name, SolrConfig config) {
|
||||
if (config.jmxConfig.enabled) {
|
||||
return new JmxMonitoredMap<String, SolrInfoMBean>(name, String.valueOf(this.hashCode()), config.jmxConfig);
|
||||
} else {
|
||||
log.info("JMX monitoring not detected for core: " + name);
|
||||
return new ConcurrentHashMap<>();
|
||||
}
|
||||
}
|
||||
|
||||
private void checkVersionFieldExistsInSchema(IndexSchema schema, CoreDescriptor coreDescriptor) {
|
||||
if (null != coreDescriptor.getCloudDescriptor()) {
|
||||
// we are evidently running in cloud mode.
|
||||
//
|
||||
// In cloud mode, version field is required for correct consistency
|
||||
// ideally this check would be more fine grained, and individual features
|
||||
// would assert it when they initialize, but DistributedUpdateProcessor
|
||||
// is currently a big ball of wax that does more then just distributing
|
||||
// updates (ie: partial document updates), so it needs to work in no cloud
|
||||
// mode as well, and can't assert version field support on init.
|
||||
|
||||
try {
|
||||
VersionInfo.getAndCheckVersionField(schema);
|
||||
} catch (SolrException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Schema will not work with SolrCloud mode: " +
|
||||
e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private String initDataDir(String dataDir, SolrConfig config, CoreDescriptor coreDescriptor) {
|
||||
if (dataDir == null) {
|
||||
if (coreDescriptor.usingDefaultDataDir()) {
|
||||
dataDir = config.getDataDir();
|
||||
}
|
||||
if (dataDir == null) {
|
||||
try {
|
||||
dataDir = coreDescriptor.getDataDir();
|
||||
if (!directoryFactory.isAbsolute(dataDir)) {
|
||||
dataDir = directoryFactory.getDataHome(coreDescriptor);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
return SolrResourceLoader.normalizeDir(dataDir);
|
||||
}
|
||||
|
||||
private String initUpdateLogDir(CoreDescriptor coreDescriptor) {
|
||||
String updateLogDir = coreDescriptor.getUlogDir();
|
||||
if (updateLogDir == null) {
|
||||
updateLogDir = dataDir;
|
||||
if (new File(updateLogDir).isAbsolute() == false) {
|
||||
updateLogDir = SolrResourceLoader.normalizeDir(coreDescriptor.getInstanceDir()) + updateLogDir;
|
||||
}
|
||||
}
|
||||
return updateLogDir;
|
||||
}
|
||||
|
||||
private Codec initCodec(SolrConfig solrConfig, final IndexSchema schema) {
|
||||
|
@ -1070,6 +1118,7 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
* </ul>
|
||||
* @see #isClosed()
|
||||
*/
|
||||
@Override
|
||||
public void close() {
|
||||
int count = refCount.decrementAndGet();
|
||||
if (count > 0) return; // close is called often, and only actually closes if nothing is using it.
|
||||
|
@ -1080,7 +1129,6 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
}
|
||||
log.info(logid+" CLOSING SolrCore " + this);
|
||||
|
||||
|
||||
if( closeHooks != null ) {
|
||||
for( CloseHook hook : closeHooks ) {
|
||||
try {
|
||||
|
|
|
@ -605,7 +605,7 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
catch (Exception ex) {
|
||||
if (coreContainer.isZooKeeperAware() && dcore != null && !preExisitingZkEntry) {
|
||||
try {
|
||||
coreContainer.getZkController().unregister(dcore.getName(), dcore,null);
|
||||
coreContainer.getZkController().unregister(dcore.getName(), dcore);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
SolrException.log(log, null, e);
|
||||
|
|
|
@ -22,10 +22,12 @@ import java.io.IOException;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.lucene.util.LuceneTestCase.Nightly;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
import org.apache.solr.client.solrj.SolrQuery;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
|
||||
|
@ -33,6 +35,7 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
@Nightly
|
||||
public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
|
||||
|
||||
private MiniSolrCloudCluster solrCluster;
|
||||
|
@ -61,7 +64,8 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
|
|||
final String collectionName = "collection" + i;
|
||||
uploadConfig(configDir, collectionName);
|
||||
final SolrClient solrClient = new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
|
||||
threads[i] = new CreateDeleteCollectionThread("create-delete-" + i, collectionName, timeToRunSec, solrClient, failure);
|
||||
threads[i] = new CreateDeleteSearchCollectionThread("create-delete-search-" + i, collectionName, collectionName,
|
||||
timeToRunSec, solrClient, failure);
|
||||
}
|
||||
|
||||
startAll(threads);
|
||||
|
@ -70,6 +74,33 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
|
|||
assertNull("concurrent create and delete collection failed: " + failure.get(), failure.get());
|
||||
}
|
||||
|
||||
public void testConcurrentCreateAndDeleteOverTheSameConfig() {
|
||||
Logger.getLogger("org.apache.solr").setLevel(Level.WARN);
|
||||
final String configName = "testconfig";
|
||||
final File configDir = getFile("solr").toPath().resolve("configsets/configset-2/conf").toFile();
|
||||
uploadConfig(configDir, configName); // upload config once, to be used by all collections
|
||||
final SolrClient solrClient = new HttpSolrClient(solrCluster.getJettySolrRunners().get(0).getBaseUrl().toString());
|
||||
final AtomicReference<Exception> failure = new AtomicReference<>();
|
||||
final int timeToRunSec = 30;
|
||||
final Thread[] threads = new Thread[2];
|
||||
for (int i = 0; i < threads.length; i++) {
|
||||
final String collectionName = "collection" + i;
|
||||
threads[i] = new CreateDeleteCollectionThread("create-delete-" + i, collectionName, configName,
|
||||
timeToRunSec, solrClient, failure);
|
||||
}
|
||||
|
||||
startAll(threads);
|
||||
joinAll(threads);
|
||||
|
||||
assertNull("concurrent create and delete collection failed: " + failure.get(), failure.get());
|
||||
|
||||
try {
|
||||
solrClient.close();
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private void uploadConfig(File configDir, String configName) {
|
||||
try {
|
||||
solrCluster.uploadConfigDir(configDir, configName);
|
||||
|
@ -96,58 +127,59 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
|
|||
}
|
||||
|
||||
private static class CreateDeleteCollectionThread extends Thread {
|
||||
private final String collectionName;
|
||||
private final long timeToRunSec;
|
||||
private final SolrClient solrClient;
|
||||
private final AtomicReference<Exception> failure;
|
||||
protected final String collectionName;
|
||||
protected final String configName;
|
||||
protected final long timeToRunSec;
|
||||
protected final SolrClient solrClient;
|
||||
protected final AtomicReference<Exception> failure;
|
||||
|
||||
public CreateDeleteCollectionThread(String name, String collectionName,
|
||||
long timeToRunSec, SolrClient solrClient, AtomicReference<Exception> failure) {
|
||||
public CreateDeleteCollectionThread(String name, String collectionName, String configName, long timeToRunSec,
|
||||
SolrClient solrClient, AtomicReference<Exception> failure) {
|
||||
super(name);
|
||||
this.collectionName = collectionName;
|
||||
this.timeToRunSec = timeToRunSec;
|
||||
this.solrClient = solrClient;
|
||||
this.failure = failure;
|
||||
this.configName = configName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
final long timeToStop = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(timeToRunSec);
|
||||
while (System.currentTimeMillis() < timeToStop && failure.get() == null) {
|
||||
createCollection(collectionName);
|
||||
deleteCollection();
|
||||
searchNonExistingCollection();
|
||||
doWork();
|
||||
}
|
||||
}
|
||||
|
||||
private void searchNonExistingCollection() {
|
||||
try {
|
||||
solrClient.query(collectionName, new SolrQuery("*"));
|
||||
} catch (Exception e) {
|
||||
if (!e.getMessage().contains("not found") && !e.getMessage().contains("Can not find")) {
|
||||
synchronized (failure) {
|
||||
if (failure.get() != null) {
|
||||
failure.get().addSuppressed(e);
|
||||
} else {
|
||||
failure.set(e);
|
||||
}
|
||||
}
|
||||
protected void doWork() {
|
||||
createCollection();
|
||||
deleteCollection();
|
||||
}
|
||||
|
||||
protected void addFailure(Exception e) {
|
||||
synchronized (failure) {
|
||||
if (failure.get() != null) {
|
||||
failure.get().addSuppressed(e);
|
||||
} else {
|
||||
failure.set(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void createCollection(String collectionName) {
|
||||
private void createCollection() {
|
||||
try {
|
||||
final CollectionAdminRequest.Create createCollectionRequest = new CollectionAdminRequest.Create();
|
||||
createCollectionRequest.setCollectionName(collectionName);
|
||||
createCollectionRequest.setNumShards(1);
|
||||
createCollectionRequest.setReplicationFactor(1);
|
||||
createCollectionRequest.setConfigName(collectionName);
|
||||
createCollectionRequest.setConfigName(configName);
|
||||
|
||||
final CollectionAdminResponse response = createCollectionRequest.process(solrClient);
|
||||
assertEquals(0, response.getStatus());
|
||||
} catch (IOException | SolrServerException e) {
|
||||
throw new RuntimeException(e);
|
||||
if (response.getStatus() != 0) {
|
||||
addFailure(new RuntimeException("failed to create collection " + collectionName));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
addFailure(e);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -158,11 +190,38 @@ public class ConcurrentDeleteAndCreateCollectionTest extends SolrTestCaseJ4 {
|
|||
deleteCollectionRequest.setCollectionName(collectionName);
|
||||
|
||||
final CollectionAdminResponse response = deleteCollectionRequest.process(solrClient);
|
||||
assertEquals(0, response.getStatus());
|
||||
} catch (IOException | SolrServerException e) {
|
||||
throw new RuntimeException(e);
|
||||
if (response.getStatus() != 0) {
|
||||
addFailure(new RuntimeException("failed to delete collection " + collectionName));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
addFailure(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class CreateDeleteSearchCollectionThread extends CreateDeleteCollectionThread {
|
||||
|
||||
public CreateDeleteSearchCollectionThread(String name, String collectionName, String configName, long timeToRunSec,
|
||||
SolrClient solrClient, AtomicReference<Exception> failure) {
|
||||
super(name, collectionName, configName, timeToRunSec, solrClient, failure);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWork() {
|
||||
super.doWork();
|
||||
searchNonExistingCollection();
|
||||
}
|
||||
|
||||
private void searchNonExistingCollection() {
|
||||
try {
|
||||
solrClient.query(collectionName, new SolrQuery("*"));
|
||||
} catch (Exception e) {
|
||||
if (!e.getMessage().contains("not found") && !e.getMessage().contains("Can not find")) {
|
||||
addFailure(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue