diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index a214981eb8e..9390f52f81f 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -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 diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java index 52d22f0f69e..6f11daa29f2 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java +++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java @@ -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; diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java index 5210df4d6aa..65de23bf461 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -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); diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSolr.java b/solr/core/src/java/org/apache/solr/core/ConfigSolr.java index d184a2f9544..3c6cd7abcd1 100644 --- a/solr/core/src/java/org/apache/solr/core/ConfigSolr.java +++ b/solr/core/src/java/org/apache/solr/core/ConfigSolr.java @@ -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, diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java b/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java index efe1fb715ff..961803b647e 100644 --- a/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java +++ b/solr/core/src/java/org/apache/solr/core/ConfigSolrXml.java @@ -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; diff --git a/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java b/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java index 005a6bb4a6f..4a9aadf4d04 100644 --- a/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java +++ b/solr/core/src/java/org/apache/solr/core/ConfigSolrXmlOld.java @@ -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 = "\n" + "\n" + " \n" + " \n" + " \n" + ""; 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 f80bbd03ed1..80cc43510f3 100644 --- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java +++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java @@ -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 coreInitFailures = Collections.synchronizedMap(new LinkedHashMap()); - - 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 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(); } - 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(), + ExecutorService coreLoadExecutor = Executors.newFixedThreadPool(cfg.getCoreLoadThreadCount(), new DefaultSolrThreadFactory("coreLoadExecutor")); + try { CompletionService completionService = new ExecutorCompletionService( coreLoadExecutor); + Set> pending = new HashSet>(); List 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 diff --git a/solr/core/src/java/org/apache/solr/core/SolrCores.java b/solr/core/src/java/org/apache/solr/core/SolrCores.java index ce20f40e5ef..3c7573a7846 100644 --- a/solr/core/src/java/org/apache/solr/core/SolrCores.java +++ b/solr/core/src/java/org/apache/solr/core/SolrCores.java @@ -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(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(cacheSize, 0.75f, true) { @Override protected boolean removeEldestEntry(Map.Entry eldest) { - if (size() > transientCacheSize) { + if (size() > cacheSize) { synchronized (modifyLock) { SolrCore coreToClose = eldest.getValue(); logger.info("Closing transient core [{}]", coreToClose.getName()); diff --git a/solr/core/src/java/org/apache/solr/core/ZkContainer.java b/solr/core/src/java/org/apache/solr/core/ZkContainer.java index c5e8fc5007d..7440dba7c8a 100644 --- a/solr/core/src/java/org/apache/solr/core/ZkContainer.java +++ b/solr/core/src/java/org/apache/solr/core/ZkContainer.java @@ -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 diff --git a/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java b/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java index 486a6d4380f..9a9ad8d00b8 100644 --- a/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java +++ b/solr/core/src/java/org/apache/solr/logging/ListenerConfig.java @@ -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 } diff --git a/solr/core/src/java/org/apache/solr/logging/LogWatcher.java b/solr/core/src/java/org/apache/solr/logging/LogWatcher.java index d2a23bdabc2..23f260ba4ef 100644 --- a/solr/core/src/java/org/apache/solr/logging/LogWatcher.java +++ b/solr/core/src/java/org/apache/solr/logging/LogWatcher.java @@ -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 { * 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 { diff --git a/solr/core/src/java/org/apache/solr/logging/LogWatcherConfig.java b/solr/core/src/java/org/apache/solr/logging/LogWatcherConfig.java new file mode 100644 index 00000000000..91df842be5a --- /dev/null +++ b/solr/core/src/java/org/apache/solr/logging/LogWatcherConfig.java @@ -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); + } +} diff --git a/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java b/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java index 3a31ea4e75a..f0bc70b193b 100644 --- a/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java @@ -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 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 getCurrentDescriptors() { diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java index ad31891ad14..6d3469ce71b 100644 --- a/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java +++ b/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java @@ -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()); diff --git a/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java b/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java index b1558fabbaf..7648b44f072 100644 --- a/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java +++ b/solr/core/src/test/org/apache/solr/core/TestCoreDiscovery.java @@ -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"); diff --git a/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java b/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java index 5a5dbe9fd01..37977c10256 100644 --- a/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java +++ b/solr/core/src/test/org/apache/solr/core/TestSolrXmlPersistence.java @@ -572,7 +572,7 @@ public class TestSolrXmlPersistence extends SolrTestCaseJ4 { public static String SOLR_XML_LOTS_SYSVARS = "\n" + " \n" + - " \n" + + " \n" + " \n" + " \n" + " \n" + "