SOLR-5056: Further cleanup of ConfigSolr API

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1506020 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alan Woodward 2013-07-23 12:53:31 +00:00
parent ee0a91c054
commit 30d15988b3
23 changed files with 304 additions and 232 deletions

View File

@ -82,6 +82,8 @@ Other Changes
* SOLR-4914: Factor out core list persistence and discovery into a
new CoresLocator interface. (Alan Woodward)
* SOLR-5056: Improve type safety of ConfigSolr class. (Alan Woodward)
================== 4.4.0 ==================
Versions of Major Components

View File

@ -1,8 +1,5 @@
package org.apache.solr.cloud;
import java.io.IOException;
import java.util.Map;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState;
@ -21,6 +18,9 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Map;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -148,8 +148,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
collection);
Overseer.getInQueue(zkClient).offer(ZkStateReader.toJSON(m));
String leaderVoteWait = cc.getZkController().getLeaderVoteWait();
if (!weAreReplacement && leaderVoteWait != null) {
int leaderVoteWait = cc.getZkController().getLeaderVoteWait();
if (!weAreReplacement) {
waitForReplicasToComeUp(weAreReplacement, leaderVoteWait);
}
@ -309,8 +309,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
}
private void waitForReplicasToComeUp(boolean weAreReplacement,
String leaderVoteWait) throws InterruptedException {
int timeout = Integer.parseInt(leaderVoteWait);
int timeout) throws InterruptedException {
long timeoutAt = System.currentTimeMillis() + timeout;
final String shardsElectZkPath = electionPath + LeaderElector.ELECTION_NODE;

View File

@ -130,7 +130,7 @@ public final class ZkController {
protected volatile Overseer overseer;
private String leaderVoteWait;
private int leaderVoteWait;
private boolean genericCoreNodeNames;
@ -141,7 +141,7 @@ public final class ZkController {
private UpdateShardHandler updateShardHandler;
public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
String localHostContext, String leaderVoteWait, boolean genericCoreNodeNames, int distribUpdateConnTimeout, int distribUpdateSoTimeout, final CurrentCoreDescriptorProvider registerOnReconnect) throws InterruptedException,
String localHostContext, int leaderVoteWait, boolean genericCoreNodeNames, int distribUpdateConnTimeout, int distribUpdateSoTimeout, final CurrentCoreDescriptorProvider registerOnReconnect) throws InterruptedException,
TimeoutException, IOException {
if (cc == null) throw new IllegalArgumentException("CoreContainer cannot be null.");
this.cc = cc;
@ -242,7 +242,7 @@ public final class ZkController {
init(registerOnReconnect);
}
public String getLeaderVoteWait() {
public int getLeaderVoteWait() {
return leaderVoteWait;
}
@ -769,7 +769,7 @@ public final class ZkController {
// in this case, we want to wait for the leader as long as the leader might
// wait for a vote, at least - but also long enough that a large cluster has
// time to get its act together
String leaderUrl = getLeader(cloudDesc, Integer.parseInt(leaderVoteWait) + 600000);
String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
log.info("We are " + ourUrl + " and leader is " + leaderUrl);

View File

@ -21,6 +21,7 @@ import com.google.common.base.Charsets;
import com.google.common.io.ByteStreams;
import org.apache.commons.io.IOUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.logging.LogWatcherConfig;
import org.apache.solr.util.DOMUtil;
import org.apache.solr.util.PropertiesUtil;
import org.slf4j.Logger;
@ -100,20 +101,106 @@ public abstract class ConfigSolr {
public abstract CoresLocator getCoresLocator();
public PluginInfo getShardHandlerFactoryPluginInfo() {
Node node = config.getNode(getShardHandlerFactoryConfigPath(), false);
return (node == null) ? null : new PluginInfo(node, "shardHandlerFactory", false, true);
}
public Node getUnsubsititutedShardHandlerFactoryPluginNode() {
return config.getUnsubstitutedNode(getShardHandlerFactoryConfigPath(), false);
}
protected abstract String getShardHandlerFactoryConfigPath();
public String getZkHost() {
String sysZkHost = System.getProperty("zkHost");
if (sysZkHost != null)
return sysZkHost;
return get(CfgProp.SOLR_ZKHOST, null);
}
public int getZkClientTimeout() {
String sysProp = System.getProperty("zkClientTimeout");
if (sysProp != null)
return Integer.parseInt(sysProp);
return getInt(CfgProp.SOLR_ZKCLIENTTIMEOUT, DEFAULT_ZK_CLIENT_TIMEOUT);
}
private static final int DEFAULT_ZK_CLIENT_TIMEOUT = 15000;
private static final int DEFAULT_LEADER_VOTE_WAIT = 180000; // 3 minutes
private static final int DEFAULT_CORE_LOAD_THREADS = 3;
protected static final String DEFAULT_CORE_ADMIN_PATH = "/admin/cores";
public String getZkHostPort() {
return get(CfgProp.SOLR_HOSTPORT, null);
}
public String getZkHostContext() {
return get(CfgProp.SOLR_HOSTCONTEXT, null);
}
public String getHost() {
return get(CfgProp.SOLR_HOST, null);
}
public int getLeaderVoteWait() {
return getInt(CfgProp.SOLR_LEADERVOTEWAIT, DEFAULT_LEADER_VOTE_WAIT);
}
public boolean getGenericCoreNodeNames() {
return getBool(CfgProp.SOLR_GENERICCORENODENAMES, false);
}
public int getDistributedConnectionTimeout() {
return getInt(CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, 0);
}
public int getDistributedSocketTimeout() {
return getInt(CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, 0);
}
public int getCoreLoadThreadCount() {
return getInt(ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, DEFAULT_CORE_LOAD_THREADS);
}
public String getSharedLibDirectory() {
return get(ConfigSolr.CfgProp.SOLR_SHAREDLIB , null);
}
public String getDefaultCoreName() {
return get(CfgProp.SOLR_CORES_DEFAULT_CORE_NAME, null);
}
public abstract boolean isPersistent();
public String getAdminPath() {
return get(CfgProp.SOLR_ADMINPATH, DEFAULT_CORE_ADMIN_PATH);
}
public String getCoreAdminHandlerClass() {
return get(CfgProp.SOLR_ADMINHANDLER, "org.apache.solr.handler.admin.CoreAdminHandler");
}
public boolean hasSchemaCache() {
return getBool(ConfigSolr.CfgProp.SOLR_SHARESCHEMA, false);
}
public String getManagementPath() {
return get(CfgProp.SOLR_MANAGEMENTPATH, null);
}
public LogWatcherConfig getLogWatcherConfig() {
return new LogWatcherConfig(
getBool(CfgProp.SOLR_LOGGING_ENABLED, false),
get(CfgProp.SOLR_LOGGING_CLASS, null),
get(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, null),
getInt(CfgProp.SOLR_LOGGING_WATCHER_SIZE, 50)
);
}
public int getTransientCacheSize() {
return getInt(CfgProp.SOLR_TRANSIENTCACHESIZE, Integer.MAX_VALUE);
}
// Ugly for now, but we'll at least be able to centralize all of the differences between 4x and 5x.
public static enum CfgProp {
protected static enum CfgProp {
SOLR_ADMINHANDLER,
SOLR_CORELOADTHREADS,
SOLR_COREROOTDIRECTORY,

View File

@ -125,11 +125,26 @@ public class ConfigSolrXml extends ConfigSolr {
propMap.put(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, doSub("solr/logging/watcher/int[@name='threshold']"));
}
@Override
public String getDefaultCoreName() {
return "collection1";
}
@Override
public boolean isPersistent() {
return true;
}
@Override
protected String getShardHandlerFactoryConfigPath() {
return "solr/shardHandlerFactory";
}
@Override
public String getAdminPath() {
return DEFAULT_CORE_ADMIN_PATH;
}
@Override
public CoresLocator getCoresLocator() {
return coresLocator;

View File

@ -110,9 +110,15 @@ public class ConfigSolrXmlOld extends ConfigSolr {
}
}
@Override
public boolean isPersistent() {
return config.getBool("solr/@persistent", false);
}
@Override
public String getDefaultCoreName() {
return get(CfgProp.SOLR_CORES_DEFAULT_CORE_NAME, DEFAULT_DEFAULT_CORE_NAME);
}
private void fillPropMap() {
@ -268,15 +274,17 @@ public class ConfigSolrXmlOld extends ConfigSolr {
return new Properties();
}
public static final String DEFAULT_DEFAULT_CORE_NAME = "collection1";
public static final String DEF_SOLR_XML = "<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n"
+ "<solr persistent=\"false\">\n"
+ " <cores adminPath=\"/admin/cores\" defaultCoreName=\""
+ CoreContainer.DEFAULT_DEFAULT_CORE_NAME
+ DEFAULT_DEFAULT_CORE_NAME
+ "\""
+ " host=\"${host:}\" hostPort=\"${hostPort:}\" hostContext=\"${hostContext:}\" zkClientTimeout=\"${zkClientTimeout:15000}\""
+ ">\n"
+ " <core name=\""
+ CoreContainer.DEFAULT_DEFAULT_CORE_NAME
+ DEFAULT_DEFAULT_CORE_NAME
+ "\" shard=\"${shard:}\" collection=\"${collection:collection1}\" instanceDir=\"collection1\" />\n"
+ " </cores>\n" + "</solr>";

View File

@ -55,10 +55,9 @@ import java.util.concurrent.CompletionService;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import static com.google.common.base.Preconditions.checkNotNull;
@ -67,55 +66,34 @@ import static com.google.common.base.Preconditions.checkNotNull;
*
* @since solr 1.3
*/
public class CoreContainer
{
private static final String LEADER_VOTE_WAIT = "180000"; // 3 minutes
private static final int CORE_LOAD_THREADS = 3;
private static final int DEFAULT_ZK_CLIENT_TIMEOUT = 15000;
public static final String DEFAULT_DEFAULT_CORE_NAME = "collection1";
private static final boolean DEFAULT_SHARE_SCHEMA = false;
protected static Logger log = LoggerFactory.getLogger(CoreContainer.class);
public class CoreContainer {
protected static final Logger log = LoggerFactory.getLogger(CoreContainer.class);
private final SolrCores solrCores = new SolrCores(this);
protected final Map<String,Exception> coreInitFailures =
Collections.synchronizedMap(new LinkedHashMap<String,Exception>());
protected boolean persistent = false;
protected String adminPath = null;
protected volatile String managementPath = null;
protected CoreAdminHandler coreAdminHandler = null;
protected CollectionsHandler collectionsHandler = null;
protected String libDir = null;
private InfoHandler infoHandler;
protected Properties containerProperties;
protected Map<String ,IndexSchema> indexSchemaCache;
protected String adminHandler;
protected boolean shareSchema;
protected Integer zkClientTimeout;
protected String defaultCoreName = null;
protected int distribUpdateConnTimeout = 0;
protected int distribUpdateSoTimeout = 0;
protected ZkContainer zkSys = new ZkContainer();
private ShardHandlerFactory shardHandlerFactory;
protected LogWatcher logging = null;
private String zkHost;
private int transientCacheSize = Integer.MAX_VALUE;
private int coreLoadThreads;
private CloserThread backgroundCloser = null;
protected final ConfigSolr cfg;
protected final SolrResourceLoader loader;
protected final String solrHome;
private InfoHandler infoHandler;
protected final String solrHome;
protected final CoresLocator coresLocator;
@ -202,10 +180,8 @@ public class CoreContainer
log.info("Loading cores into CoreContainer [instanceDir={}]", loader.getInstanceDir());
ThreadPoolExecutor coreLoadExecutor = null;
// add the sharedLib to the shared resource loader before initializing cfg based plugins
libDir = cfg.get(ConfigSolr.CfgProp.SOLR_SHAREDLIB , null);
String libDir = cfg.getSharedLibDirectory();
if (libDir != null) {
File f = FileUtils.resolvePath(new File(solrHome), libDir);
log.info("loading shared library: " + f.getAbsolutePath());
@ -215,79 +191,32 @@ public class CoreContainer
shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader);
solrCores.allocateLazyCores(cfg, loader);
solrCores.allocateLazyCores(cfg.getTransientCacheSize(), loader);
logging = JulWatcher.newRegisteredLogWatcher(cfg, loader);
logging = JulWatcher.newRegisteredLogWatcher(cfg.getLogWatcherConfig(), loader);
if (cfg instanceof ConfigSolrXmlOld) { //TODO: Remove for 5.0
String dcoreName = cfg.get(ConfigSolr.CfgProp.SOLR_CORES_DEFAULT_CORE_NAME, null);
if (dcoreName != null && !dcoreName.isEmpty()) {
defaultCoreName = dcoreName;
}
persistent = cfg.getBool(ConfigSolr.CfgProp.SOLR_PERSISTENT, false);
adminPath = cfg.get(ConfigSolr.CfgProp.SOLR_ADMINPATH, "/admin/cores");
} else {
adminPath = "/admin/cores";
defaultCoreName = DEFAULT_DEFAULT_CORE_NAME;
}
zkHost = cfg.get(ConfigSolr.CfgProp.SOLR_ZKHOST, null);
coreLoadThreads = cfg.getInt(ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, CORE_LOAD_THREADS);
shareSchema = cfg.getBool(ConfigSolr.CfgProp.SOLR_SHARESCHEMA, DEFAULT_SHARE_SCHEMA);
zkClientTimeout = cfg.getInt(ConfigSolr.CfgProp.SOLR_ZKCLIENTTIMEOUT, DEFAULT_ZK_CLIENT_TIMEOUT);
distribUpdateConnTimeout = cfg.getInt(ConfigSolr.CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, 0);
distribUpdateSoTimeout = cfg.getInt(ConfigSolr.CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, 0);
// Note: initZooKeeper will apply hardcoded default if cloud mode
String hostPort = cfg.get(ConfigSolr.CfgProp.SOLR_HOSTPORT, null);
// Note: initZooKeeper will apply hardcoded default if cloud mode
String hostContext = cfg.get(ConfigSolr.CfgProp.SOLR_HOSTCONTEXT, null);
String host = cfg.get(ConfigSolr.CfgProp.SOLR_HOST, null);
String leaderVoteWait = cfg.get(ConfigSolr.CfgProp.SOLR_LEADERVOTEWAIT, LEADER_VOTE_WAIT);
adminHandler = cfg.get(ConfigSolr.CfgProp.SOLR_ADMINHANDLER, null);
managementPath = cfg.get(ConfigSolr.CfgProp.SOLR_MANAGEMENTPATH, null);
transientCacheSize = cfg.getInt(ConfigSolr.CfgProp.SOLR_TRANSIENTCACHESIZE, Integer.MAX_VALUE);
boolean genericCoreNodeNames = cfg.getBool(ConfigSolr.CfgProp.SOLR_GENERICCORENODENAMES, false);
shareSchema = cfg.hasSchemaCache();
if (shareSchema) {
indexSchemaCache = new ConcurrentHashMap<String,IndexSchema>();
}
zkClientTimeout = Integer.parseInt(System.getProperty("zkClientTimeout",
Integer.toString(zkClientTimeout)));
zkSys.initZooKeeper(this, solrHome, zkHost, zkClientTimeout, hostPort, hostContext, host, leaderVoteWait, genericCoreNodeNames, distribUpdateConnTimeout, distribUpdateSoTimeout);
if (isZooKeeperAware() && coreLoadThreads <= 1) {
throw new SolrException(ErrorCode.SERVER_ERROR,
"SolrCloud requires a value of at least 2 in solr.xml for coreLoadThreads");
}
if (adminPath != null) {
if (adminHandler == null) {
coreAdminHandler = new CoreAdminHandler(this);
} else {
coreAdminHandler = this.createMultiCoreHandler(adminHandler);
}
}
zkSys.initZooKeeper(this, solrHome, cfg);
collectionsHandler = new CollectionsHandler(this);
infoHandler = new InfoHandler(this);
coreAdminHandler = createMultiCoreHandler(cfg.getCoreAdminHandlerClass());
containerProperties = cfg.getSolrProperties("solr");
// setup executor to load cores in parallel
coreLoadExecutor = new ThreadPoolExecutor(coreLoadThreads, coreLoadThreads, 1,
TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>(),
ExecutorService coreLoadExecutor = Executors.newFixedThreadPool(cfg.getCoreLoadThreadCount(),
new DefaultSolrThreadFactory("coreLoadExecutor"));
try {
CompletionService<SolrCore> completionService = new ExecutorCompletionService<SolrCore>(
coreLoadExecutor);
Set<Future<SolrCore>> pending = new HashSet<Future<SolrCore>>();
List<CoreDescriptor> cds = coresLocator.discover(this);
@ -341,7 +270,7 @@ public class CoreContainer
SolrException.log(log, null, ex);
}
}
while (pending != null && pending.size() > 0) {
try {
@ -358,7 +287,7 @@ public class CoreContainer
} catch (ExecutionException e) {
SolrException.log(SolrCore.log, "Error loading core", e);
}
} catch (InterruptedException e) {
throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,
"interrupted while loading core", e);
@ -759,7 +688,7 @@ public class CoreContainer
//5.0 remove all checkDefaults?
private String checkDefault(String name) {
return (null == name || name.isEmpty()) ? defaultCoreName : name;
return (null == name || name.isEmpty()) ? getDefaultCoreName() : name;
}
/**
@ -918,46 +847,34 @@ public class CoreContainer
* the default core name, or null if there is no default core name
*/
public String getDefaultCoreName() {
return defaultCoreName;
return cfg.getDefaultCoreName();
}
// all of the following properties aren't synchronized
// but this should be OK since they normally won't be changed rapidly
@Deprecated
public boolean isPersistent() {
return persistent;
}
@Deprecated
public void setPersistent(boolean persistent) {
this.persistent = persistent;
return cfg.isPersistent();
}
public String getAdminPath() {
return adminPath;
return cfg.getAdminPath();
}
public String getManagementPath() {
return managementPath;
}
/**
* Sets the alternate path for multicore handling:
* Gets the alternate path for multicore handling:
* This is used in case there is a registered unnamed core (aka name is "") to
* declare an alternate way of accessing named cores.
* This can also be used in a pseudo single-core environment so admins can prepare
* a new version before swapping.
*/
public void setManagementPath(String path) {
this.managementPath = path;
public String getManagementPath() {
return cfg.getManagementPath();
}
public LogWatcher getLogging() {
return logging;
}
public void setLogging(LogWatcher v) {
logging = v;
}
/**
* Determines whether the core is already loaded or not but does NOT load the core

View File

@ -65,14 +65,13 @@ class SolrCores {
// Trivial helper method for load, note it implements LRU on transient cores. Also note, if
// there is no setting for max size, nothing is done and all cores go in the regular "cores" list
protected void allocateLazyCores(final ConfigSolr cfg, final SolrResourceLoader loader) {
final int transientCacheSize = cfg.getInt(ConfigSolr.CfgProp.SOLR_TRANSIENTCACHESIZE, Integer.MAX_VALUE);
if (transientCacheSize != Integer.MAX_VALUE) {
CoreContainer.log.info("Allocating transient cache for {} transient cores", transientCacheSize);
transientCores = new LinkedHashMap<String, SolrCore>(transientCacheSize, 0.75f, true) {
protected void allocateLazyCores(final int cacheSize, final SolrResourceLoader loader) {
if (cacheSize != Integer.MAX_VALUE) {
CoreContainer.log.info("Allocating transient cache for {} transient cores", cacheSize);
transientCores = new LinkedHashMap<String, SolrCore>(cacheSize, 0.75f, true) {
@Override
protected boolean removeEldestEntry(Map.Entry<String, SolrCore> eldest) {
if (size() > transientCacheSize) {
if (size() > cacheSize) {
synchronized (modifyLock) {
SolrCore coreToClose = eldest.getValue();
logger.info("Closing transient core [{}]", coreToClose.getName());

View File

@ -17,14 +17,6 @@ package org.apache.solr.core;
* limitations under the License.
*/
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeoutException;
import org.apache.solr.cloud.CurrentCoreDescriptorProvider;
import org.apache.solr.cloud.SolrZkServer;
import org.apache.solr.cloud.ZkController;
@ -40,6 +32,14 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.xml.sax.InputSource;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeoutException;
public class ZkContainer {
protected static Logger log = LoggerFactory.getLogger(ZkContainer.class);
@ -49,53 +49,31 @@ public class ZkContainer {
private String hostPort;
private String hostContext;
private String host;
private String leaderVoteWait;
private int leaderVoteWait;
private Boolean genericCoreNodeNames;
private int distribUpdateConnTimeout;
public SolrZkServer getZkServer() {
return zkServer;
}
public int getZkClientTimeout() {
return zkClientTimeout;
}
public String getHostPort() {
return hostPort;
}
public String getHostContext() {
return hostContext;
}
public String getHost() {
return host;
}
public String getLeaderVoteWait() {
return leaderVoteWait;
}
public boolean getGenericCoreNodeNames() {
return genericCoreNodeNames;
}
public int getDistribUpdateConnTimeout() {
return distribUpdateConnTimeout;
}
public int getDistribUpdateSoTimeout() {
return distribUpdateSoTimeout;
}
private int distribUpdateSoTimeout;
public ZkContainer() {
}
public void initZooKeeper(final CoreContainer cc, String solrHome, ConfigSolr config) {
if (config.getCoreLoadThreadCount() <= 1) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"SolrCloud requires a value of at least 2 in solr.xml for coreLoadThreads");
}
initZooKeeper(cc, solrHome,
config.getZkHost(), config.getZkClientTimeout(), config.getZkHostPort(), config.getZkHostContext(),
config.getHost(), config.getLeaderVoteWait(), config.getGenericCoreNodeNames(),
config.getDistributedConnectionTimeout(), config.getDistributedSocketTimeout());
}
public void initZooKeeper(final CoreContainer cc, String solrHome, String zkHost, int zkClientTimeout, String hostPort, String hostContext, String host, String leaderVoteWait, boolean genericCoreNodeNames, int distribUpdateConnTimeout, int distribUpdateSoTimeout) {
public void initZooKeeper(final CoreContainer cc, String solrHome, String zkHost, int zkClientTimeout, String hostPort,
String hostContext, String host, int leaderVoteWait, boolean genericCoreNodeNames,
int distribUpdateConnTimeout, int distribUpdateSoTimeout) {
ZkController zkController = null;
// if zkHost sys property is not set, we are not using ZooKeeper

View File

@ -19,8 +19,18 @@ package org.apache.solr.logging;
*/
public class ListenerConfig {
public int size = 50;
public String threshold = null;
public final int size;
public final String threshold;
public ListenerConfig(int size, String threshold) {
this.size = size;
this.threshold = threshold;
}
public ListenerConfig() {
this(50, null);
}
// Down the line, settings for solr URL/core to store logging
}

View File

@ -19,7 +19,6 @@ package org.apache.solr.logging;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.logging.jul.JulWatcher;
import org.apache.solr.logging.log4j.Log4jWatcher;
@ -119,35 +118,32 @@ public abstract class LogWatcher<E> {
* JUL and Log4j watchers are supported out-of-the-box. You can register your own
* LogWatcher implementation via the plugins architecture
*
* @param config the CoreContainer's config, with logging configuration details
* @param config a LogWatcherConfig object, containing the configuration for this LogWatcher.
* @param loader a SolrResourceLoader, to be used to load plugin LogWatcher implementations.
* Can be null if
*
* @return a LogWatcher configured for the container's logging framework
*/
public static LogWatcher newRegisteredLogWatcher(ConfigSolr config, SolrResourceLoader loader) {
public static LogWatcher newRegisteredLogWatcher(LogWatcherConfig config, SolrResourceLoader loader) {
if (!config.getBool(ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, true))
if (!config.isEnabled())
return null;
LogWatcher logWatcher = createWatcher(config, loader);
if (logWatcher != null) {
ListenerConfig v = new ListenerConfig();
v.size = config.getInt(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_SIZE, 50);
v.threshold = config.get(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, null);
if (v.size > 0) {
if (config.getWatcherSize() > 0) {
log.info("Registering Log Listener [{}]", logWatcher.getName());
logWatcher.registerListener(v);
logWatcher.registerListener(config.asListenerConfig());
}
}
return logWatcher;
}
private static LogWatcher createWatcher(ConfigSolr config, SolrResourceLoader loader) {
private static LogWatcher createWatcher(LogWatcherConfig config, SolrResourceLoader loader) {
String fname = config.get(ConfigSolr.CfgProp.SOLR_LOGGING_CLASS, null);
String fname = config.getLoggingClass();
String slf4jImpl;
try {

View File

@ -0,0 +1,74 @@
package org.apache.solr.logging;
/*
* 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.
*/
/**
* Defines the configuration of a {@link LogWatcher}
*/
public class LogWatcherConfig {
private final boolean enabled;
private final String loggingClass;
private final int watcherSize;
private final String watcherThreshold;
public LogWatcherConfig(boolean enabled, String loggingClass, String watcherThreshold, int watcherSize) {
this.enabled = enabled;
this.loggingClass = loggingClass;
this.watcherThreshold = watcherThreshold;
this.watcherSize = watcherSize;
}
/**
* @return true if the LogWatcher is enabled
*/
public boolean isEnabled() {
return enabled;
}
/**
* Get the implementation of the LogWatcher to use. May be "JUL" or "log4j" for the default
* java.util.logging or log4j implementations, or the fully-qualified name of a class extending
* {@link LogWatcher}.
* @return the LogWatcher class to use
*/
public String getLoggingClass() {
return loggingClass;
}
/**
* @return the size of the LogWatcher queue
*/
public int getWatcherSize() {
return watcherSize;
}
/**
* @return the threshold above which logging events will be recorded
*/
public String getWatcherThreshold() {
return watcherThreshold;
}
/**
* @return a {@link ListenerConfig} object using this config's settings.
*/
public ListenerConfig asListenerConfig() {
return new ListenerConfig(watcherSize, watcherThreshold);
}
}

View File

@ -163,7 +163,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
cc = getCoreContainer();
ZkController zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, 10000,
"127.0.0.1", "8983", "solr", "0", true, 10000, 10000, new CurrentCoreDescriptorProvider() {
"127.0.0.1", "8983", "solr", 0, true, 10000, 10000, new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {
@ -203,7 +203,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
cc = getCoreContainer();
zkController = new ZkController(cc, server.getZkAddress(),
TIMEOUT, 10000, "127.0.0.1", "8983", "solr", "0", true, 10000, 10000, new CurrentCoreDescriptorProvider() {
TIMEOUT, 10000, "127.0.0.1", "8983", "solr", 0, true, 10000, 10000, new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {

View File

@ -81,7 +81,6 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
System.setProperty("shareSchema", "true");
final CoreContainer cores = init("_shareSchema");
try {
cores.setPersistent(false);
assertTrue(cores.isShareSchema());
CoreDescriptor descriptor1 = new CoreDescriptor(cores, "core1", "./collection1");
@ -205,7 +204,6 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
public void testClassLoaderHierarchy() throws Exception {
final CoreContainer cc = init("_classLoaderHierarchy");
try {
cc.setPersistent(false);
ClassLoader sharedLoader = cc.loader.getClassLoader();
ClassLoader contextLoader = Thread.currentThread().getContextClassLoader();
assertSame(contextLoader, sharedLoader.getParent());

View File

@ -136,7 +136,7 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 {
CoreContainer cc = init();
try {
assertEquals(CoreContainer.DEFAULT_DEFAULT_CORE_NAME,
assertEquals(ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME,
cc.getDefaultCoreName());
TestLazyCores.checkInCores(cc, "core1");

View File

@ -572,7 +572,7 @@ public class TestSolrXmlPersistence extends SolrTestCaseJ4 {
public static String SOLR_XML_LOTS_SYSVARS =
"<solr persistent=\"${solr.xml.persist:false}\" coreLoadThreads=\"12\" sharedLib=\"${something:.}\" >\n" +
" <logging class=\"${logclass:log4j.class}\" enabled=\"{logenable:true}\">\n" +
" <watcher size=\"{watchSize:13}\" threshold=\"${logThresh:54}\" />\n" +
" <watcher size=\"${watchSize:13}\" threshold=\"${logThresh:54}\" />\n" +
" </logging>\n" +
" <cores adminPath=\"/admin/cores\" defaultCoreName=\"SystemVars1\" host=\"127.0.0.1\" \n" +
" hostPort=\"${hostPort:8983}\" hostContext=\"${hostContext:solr}\" \n" +

View File

@ -19,7 +19,6 @@ package org.apache.solr.logging;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.core.ConfigSolr;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
@ -27,24 +26,16 @@ import org.slf4j.LoggerFactory;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.replay;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestLogWatcher {
private ConfigSolr config;
private LogWatcherConfig config;
@Before
public void setUp() {
config = createMock(ConfigSolr.class);
expect(config.getBool(ConfigSolr.CfgProp.SOLR_LOGGING_ENABLED, true)).andReturn(true);
expect(config.getInt(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_SIZE, 50)).andReturn(50);
expect(config.get(ConfigSolr.CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, null)).andReturn(null);
expect(config.get(ConfigSolr.CfgProp.SOLR_LOGGING_CLASS, null)).andReturn(null);
replay(config);
config = new LogWatcherConfig(true, null, null, 50);
}
@Test

View File

@ -16,13 +16,6 @@ package org.apache.solr.schema;
* limitations under the License.
*/
import java.io.File;
import java.io.FileInputStream;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.regex.Pattern;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.solr.common.SolrException;
@ -36,6 +29,13 @@ import org.apache.solr.response.SolrQueryResponse;
import org.junit.After;
import org.junit.Before;
import java.io.File;
import java.io.FileInputStream;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.regex.Pattern;
public class TestManagedSchema extends AbstractBadConfigTestBase {
private static File tmpSolrHome;
@ -122,7 +122,6 @@ public class TestManagedSchema extends AbstractBadConfigTestBase {
private void assertSchemaResource(String collection, String expectedSchemaResource) throws Exception {
final CoreContainer cores = h.getCoreContainer();
cores.setPersistent(false);
final CoreAdminHandler admin = new CoreAdminHandler(cores);
SolrQueryRequest request = req(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.STATUS.toString());
SolrQueryResponse response = new SolrQueryResponse();

View File

@ -81,7 +81,6 @@ public abstract class MergeIndexesExampleTestBase extends SolrExampleTestBase {
setupCoreContainer();
SolrCore.log.info("CORES=" + cores + " : " + cores.getCoreNames());
cores.setPersistent(false);
}

View File

@ -68,7 +68,6 @@ public abstract class MultiCoreExampleTestBase extends SolrExampleTestBase
setupCoreContainer();
SolrCore.log.info("CORES=" + cores + " : " + cores.getCoreNames());
cores.setPersistent(false);
}

View File

@ -56,7 +56,6 @@ public class MultiCoreExampleJettyTest extends MultiCoreExampleTestBase {
jetty.start(false);
port = jetty.getLocalPort();
cores.setPersistent(false);
}
@Override public void tearDown() throws Exception

View File

@ -34,6 +34,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.XML;
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.ConfigSolrXmlOld;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrConfig;
@ -93,7 +94,7 @@ import static com.google.common.base.Preconditions.checkNotNull;
QuickPatchThreadsFilter.class
})
public abstract class SolrTestCaseJ4 extends LuceneTestCase {
private static String coreName = CoreContainer.DEFAULT_DEFAULT_CORE_NAME;
private static String coreName = ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME;
public static int DEFAULT_CONNECTION_TIMEOUT = 15000; // default socket connection timeout in ms
@ -128,7 +129,7 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
endTrackingSearchers();
endTrackingZkClients();
resetFactory();
coreName = CoreContainer.DEFAULT_DEFAULT_CORE_NAME;
coreName = ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME;
System.clearProperty("jetty.testMode");
System.clearProperty("tests.shardhandler.randomSeed");
System.clearProperty("enable.update.log");

View File

@ -22,6 +22,7 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.NamedList.NamedListEntry;
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.ConfigSolrXmlOld;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore;
@ -78,10 +79,10 @@ public class TestHarness extends BaseTestHarness {
/**
* Creates a SolrConfig object for the
* {@link CoreContainer#DEFAULT_DEFAULT_CORE_NAME} core using {@link #createConfig(String,String,String)}
* {@link ConfigSolrXmlOld#DEFAULT_DEFAULT_CORE_NAME} core using {@link #createConfig(String,String,String)}
*/
public static SolrConfig createConfig(String solrHome, String confFile) {
return createConfig(solrHome, CoreContainer.DEFAULT_DEFAULT_CORE_NAME, confFile);
return createConfig(solrHome, ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME, confFile);
}
/**
@ -115,7 +116,7 @@ public class TestHarness extends BaseTestHarness {
public TestHarness( String dataDirectory,
SolrConfig solrConfig,
IndexSchema indexSchema) {
this(CoreContainer.DEFAULT_DEFAULT_CORE_NAME, dataDirectory, solrConfig, indexSchema);
this(ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME, dataDirectory, solrConfig, indexSchema);
}
/**
@ -127,7 +128,7 @@ public class TestHarness extends BaseTestHarness {
public TestHarness(String coreName, String dataDir, String solrConfig, String indexSchema) {
try {
if (coreName == null)
coreName = CoreContainer.DEFAULT_DEFAULT_CORE_NAME;
coreName = ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME;
this.coreName = coreName;
SolrResourceLoader loader = new SolrResourceLoader(SolrResourceLoader.locateSolrHome());
@ -174,7 +175,7 @@ public class TestHarness extends BaseTestHarness {
+ "<solr persistent=\"false\">\n"
+ " <logging enabled=\"true\"/>\n"
+ " <cores adminPath=\"/admin/cores\" defaultCoreName=\""
+ CoreContainer.DEFAULT_DEFAULT_CORE_NAME
+ ConfigSolrXmlOld.DEFAULT_DEFAULT_CORE_NAME
+ "\""
+ " host=\"${host:}\" hostPort=\"${hostPort:}\" hostContext=\"${hostContext:}\""
+ " distribUpdateSoTimeout=\"30000\""