SOLR-7160: Decouple CoreContainer config from xml representation

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1662232 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alan Woodward 2015-02-25 14:45:51 +00:00
parent dfd94b518a
commit c5507b0c21
32 changed files with 1360 additions and 1091 deletions

View File

@ -17,26 +17,6 @@ package org.apache.solr.cloud;
* the License. * the License.
*/ */
import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_ACTIVE_NODES;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.ListIterator;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.SolrResponse; import org.apache.solr.client.solrj.SolrResponse;
import org.apache.solr.cloud.overseer.ClusterStateMutator; import org.apache.solr.cloud.overseer.ClusterStateMutator;
@ -55,10 +35,10 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams; import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.common.util.IOUtils;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.handler.component.ShardHandler; import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.update.UpdateShardHandler; import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.common.util.IOUtils;
import org.apache.solr.util.stats.Clock; import org.apache.solr.util.stats.Clock;
import org.apache.solr.util.stats.Timer; import org.apache.solr.util.stats.Timer;
import org.apache.solr.util.stats.TimerContext; import org.apache.solr.util.stats.TimerContext;
@ -67,6 +47,26 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.ListIterator;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
/** /**
* Cluster leader. Responsible for processing state updates, node assignments, creating/deleting * Cluster leader. Responsible for processing state updates, node assignments, creating/deleting
* collections, shards, replicas and setting various properties. * collections, shards, replicas and setting various properties.
@ -830,12 +830,12 @@ public class Overseer implements Closeable {
private Stats stats; private Stats stats;
private String id; private String id;
private boolean closed; private boolean closed;
private ConfigSolr config; private CloudConfig config;
// overseer not responsible for closing reader // overseer not responsible for closing reader
public Overseer(ShardHandler shardHandler, public Overseer(ShardHandler shardHandler,
UpdateShardHandler updateShardHandler, String adminPath, UpdateShardHandler updateShardHandler, String adminPath,
final ZkStateReader reader, ZkController zkController, ConfigSolr config) final ZkStateReader reader, ZkController zkController, CloudConfig config)
throws KeeperException, InterruptedException { throws KeeperException, InterruptedException {
this.reader = reader; this.reader = reader;
this.shardHandler = shardHandler; this.shardHandler = shardHandler;

View File

@ -28,7 +28,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica; import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.core.CloudConfig;
import org.apache.solr.update.UpdateShardHandler; import org.apache.solr.update.UpdateShardHandler;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -90,7 +90,7 @@ public class OverseerAutoReplicaFailoverThread implements Runnable, Closeable {
private final int workLoopDelay; private final int workLoopDelay;
private final int waitAfterExpiration; private final int waitAfterExpiration;
public OverseerAutoReplicaFailoverThread(ConfigSolr config, ZkStateReader zkStateReader, public OverseerAutoReplicaFailoverThread(CloudConfig config, ZkStateReader zkStateReader,
UpdateShardHandler updateShardHandler) { UpdateShardHandler updateShardHandler) {
this.zkStateReader = zkStateReader; this.zkStateReader = zkStateReader;

View File

@ -17,19 +17,6 @@ package org.apache.solr.cloud;
* the License. * the License.
*/ */
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import java.util.Properties;
import java.util.Map.Entry;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.zookeeper.server.ServerConfig; import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServerMain; import org.apache.zookeeper.server.ZooKeeperServerMain;
@ -40,6 +27,19 @@ import org.apache.zookeeper.server.quorum.flexible.QuorumHierarchical;
import org.apache.zookeeper.server.quorum.flexible.QuorumMaj; import org.apache.zookeeper.server.quorum.flexible.QuorumMaj;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.nio.charset.StandardCharsets;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
public class SolrZkServer { public class SolrZkServer {
static org.slf4j.Logger log = LoggerFactory.getLogger(SolrZkServer.class); static org.slf4j.Logger log = LoggerFactory.getLogger(SolrZkServer.class);
@ -47,7 +47,7 @@ public class SolrZkServer {
String zkRun; String zkRun;
String zkHost; String zkHost;
String solrPort; int solrPort;
Properties props; Properties props;
SolrZkServerProps zkProps; SolrZkServerProps zkProps;
@ -56,7 +56,7 @@ public class SolrZkServer {
private String dataHome; private String dataHome;
private String confHome; private String confHome;
public SolrZkServer(String zkRun, String zkHost, String dataHome, String confHome, String solrPort) { public SolrZkServer(String zkRun, String zkHost, String dataHome, String confHome, int solrPort) {
this.zkRun = zkRun; this.zkRun = zkRun;
this.zkHost = zkHost; this.zkHost = zkHost;
this.dataHome = dataHome; this.dataHome = dataHome;
@ -81,7 +81,7 @@ public class SolrZkServer {
// TODO: use something based on IP+port??? support ensemble all from same solr home? // TODO: use something based on IP+port??? support ensemble all from same solr home?
zkProps.setDataDir(dataHome); zkProps.setDataDir(dataHome);
zkProps.zkRun = zkRun; zkProps.zkRun = zkRun;
zkProps.solrPort = solrPort; zkProps.solrPort = Integer.toString(solrPort);
} }
try { try {
@ -89,7 +89,7 @@ public class SolrZkServer {
SolrZkServerProps.injectServers(props, zkRun, zkHost); SolrZkServerProps.injectServers(props, zkRun, zkHost);
zkProps.parseProperties(props); zkProps.parseProperties(props);
if (zkProps.getClientPortAddress() == null) { if (zkProps.getClientPortAddress() == null) {
zkProps.setClientPort(Integer.parseInt(solrPort)+1000); zkProps.setClientPort(solrPort + 1000);
} }
} catch (QuorumPeerConfig.ConfigException | IOException e) { } catch (QuorumPeerConfig.ConfigException | IOException e) {
if (zkRun != null) if (zkRun != null)

View File

@ -165,7 +165,7 @@ public class ZkCLI {
SolrZkServer zkServer = null; SolrZkServer zkServer = null;
if (solrPort != null) { if (solrPort != null) {
zkServer = new SolrZkServer("true", null, solrHome + "/zoo_data", zkServer = new SolrZkServer("true", null, solrHome + "/zoo_data",
solrHome, solrPort); solrHome, Integer.parseInt(solrPort));
zkServer.parseConfig(); zkServer.parseConfig();
zkServer.start(); zkServer.start();
} }

View File

@ -47,6 +47,7 @@ import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.URLUtil; import org.apache.solr.common.util.URLUtil;
import org.apache.solr.core.CloseHook; import org.apache.solr.core.CloseHook;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor; import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
@ -177,12 +178,12 @@ public final class ZkController {
private final String zkServerAddress; // example: 127.0.0.1:54062/solr private final String zkServerAddress; // example: 127.0.0.1:54062/solr
private final String localHostPort; // example: 54065 private final int localHostPort; // example: 54065
private final String localHostContext; // example: solr
private final String hostName; // example: 127.0.0.1 private final String hostName; // example: 127.0.0.1
private final String nodeName; // example: 127.0.0.1:54065_solr private final String nodeName; // example: 127.0.0.1:54065_solr
private final String baseURL; // example: http://127.0.0.1:54065/solr private final String baseURL; // example: http://127.0.0.1:54065/solr
private final CloudConfig cloudConfig;
private LeaderElector overseerElector; private LeaderElector overseerElector;
@ -214,33 +215,33 @@ public final class ZkController {
// keeps track of a list of objects that need to know a new ZooKeeper session was created after expiration occurred // keeps track of a list of objects that need to know a new ZooKeeper session was created after expiration occurred
private List<OnReconnect> reconnectListeners = new ArrayList<OnReconnect>(); private List<OnReconnect> reconnectListeners = new ArrayList<OnReconnect>();
public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort, public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientConnectTimeout, CloudConfig cloudConfig, final CurrentCoreDescriptorProvider registerOnReconnect)
String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
throws InterruptedException, TimeoutException, IOException { throws InterruptedException, TimeoutException, IOException {
if (cc == null) throw new IllegalArgumentException("CoreContainer cannot be null."); if (cc == null) throw new IllegalArgumentException("CoreContainer cannot be null.");
this.cc = cc; this.cc = cc;
this.genericCoreNodeNames = genericCoreNodeNames;
this.cloudConfig = cloudConfig;
this.genericCoreNodeNames = cloudConfig.getGenericCoreNodeNames();
// be forgiving and strip this off leading/trailing slashes // be forgiving and strip this off leading/trailing slashes
// this allows us to support users specifying hostContext="/" in // this allows us to support users specifying hostContext="/" in
// solr.xml to indicate the root context, instead of hostContext="" // solr.xml to indicate the root context, instead of hostContext=""
// which means the default of "solr" // which means the default of "solr"
localHostContext = trimLeadingAndTrailingSlashes(localHostContext); String localHostContext = trimLeadingAndTrailingSlashes(cloudConfig.getSolrHostContext());
this.zkServerAddress = zkServerAddress; this.zkServerAddress = zkServerAddress;
this.localHostPort = locaHostPort; this.localHostPort = cloudConfig.getSolrHostPort();
this.localHostContext = localHostContext; this.hostName = normalizeHostName(cloudConfig.getHost());
this.hostName = normalizeHostName(localHost); this.nodeName = generateNodeName(this.hostName, Integer.toString(this.localHostPort), localHostContext);
this.nodeName = generateNodeName(this.hostName,
this.localHostPort,
this.localHostContext);
this.leaderVoteWait = leaderVoteWait; this.leaderVoteWait = cloudConfig.getLeaderVoteWait();
this.leaderConflictResolveWait = leaderConflictResolveWait; this.leaderConflictResolveWait = cloudConfig.getLeaderConflictResolveWait();
this.clientTimeout = zkClientTimeout; this.clientTimeout = cloudConfig.getZkClientTimeout();
DefaultConnectionStrategy strat = new DefaultConnectionStrategy(); DefaultConnectionStrategy strat = new DefaultConnectionStrategy();
String zkACLProviderClass = cc.getConfig().getZkACLProviderClass(); String zkACLProviderClass = cloudConfig.getZkACLProviderClass();
ZkACLProvider zkACLProvider = null; ZkACLProvider zkACLProvider = null;
if (zkACLProviderClass != null && zkACLProviderClass.trim().length() > 0) { if (zkACLProviderClass != null && zkACLProviderClass.trim().length() > 0) {
zkACLProvider = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class); zkACLProvider = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
@ -248,7 +249,7 @@ public final class ZkController {
zkACLProvider = new DefaultZkACLProvider(); zkACLProvider = new DefaultZkACLProvider();
} }
String zkCredentialsProviderClass = cc.getConfig().getZkCredentialsProviderClass(); String zkCredentialsProviderClass = cloudConfig.getZkCredentialsProviderClass();
if (zkCredentialsProviderClass != null && zkCredentialsProviderClass.trim().length() > 0) { if (zkCredentialsProviderClass != null && zkCredentialsProviderClass.trim().length() > 0) {
strat.setZkCredentialsToAddAutomatically(cc.getResourceLoader().newInstance(zkCredentialsProviderClass, ZkCredentialsProvider.class)); strat.setZkCredentialsToAddAutomatically(cc.getResourceLoader().newInstance(zkCredentialsProviderClass, ZkCredentialsProvider.class));
} else { } else {
@ -256,8 +257,7 @@ public final class ZkController {
} }
addOnReconnectListener(getConfigDirListener()); addOnReconnectListener(getConfigDirListener());
zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClient = new SolrZkClient(zkServerAddress, clientTimeout, zkClientConnectTimeout, strat,
zkClientConnectTimeout, strat,
// on reconnect, reload cloud info // on reconnect, reload cloud info
new OnReconnect() { new OnReconnect() {
@ -362,7 +362,7 @@ public final class ZkController {
this.overseerRunningMap = Overseer.getRunningMap(zkClient); this.overseerRunningMap = Overseer.getRunningMap(zkClient);
this.overseerCompletedMap = Overseer.getCompletedMap(zkClient); this.overseerCompletedMap = Overseer.getCompletedMap(zkClient);
this.overseerFailureMap = Overseer.getFailureMap(zkClient); this.overseerFailureMap = Overseer.getFailureMap(zkClient);
cmdExecutor = new ZkCmdExecutor(zkClientTimeout); cmdExecutor = new ZkCmdExecutor(clientTimeout);
leaderElector = new LeaderElector(zkClient); leaderElector = new LeaderElector(zkClient);
zkStateReader = new ZkStateReader(zkClient); zkStateReader = new ZkStateReader(zkClient);
@ -584,7 +584,7 @@ public final class ZkController {
return hostName; return hostName;
} }
public String getHostPort() { public int getHostPort() {
return localHostPort; return localHostPort;
} }
@ -624,7 +624,7 @@ public final class ZkController {
if (!zkRunOnly) { if (!zkRunOnly) {
overseerElector = new LeaderElector(zkClient); overseerElector = new LeaderElector(zkClient);
this.overseer = new Overseer(shardHandler, updateShardHandler, this.overseer = new Overseer(shardHandler, updateShardHandler,
CoreContainer.CORES_HANDLER_PATH, zkStateReader, this, cc.getConfig()); CoreContainer.CORES_HANDLER_PATH, zkStateReader, this, cloudConfig);
ElectionContext context = new OverseerElectionContext(zkClient, ElectionContext context = new OverseerElectionContext(zkClient,
overseer, getNodeName()); overseer, getNodeName());
overseerElector.setup(context); overseerElector.setup(context);

View File

@ -0,0 +1,212 @@
package org.apache.solr.core;
/*
* 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.
*/
import org.apache.solr.common.SolrException;
public class CloudConfig {
private final String zkHost;
private final int zkClientTimeout;
private final int hostPort;
private final String hostName;
private final String hostContext;
private final boolean useGenericCoreNames;
private final int leaderVoteWait;
private final int leaderConflictResolveWait;
private final int autoReplicaFailoverWaitAfterExpiration;
private final int autoReplicaFailoverWorkLoopDelay;
private final int autoReplicaFailoverBadNodeExpiration;
private final String zkCredentialsProviderClass;
private final String zkACLProviderClass;
CloudConfig(String zkHost, int zkClientTimeout, int hostPort, String hostName, String hostContext, boolean useGenericCoreNames, int leaderVoteWait, int leaderConflictResolveWait, int autoReplicaFailoverWaitAfterExpiration, int autoReplicaFailoverWorkLoopDelay, int autoReplicaFailoverBadNodeExpiration, String zkCredentialsProviderClass, String zkACLProviderClass) {
this.zkHost = zkHost;
this.zkClientTimeout = zkClientTimeout;
this.hostPort = hostPort;
this.hostName = hostName;
this.hostContext = hostContext;
this.useGenericCoreNames = useGenericCoreNames;
this.leaderVoteWait = leaderVoteWait;
this.leaderConflictResolveWait = leaderConflictResolveWait;
this.autoReplicaFailoverWaitAfterExpiration = autoReplicaFailoverWaitAfterExpiration;
this.autoReplicaFailoverWorkLoopDelay = autoReplicaFailoverWorkLoopDelay;
this.autoReplicaFailoverBadNodeExpiration = autoReplicaFailoverBadNodeExpiration;
this.zkCredentialsProviderClass = zkCredentialsProviderClass;
this.zkACLProviderClass = zkACLProviderClass;
if (this.hostPort == -1)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "'hostPort' must be configured to run SolrCloud");
if (this.hostContext == null)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "'hostContext' must be configured to run SolrCloud");
}
public String getZkHost() {
return zkHost;
}
public int getZkClientTimeout() {
return zkClientTimeout;
}
public int getSolrHostPort() {
return hostPort;
}
public String getSolrHostContext() {
return hostContext;
}
public String getHost() {
return hostName;
}
public String getZkCredentialsProviderClass() {
return zkCredentialsProviderClass;
}
public String getZkACLProviderClass() {
return zkACLProviderClass;
}
public int getLeaderVoteWait() {
return leaderVoteWait;
}
public int getLeaderConflictResolveWait() {
return leaderConflictResolveWait;
}
public int getAutoReplicaFailoverWaitAfterExpiration() {
return autoReplicaFailoverWaitAfterExpiration;
}
public int getAutoReplicaFailoverWorkLoopDelay() {
return autoReplicaFailoverWorkLoopDelay;
}
public int getAutoReplicaFailoverBadNodeExpiration() {
return autoReplicaFailoverBadNodeExpiration;
}
public boolean getGenericCoreNodeNames() {
return useGenericCoreNames;
}
public static class CloudConfigBuilder {
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_LEADER_CONFLICT_RESOLVE_WAIT = 180000;
// TODO: tune defaults
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION = 30000;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WORKLOOP_DELAY = 10000;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_BAD_NODE_EXPIRATION = 60000;
private String zkHost = System.getProperty("zkHost");
private int zkClientTimeout = Integer.getInteger("zkClientTimeout", DEFAULT_ZK_CLIENT_TIMEOUT);
private final int hostPort;
private final String hostName;
private final String hostContext;
private boolean useGenericCoreNames;
private int leaderVoteWait = DEFAULT_LEADER_VOTE_WAIT;
private int leaderConflictResolveWait = DEFAULT_LEADER_CONFLICT_RESOLVE_WAIT;
private int autoReplicaFailoverWaitAfterExpiration = DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION;
private int autoReplicaFailoverWorkLoopDelay = DEFAULT_AUTO_REPLICA_FAILOVER_WORKLOOP_DELAY;
private int autoReplicaFailoverBadNodeExpiration = DEFAULT_AUTO_REPLICA_FAILOVER_BAD_NODE_EXPIRATION;
private String zkCredentialsProviderClass;
private String zkACLProviderClass;
public CloudConfigBuilder(String hostName, int hostPort) {
this(hostName, hostPort, null);
}
public CloudConfigBuilder(String hostName, int hostPort, String hostContext) {
this.hostName = hostName;
this.hostPort = hostPort;
this.hostContext = hostContext;
}
public CloudConfigBuilder setZkHost(String zkHost) {
this.zkHost = zkHost;
return this;
}
public CloudConfigBuilder setZkClientTimeout(int zkClientTimeout) {
this.zkClientTimeout = zkClientTimeout;
return this;
}
public CloudConfigBuilder setUseGenericCoreNames(boolean useGenericCoreNames) {
this.useGenericCoreNames = useGenericCoreNames;
return this;
}
public CloudConfigBuilder setLeaderVoteWait(int leaderVoteWait) {
this.leaderVoteWait = leaderVoteWait;
return this;
}
public CloudConfigBuilder setLeaderConflictResolveWait(int leaderConflictResolveWait) {
this.leaderConflictResolveWait = leaderConflictResolveWait;
return this;
}
public CloudConfigBuilder setAutoReplicaFailoverWaitAfterExpiration(int autoReplicaFailoverWaitAfterExpiration) {
this.autoReplicaFailoverWaitAfterExpiration = autoReplicaFailoverWaitAfterExpiration;
return this;
}
public CloudConfigBuilder setAutoReplicaFailoverWorkLoopDelay(int autoReplicaFailoverWorkLoopDelay) {
this.autoReplicaFailoverWorkLoopDelay = autoReplicaFailoverWorkLoopDelay;
return this;
}
public CloudConfigBuilder setAutoReplicaFailoverBadNodeExpiration(int autoReplicaFailoverBadNodeExpiration) {
this.autoReplicaFailoverBadNodeExpiration = autoReplicaFailoverBadNodeExpiration;
return this;
}
public CloudConfigBuilder setZkCredentialsProviderClass(String zkCredentialsProviderClass) {
this.zkCredentialsProviderClass = zkCredentialsProviderClass;
return this;
}
public CloudConfigBuilder setZkACLProviderClass(String zkACLProviderClass) {
this.zkACLProviderClass = zkACLProviderClass;
return this;
}
public CloudConfig build() {
return new CloudConfig(zkHost, zkClientTimeout, hostPort, hostName, hostContext, useGenericCoreNames, leaderVoteWait, leaderConflictResolveWait, autoReplicaFailoverWaitAfterExpiration, autoReplicaFailoverWorkLoopDelay, autoReplicaFailoverBadNodeExpiration, zkCredentialsProviderClass, zkACLProviderClass);
}
}
}

View File

@ -19,6 +19,8 @@ package org.apache.solr.core;
import com.google.common.cache.Cache; import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheBuilder;
import org.apache.solr.cloud.CloudConfigSetService;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.IndexSchemaFactory; import org.apache.solr.schema.IndexSchemaFactory;
@ -38,6 +40,16 @@ import java.util.concurrent.ExecutionException;
*/ */
public abstract class ConfigSetService { public abstract class ConfigSetService {
public static ConfigSetService createConfigSetService(NodeConfig nodeConfig, SolrResourceLoader loader, ZkController zkController) {
if (zkController != null)
return new CloudConfigSetService(loader, zkController);
if (nodeConfig.hasSchemaCache())
return new SchemaCaching(loader, nodeConfig.getConfigSetBaseDirectory());
return new Default(loader, nodeConfig.getConfigSetBaseDirectory());
}
protected final SolrResourceLoader parentLoader; protected final SolrResourceLoader parentLoader;
/** /**

View File

@ -1,320 +0,0 @@
package org.apache.solr.core;
/*
* 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.
*/
import org.apache.commons.io.IOUtils;
import org.apache.solr.cloud.CloudConfigSetService;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.logging.LogWatcherConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.xml.sax.InputSource;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Properties;
public abstract class ConfigSolr {
protected static Logger log = LoggerFactory.getLogger(ConfigSolr.class);
public final static String SOLR_XML_FILE = "solr.xml";
public static ConfigSolr fromFile(SolrResourceLoader loader, File configFile) {
log.info("Loading container configuration from {}", configFile.getAbsolutePath());
if (!configFile.exists()) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"solr.xml does not exist in " + configFile.getAbsolutePath() + " cannot start Solr");
}
try (InputStream inputStream = new FileInputStream(configFile)) {
return fromInputStream(loader, inputStream);
} catch (SolrException exc) {
throw exc;
} catch (Exception exc) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Could not load SOLR configuration", exc);
}
}
public static ConfigSolr fromString(SolrResourceLoader loader, String xml) {
return fromInputStream(loader, new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
}
public static ConfigSolr fromInputStream(SolrResourceLoader loader, InputStream is) {
try {
byte[] buf = IOUtils.toByteArray(is);
try (ByteArrayInputStream dup = new ByteArrayInputStream(buf)) {
Config config = new Config(loader, null, new InputSource(dup), null, false);
return new ConfigSolrXml(config);
}
} catch (SolrException exc) {
throw exc;
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public static ConfigSolr fromSolrHome(SolrResourceLoader loader, String solrHome) {
return fromFile(loader, new File(solrHome, SOLR_XML_FILE));
}
public abstract CoresLocator getCoresLocator();
/**
* The directory against which relative core instance dirs are resolved. If none is
* specified in the config, uses solr home.
*
* @return core root directory
*/
public String getCoreRootDirectory() {
String relativeDir = getString(CfgProp.SOLR_COREROOTDIRECTORY, null);
if (relativeDir != null)
return loader.resolve(relativeDir);
return loader.getInstanceDir();
}
public abstract PluginInfo getShardHandlerFactoryPluginInfo();
public String getZkHost() {
String sysZkHost = System.getProperty("zkHost");
if (sysZkHost != null)
return sysZkHost;
return getString(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_LEADER_CONFLICT_RESOLVE_WAIT = 180000;
private static final int DEFAULT_CORE_LOAD_THREADS = 3;
// TODO: tune defaults
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION = 30000;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_WORKLOOP_DELAY = 10000;
private static final int DEFAULT_AUTO_REPLICA_FAILOVER_BAD_NODE_EXPIRATION = 60000;
public static final int DEFAULT_DISTRIBUPDATECONNTIMEOUT = 60000;
public static final int DEFAULT_DISTRIBUPDATESOTIMEOUT = 600000;
protected static final String DEFAULT_CORE_ADMIN_PATH = "/admin/cores";
public String getSolrHostPort() {
return getString(CfgProp.SOLR_HOSTPORT, null);
}
public String getZkHostContext() {
return getString(CfgProp.SOLR_HOSTCONTEXT, null);
}
public String getHost() {
return getString(CfgProp.SOLR_HOST, null);
}
public int getLeaderVoteWait() {
return getInt(CfgProp.SOLR_LEADERVOTEWAIT, DEFAULT_LEADER_VOTE_WAIT);
}
public int getLeaderConflictResolveWait() {
return getInt(CfgProp.SOLR_LEADERCONFLICTRESOLVEWAIT, DEFAULT_LEADER_CONFLICT_RESOLVE_WAIT);
}
public int getAutoReplicaFailoverWaitAfterExpiration() {
return getInt(CfgProp.SOLR_AUTOREPLICAFAILOVERWAITAFTEREXPIRATION, DEFAULT_AUTO_REPLICA_FAILOVER_WAIT_AFTER_EXPIRATION);
}
public int getAutoReplicaFailoverWorkLoopDelay() {
return getInt(CfgProp.SOLR_AUTOREPLICAFAILOVERWORKLOOPDELAY, DEFAULT_AUTO_REPLICA_FAILOVER_WORKLOOP_DELAY);
}
public int getAutoReplicaFailoverBadNodeExpiration() {
return getInt(CfgProp.SOLR_AUTOREPLICAFAILOVERBADNODEEXPIRATION, DEFAULT_AUTO_REPLICA_FAILOVER_BAD_NODE_EXPIRATION);
}
public boolean getGenericCoreNodeNames() {
return getBoolean(CfgProp.SOLR_GENERICCORENODENAMES, false);
}
public int getDistributedConnectionTimeout() {
return getInt(CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT);
}
public int getDistributedSocketTimeout() {
return getInt(CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATESOTIMEOUT);
}
public int getMaxUpdateConnections() {
return getInt(CfgProp.SOLR_MAXUPDATECONNECTIONS, 10000);
}
public int getMaxUpdateConnectionsPerHost() {
return getInt(CfgProp.SOLR_MAXUPDATECONNECTIONSPERHOST, 100);
}
public int getCoreLoadThreadCount() {
return getInt(ConfigSolr.CfgProp.SOLR_CORELOADTHREADS, DEFAULT_CORE_LOAD_THREADS);
}
public String getSharedLibDirectory() {
return getString(ConfigSolr.CfgProp.SOLR_SHAREDLIB, null);
}
public String getCoreAdminHandlerClass() {
return getString(CfgProp.SOLR_ADMINHANDLER, "org.apache.solr.handler.admin.CoreAdminHandler");
}
public String getZkCredentialsProviderClass() {
return getString(CfgProp.SOLR_ZKCREDENTIALSPROVIDER, null);
}
public String getZkACLProviderClass() {
return getString(CfgProp.SOLR_ZKACLPROVIDER, null);
}
public String getCollectionsHandlerClass() {
return getString(CfgProp.SOLR_COLLECTIONSHANDLER, "org.apache.solr.handler.admin.CollectionsHandler");
}
public String getInfoHandlerClass() {
return getString(CfgProp.SOLR_INFOHANDLER, "org.apache.solr.handler.admin.InfoHandler");
}
public boolean hasSchemaCache() {
return getBoolean(ConfigSolr.CfgProp.SOLR_SHARESCHEMA, false);
}
public String getManagementPath() {
return getString(CfgProp.SOLR_MANAGEMENTPATH, null);
}
public String getConfigSetBaseDirectory() {
return getString(CfgProp.SOLR_CONFIGSETBASEDIR, "configsets");
}
public LogWatcherConfig getLogWatcherConfig() {
String loggingClass = getString(CfgProp.SOLR_LOGGING_CLASS, null);
String loggingWatcherThreshold = getString(CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, null);
return new LogWatcherConfig(
getBoolean(CfgProp.SOLR_LOGGING_ENABLED, true),
loggingClass,
loggingWatcherThreshold,
getInt(CfgProp.SOLR_LOGGING_WATCHER_SIZE, 50)
);
}
public int getTransientCacheSize() {
return getInt(CfgProp.SOLR_TRANSIENTCACHESIZE, Integer.MAX_VALUE);
}
public ConfigSetService createCoreConfigService(SolrResourceLoader loader, ZkController zkController) {
if (getZkHost() != null || System.getProperty("zkRun") != null)
return new CloudConfigSetService(loader, zkController);
if (hasSchemaCache())
return new ConfigSetService.SchemaCaching(loader, getConfigSetBaseDirectory());
return new ConfigSetService.Default(loader, getConfigSetBaseDirectory());
}
// Ugly for now, but we'll at least be able to centralize all of the differences between 4x and 5x.
public static enum CfgProp {
SOLR_ADMINHANDLER,
SOLR_COLLECTIONSHANDLER,
SOLR_CORELOADTHREADS,
SOLR_COREROOTDIRECTORY,
SOLR_DISTRIBUPDATECONNTIMEOUT,
SOLR_DISTRIBUPDATESOTIMEOUT,
SOLR_MAXUPDATECONNECTIONS,
SOLR_MAXUPDATECONNECTIONSPERHOST,
SOLR_HOST,
SOLR_HOSTCONTEXT,
SOLR_HOSTPORT,
SOLR_INFOHANDLER,
SOLR_LEADERVOTEWAIT,
SOLR_LOGGING_CLASS,
SOLR_LOGGING_ENABLED,
SOLR_LOGGING_WATCHER_SIZE,
SOLR_LOGGING_WATCHER_THRESHOLD,
SOLR_MANAGEMENTPATH,
SOLR_SHAREDLIB,
SOLR_SHARESCHEMA,
SOLR_TRANSIENTCACHESIZE,
SOLR_GENERICCORENODENAMES,
SOLR_ZKCLIENTTIMEOUT,
SOLR_ZKHOST,
SOLR_LEADERCONFLICTRESOLVEWAIT,
SOLR_CONFIGSETBASEDIR,
SOLR_AUTOREPLICAFAILOVERWAITAFTEREXPIRATION,
SOLR_AUTOREPLICAFAILOVERWORKLOOPDELAY,
SOLR_AUTOREPLICAFAILOVERBADNODEEXPIRATION,
SOLR_ZKCREDENTIALSPROVIDER,
SOLR_ZKACLPROVIDER,
}
protected final SolrResourceLoader loader;
protected final Properties solrProperties;
public ConfigSolr(SolrResourceLoader loader, Properties solrProperties) {
this.loader = loader;
this.solrProperties = solrProperties;
}
public ConfigSolr(SolrResourceLoader loader) {
this(loader, new Properties());
}
protected abstract String getProperty(CfgProp key);
private String getString(CfgProp key, String defaultValue) {
String v = getProperty(key);
return v == null ? defaultValue : v;
}
private int getInt(CfgProp key, int defaultValue) {
String v = getProperty(key);
return v == null ? defaultValue : Integer.parseInt(v);
}
private boolean getBoolean(CfgProp key, boolean defaultValue) {
String v = getProperty(key);
return v == null ? defaultValue : Boolean.parseBoolean(v);
}
public Properties getSolrProperties() {
return solrProperties;
}
public SolrResourceLoader getSolrResourceLoader() {
return loader;
}
}

View File

@ -1,294 +0,0 @@
package org.apache.solr.core;
/*
* 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.
*/
import com.google.common.base.Function;
import com.google.common.base.Functions;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.util.DOMUtil;
import org.apache.solr.util.PropertiesUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import javax.xml.xpath.XPath;
import javax.xml.xpath.XPathConstants;
import javax.xml.xpath.XPathExpressionException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Properties;
/**
*
*/
public class ConfigSolrXml extends ConfigSolr {
protected static Logger log = LoggerFactory.getLogger(ConfigSolrXml.class);
private final CoresLocator coresLocator;
private final Config config;
private final Map<CfgProp, Object> propMap = new HashMap<>();
public ConfigSolrXml(Config config) {
super(config.getResourceLoader(), loadProperties(config));
this.config = config;
this.config.substituteProperties();
checkForIllegalConfig();
fillPropMap();
coresLocator = new CorePropertiesLocator(getCoreRootDirectory());
}
private void checkForIllegalConfig() {
failIfFound("solr/@coreLoadThreads");
failIfFound("solr/@persistent");
failIfFound("solr/@sharedLib");
failIfFound("solr/@zkHost");
failIfFound("solr/cores");
failIfFound("solr/@persistent");
}
private void failIfFound(String xPath) {
if (config.getVal(xPath, false) != null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Should not have found " + xPath +
"\n. Please upgrade your solr.xml: https://cwiki.apache.org/confluence/display/solr/Format+of+solr.xml");
}
}
protected String getProperty(CfgProp key) {
if (!propMap.containsKey(key) || propMap.get(key) == null)
return null;
return propMap.get(key).toString();
}
private static Properties loadProperties(Config config) {
try {
Node node = ((NodeList) config.evaluate("solr", XPathConstants.NODESET)).item(0);
XPath xpath = config.getXPath();
NodeList props = (NodeList) xpath.evaluate("property", node, XPathConstants.NODESET);
Properties properties = new Properties();
for (int i = 0; i < props.getLength(); i++) {
Node prop = props.item(i);
properties.setProperty(DOMUtil.getAttr(prop, "name"),
PropertiesUtil.substituteProperty(DOMUtil.getAttr(prop, "value"), null));
}
return properties;
}
catch (XPathExpressionException e) {
log.warn("Error parsing solr.xml: " + e.getMessage());
return null;
}
}
private NamedList<Object> readNodeListAsNamedList(String path) {
NodeList nodes = config.getNodeList(path, false);
if (nodes != null) {
NamedList<Object> namedList = DOMUtil.nodesToNamedList(nodes);
return namedList;
}
return new NamedList<>();
}
private void fillPropMap() {
NamedList<Object> unknownConfigParams = new NamedList<>();
// shardHandlerFactory is parsed differently in the base class as a plugin, so we're excluding this node from the node list
fillSolrSection(readNodeListAsNamedList("solr/*[@name][not(name()='shardHandlerFactory')]"));
thereCanBeOnlyOne("solr/solrcloud","<solrcloud>");
fillSolrCloudSection(readNodeListAsNamedList("solr/solrcloud/*[@name]"));
thereCanBeOnlyOne("solr/logging","<logging>");
thereCanBeOnlyOne("solr/logging/watcher","Logging <watcher>");
fillLoggingSection(readNodeListAsNamedList("solr/logging/*[@name]"),
readNodeListAsNamedList("solr/logging/watcher/*[@name]"));
}
private void fillSolrSection(NamedList<Object> nl) {
String s = "Main";
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_ADMINHANDLER, "adminHandler");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_COLLECTIONSHANDLER, "collectionsHandler");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_INFOHANDLER, "infoHandler");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_COREROOTDIRECTORY, "coreRootDirectory");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_MANAGEMENTPATH, "managementPath");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_SHAREDLIB, "sharedLib");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_CONFIGSETBASEDIR, "configSetBaseDir");
storeConfigPropertyAsBoolean(s, nl, CfgProp.SOLR_SHARESCHEMA, "shareSchema");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_CORELOADTHREADS, "coreLoadThreads");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_TRANSIENTCACHESIZE, "transientCacheSize");
errorOnLeftOvers(s, nl);
}
private void fillSolrCloudSection(NamedList<Object> nl) {
String s = "<solrcloud>";
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_DISTRIBUPDATECONNTIMEOUT, "distribUpdateConnTimeout");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_DISTRIBUPDATESOTIMEOUT, "distribUpdateSoTimeout");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_MAXUPDATECONNECTIONS, "maxUpdateConnections");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_MAXUPDATECONNECTIONSPERHOST, "maxUpdateConnectionsPerHost");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_LEADERVOTEWAIT, "leaderVoteWait");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_LEADERCONFLICTRESOLVEWAIT, "leaderConflictResolveWait");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_ZKCLIENTTIMEOUT, "zkClientTimeout");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_AUTOREPLICAFAILOVERBADNODEEXPIRATION, "autoReplicaFailoverBadNodeExpiration");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_AUTOREPLICAFAILOVERWAITAFTEREXPIRATION, "autoReplicaFailoverWaitAfterExpiration");
storeConfigPropertyAsInt(s, nl, CfgProp.SOLR_AUTOREPLICAFAILOVERWORKLOOPDELAY, "autoReplicaFailoverWorkLoopDelay");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_HOST, "host");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_HOSTCONTEXT, "hostContext");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_HOSTPORT, "hostPort");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_ZKHOST, "zkHost");
storeConfigPropertyAsBoolean(s, nl, CfgProp.SOLR_GENERICCORENODENAMES, "genericCoreNodeNames");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_ZKACLPROVIDER, "zkACLProvider");
storeConfigPropertyAsString(s, nl, CfgProp.SOLR_ZKCREDENTIALSPROVIDER, "zkCredentialsProvider");
errorOnLeftOvers(s, nl);
}
private void fillLoggingSection(NamedList<Object> loggingConfig,
NamedList<Object> loggingWatcherConfig) {
String s = "<logging>";
storeConfigPropertyAsString(s, loggingConfig, CfgProp.SOLR_LOGGING_CLASS, "class");
storeConfigPropertyAsBoolean(s, loggingConfig, CfgProp.SOLR_LOGGING_ENABLED, "enabled");
errorOnLeftOvers(s, loggingConfig);
s = "Logging <watcher>";
storeConfigPropertyAsInt(s, loggingWatcherConfig, CfgProp.SOLR_LOGGING_WATCHER_SIZE, "size");
storeConfigPropertyAsString(s, loggingWatcherConfig, CfgProp.SOLR_LOGGING_WATCHER_THRESHOLD, "threshold");
errorOnLeftOvers(s, loggingWatcherConfig);
}
private <T> void storeConfigProperty(String section, NamedList<Object> config, CfgProp propertyKey, String name, Function<Object, T> valueTransformer, Class<T> clazz) {
List<Object> values = config.removeAll(name);
if (null != values && 0 != values.size()) {
if (1 < values.size()) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
String.format(Locale.ROOT,
"%s section of solr.xml contains duplicated '%s'"+
" in solr.xml: %s", section, name, values));
} else {
Object value = values.get(0);
if (value != null) {
if (value.getClass().isAssignableFrom(clazz)) {
propMap.put(propertyKey, value);
} else {
propMap.put(propertyKey, valueTransformer.apply(value));
}
} else {
propMap.put(propertyKey, null);
}
}
}
}
private void storeConfigPropertyAsString(String section, NamedList<Object> config, CfgProp propertyKey, String name) {
storeConfigProperty(section, config, propertyKey, name, Functions.toStringFunction(), String.class);
}
private void storeConfigPropertyAsInt(String section, NamedList<Object> config, CfgProp propertyKey, String xmlElementName) {
storeConfigProperty(section, config, propertyKey, xmlElementName, TO_INT_FUNCTION, Integer.class);
}
private void storeConfigPropertyAsBoolean(String section, NamedList<Object> config, CfgProp propertyKey, String name) {
storeConfigProperty(section, config, propertyKey, name, TO_BOOLEAN_FUNCTION, Boolean.class);
}
/** throws an error if more then one element matching the xpath */
private void thereCanBeOnlyOne(String xpath, String section) {
NodeList lst = config.getNodeList(xpath, false);
if (1 < lst.getLength())
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
lst.getLength() + " instances of " + section + " found in solr.xml");
}
/** logs each item in leftovers and then throws an exception with a summary */
private void errorOnLeftOvers(String section, NamedList<Object> leftovers) {
if (null == leftovers || 0 == leftovers.size()) return;
List<String> unknownElements = new ArrayList<String>(leftovers.size());
for (Map.Entry<String, Object> unknownElement : leftovers) {
log.error("Unknown config parameter in {} section of solr.xml: {} -> {}",
section, unknownElement.getKey(), unknownElement.getValue());
unknownElements.add(unknownElement.getKey());
}
if (! unknownElements.isEmpty() ) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
String.format(Locale.ROOT, "%s section of solr.xml contains %d unknown config parameter(s): %s", section, unknownElements.size(), unknownElements));
}
}
public PluginInfo getShardHandlerFactoryPluginInfo() {
Node node = config.getNode("solr/shardHandlerFactory", false);
return (node == null) ? null : new PluginInfo(node, "shardHandlerFactory", false, true);
}
@Override
public CoresLocator getCoresLocator() {
return coresLocator;
}
private static final Function<Map.Entry<String, Object>, String> GET_KEY_FUNCTION = new Function<Map.Entry<String, Object>, String>() {
@Override
public String apply(Map.Entry<String, Object> input) {
return input.getKey();
}
};
private static final Function<Object, Integer> TO_INT_FUNCTION = new Function<Object, Integer>() {
@Override
public Integer apply(Object input) {
try {
return Integer.parseInt(String.valueOf(input));
} catch (NumberFormatException exc) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
String.format(Locale.ROOT,
"Value of '%s' can not be parsed as 'int'", input));
}
}
};
private static final Function<Object, Boolean> TO_BOOLEAN_FUNCTION = new Function<Object, Boolean>() {
@Override
public Boolean apply(Object input) {
if (input instanceof String) {
return Boolean.valueOf(String.valueOf(input));
}
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, String.format(Locale.ROOT, "Value of '%s' can not be parsed as 'bool'", input));
}
};
}

View File

@ -93,7 +93,7 @@ public class CoreContainer {
protected LogWatcher logging = null; protected LogWatcher logging = null;
private CloserThread backgroundCloser = null; private CloserThread backgroundCloser = null;
protected final ConfigSolr cfg; protected final NodeConfig cfg;
protected final SolrResourceLoader loader; protected final SolrResourceLoader loader;
protected final String solrHome; protected final String solrHome;
@ -140,7 +140,7 @@ public class CoreContainer {
* @see #load() * @see #load()
*/ */
public CoreContainer(SolrResourceLoader loader) { public CoreContainer(SolrResourceLoader loader) {
this(ConfigSolr.fromSolrHome(loader, loader.getInstanceDir())); this(SolrXmlConfig.fromSolrHome(loader, loader.getInstanceDir()));
} }
/** /**
@ -160,11 +160,11 @@ public class CoreContainer {
* @param config a ConfigSolr representation of this container's configuration * @param config a ConfigSolr representation of this container's configuration
* @see #load() * @see #load()
*/ */
public CoreContainer(ConfigSolr config) { public CoreContainer(NodeConfig config) {
this(config, config.getCoresLocator()); this(config, new CorePropertiesLocator(config.getCoreRootDirectory()));
} }
public CoreContainer(ConfigSolr config, CoresLocator locator) { public CoreContainer(NodeConfig config, CoresLocator locator) {
this.loader = config.getSolrResourceLoader(); this.loader = config.getSolrResourceLoader();
this.solrHome = loader.getInstanceDir(); this.solrHome = loader.getInstanceDir();
this.cfg = checkNotNull(config); this.cfg = checkNotNull(config);
@ -193,7 +193,7 @@ public class CoreContainer {
*/ */
public static CoreContainer createAndLoad(String solrHome, File configFile) { public static CoreContainer createAndLoad(String solrHome, File configFile) {
SolrResourceLoader loader = new SolrResourceLoader(solrHome); SolrResourceLoader loader = new SolrResourceLoader(solrHome);
CoreContainer cc = new CoreContainer(ConfigSolr.fromFile(loader, configFile)); CoreContainer cc = new CoreContainer(SolrXmlConfig.fromFile(loader, configFile));
try { try {
cc.load(); cc.load();
} catch (Exception e) { } catch (Exception e) {
@ -230,16 +230,16 @@ public class CoreContainer {
shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader); shardHandlerFactory = ShardHandlerFactory.newInstance(cfg.getShardHandlerFactoryPluginInfo(), loader);
updateShardHandler = new UpdateShardHandler(cfg); updateShardHandler = new UpdateShardHandler(cfg.getUpdateShardHandlerConfig());
solrCores.allocateLazyCores(cfg.getTransientCacheSize(), loader); solrCores.allocateLazyCores(cfg.getTransientCacheSize(), loader);
logging = LogWatcher.newRegisteredLogWatcher(cfg.getLogWatcherConfig(), loader); logging = LogWatcher.newRegisteredLogWatcher(cfg.getLogWatcherConfig(), loader);
hostName = cfg.getHost(); hostName = cfg.getNodeName();
log.info("Host Name: " + hostName); log.info("Node Name: " + hostName);
zkSys.initZooKeeper(this, solrHome, cfg); zkSys.initZooKeeper(this, solrHome, cfg.getCloudConfig());
collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class); collectionsHandler = createHandler(cfg.getCollectionsHandlerClass(), CollectionsHandler.class);
containerHandlers.put(COLLECTIONS_HANDLER_PATH, collectionsHandler); containerHandlers.put(COLLECTIONS_HANDLER_PATH, collectionsHandler);
@ -248,7 +248,7 @@ public class CoreContainer {
coreAdminHandler = createHandler(cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class); coreAdminHandler = createHandler(cfg.getCoreAdminHandlerClass(), CoreAdminHandler.class);
containerHandlers.put(CORES_HANDLER_PATH, coreAdminHandler); containerHandlers.put(CORES_HANDLER_PATH, coreAdminHandler);
coreConfigService = cfg.createCoreConfigService(loader, zkSys.getZkController()); coreConfigService = ConfigSetService.createConfigSetService(cfg, loader, zkSys.zkController);
containerProperties = cfg.getSolrProperties(); containerProperties = cfg.getSolrProperties();
@ -860,7 +860,7 @@ public class CoreContainer {
return zkSys.getZkController(); return zkSys.getZkController();
} }
public ConfigSolr getConfig() { public NodeConfig getConfig() {
return cfg; return cfg;
} }
@ -881,10 +881,10 @@ public class CoreContainer {
class CloserThread extends Thread { class CloserThread extends Thread {
CoreContainer container; CoreContainer container;
SolrCores solrCores; SolrCores solrCores;
ConfigSolr cfg; NodeConfig cfg;
CloserThread(CoreContainer container, SolrCores solrCores, ConfigSolr cfg) { CloserThread(CoreContainer container, SolrCores solrCores, NodeConfig cfg) {
this.container = container; this.container = container;
this.solrCores = solrCores; this.solrCores = solrCores;
this.cfg = cfg; this.cfg = cfg;

View File

@ -0,0 +1,297 @@
package org.apache.solr.core;
/*
* 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.
*/
import org.apache.solr.common.SolrException;
import org.apache.solr.logging.LogWatcherConfig;
import org.apache.solr.update.UpdateShardHandlerConfig;
import java.util.Properties;
public class NodeConfig {
private final String nodeName;
private final String coreRootDirectory;
private final String configSetBaseDirectory;
private final String sharedLibDirectory;
private final PluginInfo shardHandlerFactoryConfig;
private final UpdateShardHandlerConfig updateShardHandlerConfig;
private final String coreAdminHandlerClass;
private final String collectionsAdminHandlerClass;
private final String infoHandlerClass;
private final LogWatcherConfig logWatcherConfig;
private final CloudConfig cloudConfig;
private final int coreLoadThreads;
private final int transientCacheSize;
private final boolean useSchemaCache;
private final String managementPath;
private NodeConfig(String nodeName, String coreRootDirectory, String configSetBaseDirectory, String sharedLibDirectory,
PluginInfo shardHandlerFactoryConfig, UpdateShardHandlerConfig updateShardHandlerConfig,
String coreAdminHandlerClass, String collectionsAdminHandlerClass, String infoHandlerClass,
LogWatcherConfig logWatcherConfig, CloudConfig cloudConfig, int coreLoadThreads,
int transientCacheSize, boolean useSchemaCache, String managementPath,
SolrResourceLoader loader, Properties solrProperties) {
this.nodeName = nodeName;
this.coreRootDirectory = coreRootDirectory;
this.configSetBaseDirectory = configSetBaseDirectory;
this.sharedLibDirectory = sharedLibDirectory;
this.shardHandlerFactoryConfig = shardHandlerFactoryConfig;
this.updateShardHandlerConfig = updateShardHandlerConfig;
this.coreAdminHandlerClass = coreAdminHandlerClass;
this.collectionsAdminHandlerClass = collectionsAdminHandlerClass;
this.infoHandlerClass = infoHandlerClass;
this.logWatcherConfig = logWatcherConfig;
this.cloudConfig = cloudConfig;
this.coreLoadThreads = coreLoadThreads;
this.transientCacheSize = transientCacheSize;
this.useSchemaCache = useSchemaCache;
this.managementPath = managementPath;
this.loader = loader;
this.solrProperties = solrProperties;
if (this.cloudConfig != null && this.coreLoadThreads < 2) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"SolrCloud requires a value of at least 2 for coreLoadThreads (configured value = " + this.coreLoadThreads + ")");
}
}
public String getNodeName() {
return nodeName;
}
public String getCoreRootDirectory() {
return coreRootDirectory;
}
public PluginInfo getShardHandlerFactoryPluginInfo() {
return shardHandlerFactoryConfig;
}
public UpdateShardHandlerConfig getUpdateShardHandlerConfig() {
return updateShardHandlerConfig;
}
@Deprecated
public int getDistributedConnectionTimeout() {
return updateShardHandlerConfig.getDistributedConnectionTimeout();
}
@Deprecated
public int getDistributedSocketTimeout() {
return updateShardHandlerConfig.getDistributedSocketTimeout();
}
@Deprecated
public int getMaxUpdateConnections() {
return updateShardHandlerConfig.getMaxUpdateConnections();
}
@Deprecated
public int getMaxUpdateConnectionsPerHost() {
return updateShardHandlerConfig.getMaxUpdateConnectionsPerHost();
}
public int getCoreLoadThreadCount() {
return coreLoadThreads;
}
public String getSharedLibDirectory() {
return sharedLibDirectory;
}
public String getCoreAdminHandlerClass() {
return coreAdminHandlerClass;
}
public String getCollectionsHandlerClass() {
return collectionsAdminHandlerClass;
}
public String getInfoHandlerClass() {
return infoHandlerClass;
}
public boolean hasSchemaCache() {
return useSchemaCache;
}
public String getManagementPath() {
return managementPath;
}
public String getConfigSetBaseDirectory() {
return configSetBaseDirectory;
}
public LogWatcherConfig getLogWatcherConfig() {
return logWatcherConfig;
}
public CloudConfig getCloudConfig() {
return cloudConfig;
}
public int getTransientCacheSize() {
return transientCacheSize;
}
protected final SolrResourceLoader loader;
protected final Properties solrProperties;
public Properties getSolrProperties() {
return solrProperties;
}
public SolrResourceLoader getSolrResourceLoader() {
return loader;
}
public static class NodeConfigBuilder {
private String coreRootDirectory = "";
private String configSetBaseDirectory = "configsets";
private String sharedLibDirectory = "lib";
private PluginInfo shardHandlerFactoryConfig;
private UpdateShardHandlerConfig updateShardHandlerConfig = UpdateShardHandlerConfig.DEFAULT;
private String coreAdminHandlerClass = DEFAULT_ADMINHANDLERCLASS;
private String collectionsAdminHandlerClass = DEFAULT_COLLECTIONSHANDLERCLASS;
private String infoHandlerClass = DEFAULT_INFOHANDLERCLASS;
private LogWatcherConfig logWatcherConfig = new LogWatcherConfig(true, null, null, 50);
private CloudConfig cloudConfig;
private int coreLoadThreads = DEFAULT_CORE_LOAD_THREADS;
private int transientCacheSize = DEFAULT_TRANSIENT_CACHE_SIZE;
private boolean useSchemaCache = false;
private String managementPath;
private Properties solrProperties = new Properties();
private final SolrResourceLoader loader;
private final String nodeName;
private static final int DEFAULT_CORE_LOAD_THREADS = 3;
private static final int DEFAULT_TRANSIENT_CACHE_SIZE = Integer.MAX_VALUE;
private static final String DEFAULT_ADMINHANDLERCLASS = "org.apache.solr.handler.admin.CoreAdminHandler";
private static final String DEFAULT_INFOHANDLERCLASS = "org.apache.solr.handler.admin.InfoHandler";
private static final String DEFAULT_COLLECTIONSHANDLERCLASS = "org.apache.solr.handler.admin.CollectionsHandler";
public NodeConfigBuilder(String nodeName, SolrResourceLoader loader) {
this.nodeName = nodeName;
this.loader = loader;
this.coreRootDirectory = loader.getInstanceDir();
}
public NodeConfigBuilder setCoreRootDirectory(String coreRootDirectory) {
this.coreRootDirectory = loader.resolve(coreRootDirectory);
return this;
}
public NodeConfigBuilder setConfigSetBaseDirectory(String configSetBaseDirectory) {
this.configSetBaseDirectory = configSetBaseDirectory;
return this;
}
public NodeConfigBuilder setSharedLibDirectory(String sharedLibDirectory) {
this.sharedLibDirectory = sharedLibDirectory;
return this;
}
public NodeConfigBuilder setShardHandlerFactoryConfig(PluginInfo shardHandlerFactoryConfig) {
this.shardHandlerFactoryConfig = shardHandlerFactoryConfig;
return this;
}
public NodeConfigBuilder setUpdateShardHandlerConfig(UpdateShardHandlerConfig updateShardHandlerConfig) {
this.updateShardHandlerConfig = updateShardHandlerConfig;
return this;
}
public NodeConfigBuilder setCoreAdminHandlerClass(String coreAdminHandlerClass) {
this.coreAdminHandlerClass = coreAdminHandlerClass;
return this;
}
public NodeConfigBuilder setCollectionsAdminHandlerClass(String collectionsAdminHandlerClass) {
this.collectionsAdminHandlerClass = collectionsAdminHandlerClass;
return this;
}
public NodeConfigBuilder setInfoHandlerClass(String infoHandlerClass) {
this.infoHandlerClass = infoHandlerClass;
return this;
}
public NodeConfigBuilder setLogWatcherConfig(LogWatcherConfig logWatcherConfig) {
this.logWatcherConfig = logWatcherConfig;
return this;
}
public NodeConfigBuilder setCloudConfig(CloudConfig cloudConfig) {
this.cloudConfig = cloudConfig;
return this;
}
public NodeConfigBuilder setCoreLoadThreads(int coreLoadThreads) {
this.coreLoadThreads = coreLoadThreads;
return this;
}
public NodeConfigBuilder setTransientCacheSize(int transientCacheSize) {
this.transientCacheSize = transientCacheSize;
return this;
}
public NodeConfigBuilder setUseSchemaCache(boolean useSchemaCache) {
this.useSchemaCache = useSchemaCache;
return this;
}
public NodeConfigBuilder setManagementPath(String managementPath) {
this.managementPath = managementPath;
return this;
}
public NodeConfigBuilder setSolrProperties(Properties solrProperties) {
this.solrProperties = solrProperties;
return this;
}
public NodeConfig build() {
return new NodeConfig(nodeName, coreRootDirectory, configSetBaseDirectory, sharedLibDirectory, shardHandlerFactoryConfig,
updateShardHandlerConfig, coreAdminHandlerClass, collectionsAdminHandlerClass, infoHandlerClass,
logWatcherConfig, cloudConfig, coreLoadThreads, transientCacheSize, useSchemaCache, managementPath, loader, solrProperties);
}
}
}

View File

@ -126,7 +126,7 @@ class SolrCores {
} }
//WARNING! This should be the _only_ place you put anything into the list of transient cores! //WARNING! This should be the _only_ place you put anything into the list of transient cores!
protected SolrCore putTransientCore(ConfigSolr cfg, String name, SolrCore core, SolrResourceLoader loader) { protected SolrCore putTransientCore(NodeConfig cfg, String name, SolrCore core, SolrResourceLoader loader) {
SolrCore retCore; SolrCore retCore;
CoreContainer.log.info("Opening transient core {}", name); CoreContainer.log.info("Opening transient core {}", name);
synchronized (modifyLock) { synchronized (modifyLock) {

View File

@ -0,0 +1,412 @@
package org.apache.solr.core;
/*
* 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.
*/
import com.google.common.base.Strings;
import org.apache.commons.io.IOUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.logging.LogWatcherConfig;
import org.apache.solr.update.UpdateShardHandlerConfig;
import org.apache.solr.util.DOMUtil;
import org.apache.solr.util.PropertiesUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import org.xml.sax.InputSource;
import javax.xml.xpath.XPath;
import javax.xml.xpath.XPathConstants;
import javax.xml.xpath.XPathExpressionException;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.HashSet;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
/**
*
*/
public class SolrXmlConfig {
public final static String SOLR_XML_FILE = "solr.xml";
private static final Logger log = LoggerFactory.getLogger(SolrXmlConfig.class);
public static NodeConfig fromConfig(Config config) {
checkForIllegalConfig(config);
config.substituteProperties();
CloudConfig cloudConfig = null;
UpdateShardHandlerConfig deprecatedUpdateConfig = null;
if (config.getNodeList("solr/solrcloud", false).getLength() > 0) {
NamedList<Object> cloudSection = readNodeListAsNamedList(config, "solr/solrcloud/*[@name]", "<solrcloud>");
deprecatedUpdateConfig = loadUpdateConfig(cloudSection, false);
cloudConfig = fillSolrCloudSection(cloudSection);
}
NamedList<Object> entries = readNodeListAsNamedList(config, "solr/*[@name]", "<solr>");
String nodeName = (String) entries.remove("nodeName");
if (Strings.isNullOrEmpty(nodeName) && cloudConfig != null)
nodeName = cloudConfig.getHost();
UpdateShardHandlerConfig updateConfig;
if (deprecatedUpdateConfig == null) {
updateConfig = loadUpdateConfig(readNodeListAsNamedList(config, "solr/updateshardhandler/*[@name]", "<updateshardhandler>"), true);
}
else {
updateConfig = loadUpdateConfig(readNodeListAsNamedList(config, "solr/updateshardhandler/*[@name]", "<updateshardhandler>"), false);
if (updateConfig != null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "UpdateShardHandler configuration defined twice in solr.xml");
}
updateConfig = deprecatedUpdateConfig;
}
NodeConfig.NodeConfigBuilder configBuilder = new NodeConfig.NodeConfigBuilder(nodeName, config.getResourceLoader());
configBuilder.setUpdateShardHandlerConfig(updateConfig);
configBuilder.setShardHandlerFactoryConfig(getShardHandlerFactoryPluginInfo(config));
configBuilder.setLogWatcherConfig(loadLogWatcherConfig(config, "solr/logging/*[@name]", "solr/logging/watcher/*[@name]"));
configBuilder.setSolrProperties(loadProperties(config));
if (cloudConfig != null)
configBuilder.setCloudConfig(cloudConfig);
return fillSolrSection(configBuilder, entries);
}
public static NodeConfig fromFile(SolrResourceLoader loader, File configFile) {
log.info("Loading container configuration from {}", configFile.getAbsolutePath());
if (!configFile.exists()) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"solr.xml does not exist in " + configFile.getAbsolutePath() + " cannot start Solr");
}
try (InputStream inputStream = new FileInputStream(configFile)) {
return fromInputStream(loader, inputStream);
} catch (SolrException exc) {
throw exc;
} catch (Exception exc) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Could not load SOLR configuration", exc);
}
}
public static NodeConfig fromString(SolrResourceLoader loader, String xml) {
return fromInputStream(loader, new ByteArrayInputStream(xml.getBytes(StandardCharsets.UTF_8)));
}
public static NodeConfig fromInputStream(SolrResourceLoader loader, InputStream is) {
try {
byte[] buf = IOUtils.toByteArray(is);
try (ByteArrayInputStream dup = new ByteArrayInputStream(buf)) {
Config config = new Config(loader, null, new InputSource(dup), null, false);
return fromConfig(config);
}
} catch (SolrException exc) {
throw exc;
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
}
}
public static NodeConfig fromSolrHome(SolrResourceLoader loader, String solrHome) {
return fromFile(loader, new File(solrHome, SOLR_XML_FILE));
}
private static void checkForIllegalConfig(Config config) {
failIfFound(config, "solr/@coreLoadThreads");
failIfFound(config, "solr/@persistent");
failIfFound(config, "solr/@sharedLib");
failIfFound(config, "solr/@zkHost");
failIfFound(config, "solr/cores");
assertSingleInstance("solrcloud", config);
assertSingleInstance("logging", config);
assertSingleInstance("logging/watcher", config);
}
private static void assertSingleInstance(String section, Config config) {
if (config.getNodeList("/solr/" + section, false).getLength() > 1)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Multiple instances of " + section + " section found in solr.xml");
}
private static void failIfFound(Config config, String xPath) {
if (config.getVal(xPath, false) != null) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Should not have found " + xPath +
"\n. Please upgrade your solr.xml: https://cwiki.apache.org/confluence/display/solr/Format+of+solr.xml");
}
}
private static Properties loadProperties(Config config) {
try {
Node node = ((NodeList) config.evaluate("solr", XPathConstants.NODESET)).item(0);
XPath xpath = config.getXPath();
NodeList props = (NodeList) xpath.evaluate("property", node, XPathConstants.NODESET);
Properties properties = new Properties();
for (int i = 0; i < props.getLength(); i++) {
Node prop = props.item(i);
properties.setProperty(DOMUtil.getAttr(prop, "name"),
PropertiesUtil.substituteProperty(DOMUtil.getAttr(prop, "value"), null));
}
return properties;
}
catch (XPathExpressionException e) {
log.warn("Error parsing solr.xml: " + e.getMessage());
return null;
}
}
private static NamedList<Object> readNodeListAsNamedList(Config config, String path, String section) {
NodeList nodes = config.getNodeList(path, false);
if (nodes == null) {
return null;
}
return checkForDuplicates(section, DOMUtil.nodesToNamedList(nodes));
}
private static NamedList<Object> checkForDuplicates(String section, NamedList<Object> nl) {
Set<String> keys = new HashSet<>();
for (Map.Entry<String, Object> entry : nl) {
if (!keys.add(entry.getKey()))
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
section + " section of solr.xml contains duplicated '" + entry.getKey() + "'");
}
return nl;
}
private static int parseInt(String field, String value) {
try {
return Integer.parseInt(value);
}
catch (NumberFormatException e) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Error parsing '" + field + "', value '" + value + "' cannot be parsed as int");
}
}
private static NodeConfig fillSolrSection(NodeConfig.NodeConfigBuilder builder, NamedList<Object> nl) {
for (Map.Entry<String, Object> entry : nl) {
String name = entry.getKey();
if (entry.getValue() == null)
continue;
String value = entry.getValue().toString();
switch (name) {
case "adminHandler":
builder.setCoreAdminHandlerClass(value);
break;
case "collectionsHandler":
builder.setCollectionsAdminHandlerClass(value);
break;
case "infoHandler":
builder.setInfoHandlerClass(value);
break;
case "coreRootDirectory":
builder.setCoreRootDirectory(value);
break;
case "managementPath":
builder.setManagementPath(value);
break;
case "sharedLib":
builder.setSharedLibDirectory(value);
break;
case "configSetBaseDir":
builder.setConfigSetBaseDirectory(value);
break;
case "shareSchema":
builder.setUseSchemaCache(Boolean.parseBoolean(value));
break;
case "coreLoadThreads":
builder.setCoreLoadThreads(parseInt(name, value));
break;
case "transientCacheSize":
builder.setTransientCacheSize(parseInt(name, value));
break;
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown configuration value in solr.xml: " + name);
}
}
return builder.build();
}
private static UpdateShardHandlerConfig loadUpdateConfig(NamedList<Object> nl, boolean alwaysDefine) {
if (nl == null && !alwaysDefine)
return null;
if (nl == null)
return UpdateShardHandlerConfig.DEFAULT;
boolean defined = false;
int maxUpdateConnections = UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONS;
int maxUpdateConnectionsPerHost = UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST;
int distributedSocketTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATESOTIMEOUT;
int distributedConnectionTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
Object muc = nl.remove("maxUpdateConnections");
if (muc != null) {
maxUpdateConnections = parseInt("maxUpdateConnections", muc.toString());
defined = true;
}
Object mucph = nl.remove("maxUpdateConnectionsPerHost");
if (mucph != null) {
maxUpdateConnectionsPerHost = parseInt("maxUpdateConnectionsPerHost", mucph.toString());
defined = true;
}
Object dst = nl.remove("distribUpdateSoTimeout");
if (dst != null) {
distributedSocketTimeout = parseInt("distribUpdateSoTimeout", dst.toString());
defined = true;
}
Object dct = nl.remove("distribUpdateConnTimeout");
if (dct != null) {
distributedConnectionTimeout = parseInt("distribUpdateConnTimeout", dct.toString());
defined = true;
}
if (!defined && !alwaysDefine)
return null;
return new UpdateShardHandlerConfig(maxUpdateConnections, maxUpdateConnectionsPerHost, distributedSocketTimeout, distributedConnectionTimeout);
}
private static String removeValue(NamedList<Object> nl, String key) {
Object value = nl.remove(key);
if (value == null)
return null;
return value.toString();
}
private static String required(String section, String key, String value) {
if (value != null)
return value;
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, section + " section missing required entry '" + key + "'");
}
private static CloudConfig fillSolrCloudSection(NamedList<Object> nl) {
String hostName = required("solrcloud", "host", removeValue(nl, "host"));
int hostPort = parseInt("hostPort", required("solrcloud", "hostPort", removeValue(nl, "hostPort")));
String hostContext = required("solrcloud", "hostContext", removeValue(nl, "hostContext"));
CloudConfig.CloudConfigBuilder builder = new CloudConfig.CloudConfigBuilder(hostName, hostPort, hostContext);
for (Map.Entry<String, Object> entry : nl) {
String name = entry.getKey();
if (entry.getValue() == null)
continue;
String value = entry.getValue().toString();
switch (name) {
case "leaderVoteWait":
builder.setLeaderVoteWait(parseInt(name, value));
break;
case "leaderConflictResolveWait":
builder.setLeaderConflictResolveWait(parseInt(name, value));
break;
case "zkClientTimeout":
builder.setZkClientTimeout(parseInt(name, value));
break;
case "autoReplicaFailoverBadNodeExpiration":
builder.setAutoReplicaFailoverBadNodeExpiration(parseInt(name, value));
break;
case "autoReplicaFailoverWaitAfterExpiration":
builder.setAutoReplicaFailoverWaitAfterExpiration(parseInt(name, value));
break;
case "autoReplicaFailoverWorkLoopDelay":
builder.setAutoReplicaFailoverWorkLoopDelay(parseInt(name, value));
break;
case "zkHost":
builder.setZkHost(value);
break;
case "genericCoreNodeNames":
builder.setUseGenericCoreNames(Boolean.parseBoolean(value));
break;
case "zkACLProvider":
builder.setZkACLProviderClass(value);
break;
case "zkCredientialsProvider":
builder.setZkCredentialsProviderClass(value);
break;
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown configuration parameter in <solrcloud> section of solr.xml: " + name);
}
}
return builder.build();
}
private static LogWatcherConfig loadLogWatcherConfig(Config config, String loggingPath, String watcherPath) {
String loggingClass = null;
boolean enabled = true;
int watcherQueueSize = 50;
String watcherThreshold = null;
for (Map.Entry<String, Object> entry : readNodeListAsNamedList(config, loggingPath, "<logging>")) {
String name = entry.getKey();
String value = entry.getValue().toString();
switch (name) {
case "class":
loggingClass = value; break;
case "enabled":
enabled = Boolean.parseBoolean(value); break;
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown value in logwatcher config: " + name);
}
}
for (Map.Entry<String, Object> entry : readNodeListAsNamedList(config, watcherPath, "<watcher>")) {
String name = entry.getKey();
String value = entry.getValue().toString();
switch (name) {
case "size":
watcherQueueSize = parseInt(name, value); break;
case "threshold":
watcherThreshold = value; break;
default:
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown value in logwatcher config: " + name);
}
}
return new LogWatcherConfig(enabled, loggingClass, watcherThreshold, watcherQueueSize);
}
private static PluginInfo getShardHandlerFactoryPluginInfo(Config config) {
Node node = config.getNode("solr/shardHandlerFactory", false);
return (node == null) ? null : new PluginInfo(node, "shardHandlerFactory", false, true);
}
}

View File

@ -57,44 +57,19 @@ public class ZkContainer {
} }
public void initZooKeeper(final CoreContainer cc, String solrHome, ConfigSolr config) { public void initZooKeeper(final CoreContainer cc, String solrHome, CloudConfig 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.getSolrHostPort(), config.getZkHostContext(),
config.getHost(), config.getLeaderVoteWait(), config.getLeaderConflictResolveWait(), config.getGenericCoreNodeNames());
}
public void initZooKeeper(final CoreContainer cc, String solrHome, String zkHost, int zkClientTimeout, String solrHostPort,
String hostContext, String host, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames) {
ZkController zkController = null; ZkController zkController = null;
// if zkHost sys property is not set, we are not using ZooKeeper
String zookeeperHost;
if(zkHost == null) {
zookeeperHost = System.getProperty("zkHost");
} else {
zookeeperHost = zkHost;
}
String zkRun = System.getProperty("zkRun"); String zkRun = System.getProperty("zkRun");
if (zkRun != null && config == null)
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Cannot start Solr in cloud mode - no cloud config provided");
if (zkRun == null && zookeeperHost == null) if (config == null)
return; // not in zk mode return; // not in zk mode
if (null == solrHostPort) { String zookeeperHost = config.getZkHost();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"'hostPort' must be configured to run SolrCloud");
}
if (null == hostContext) {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"'hostContext' must be configured to run SolrCloud");
}
// zookeeper in quorum mode currently causes a failure when trying to // zookeeper in quorum mode currently causes a failure when trying to
// register log4j mbeans. See SOLR-2369 // register log4j mbeans. See SOLR-2369
@ -104,7 +79,7 @@ public class ZkContainer {
if (zkRun != null) { if (zkRun != null) {
String zkDataHome = System.getProperty("zkServerDataDir", solrHome + "zoo_data"); String zkDataHome = System.getProperty("zkServerDataDir", solrHome + "zoo_data");
String zkConfHome = System.getProperty("zkServerConfDir", solrHome); String zkConfHome = System.getProperty("zkServerConfDir", solrHome);
zkServer = new SolrZkServer(stripChroot(zkRun), stripChroot(zookeeperHost), zkDataHome, zkConfHome, solrHostPort); zkServer = new SolrZkServer(stripChroot(zkRun), stripChroot(config.getZkHost()), zkDataHome, zkConfHome, config.getSolrHostPort());
zkServer.parseConfig(); zkServer.parseConfig();
zkServer.start(); zkServer.start();
@ -134,9 +109,7 @@ public class ZkContainer {
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"A chroot was specified in ZkHost but the znode doesn't exist. " + zookeeperHost); "A chroot was specified in ZkHost but the znode doesn't exist. " + zookeeperHost);
} }
zkController = new ZkController(cc, zookeeperHost, zkClientTimeout, zkController = new ZkController(cc, zookeeperHost, zkClientConnectTimeout, config,
zkClientConnectTimeout, host, solrHostPort, hostContext,
leaderVoteWait, leaderConflictResolveWait, genericCoreNodeNames,
new CurrentCoreDescriptorProvider() { new CurrentCoreDescriptorProvider() {
@Override @Override

View File

@ -29,8 +29,8 @@ import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.StrUtils; import org.apache.solr.common.util.StrUtils;
import org.apache.solr.common.util.URLUtil; import org.apache.solr.common.util.URLUtil;
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.PluginInfo; import org.apache.solr.core.PluginInfo;
import org.apache.solr.update.UpdateShardHandlerConfig;
import org.apache.solr.util.DefaultSolrThreadFactory; import org.apache.solr.util.DefaultSolrThreadFactory;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -69,8 +69,8 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
protected HttpClient defaultClient; protected HttpClient defaultClient;
private LBHttpSolrClient loadbalancer; private LBHttpSolrClient loadbalancer;
//default values: //default values:
int soTimeout = ConfigSolr.DEFAULT_DISTRIBUPDATESOTIMEOUT; int soTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATESOTIMEOUT;
int connectionTimeout = ConfigSolr.DEFAULT_DISTRIBUPDATECONNTIMEOUT; int connectionTimeout = UpdateShardHandlerConfig.DEFAULT_DISTRIBUPDATECONNTIMEOUT;
int maxConnectionsPerHost = 20; int maxConnectionsPerHost = 20;
int maxConnections = 10000; int maxConnections = 10000;
int corePoolSize = 0; int corePoolSize = 0;

View File

@ -17,30 +17,6 @@
package org.apache.solr.servlet; package org.apache.solr.servlet;
import java.io.ByteArrayInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import javax.servlet.FilterChain;
import javax.servlet.FilterConfig;
import javax.servlet.ServletException;
import javax.servlet.ServletRequest;
import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.io.IOUtils; import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.http.Header; import org.apache.http.Header;
@ -77,11 +53,12 @@ import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SimpleOrderedMap; import org.apache.solr.common.util.SimpleOrderedMap;
import org.apache.solr.common.util.StrUtils; import org.apache.solr.common.util.StrUtils;
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.NodeConfig;
import org.apache.solr.core.SolrConfig; import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.core.SolrXmlConfig;
import org.apache.solr.handler.ContentStreamHandlerBase; import org.apache.solr.handler.ContentStreamHandlerBase;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrQueryRequestBase; import org.apache.solr.request.SolrQueryRequestBase;
@ -96,6 +73,29 @@ import org.apache.solr.update.processor.DistributingUpdateProcessorFactory;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import javax.servlet.FilterChain;
import javax.servlet.FilterConfig;
import javax.servlet.ServletException;
import javax.servlet.ServletRequest;
import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.ByteArrayInputStream;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
/** /**
* This filter looks at the incoming URL maps them to handlers defined in solrconfig.xml * This filter looks at the incoming URL maps them to handlers defined in solrconfig.xml
* *
@ -141,12 +141,12 @@ public class SolrDispatchFilter extends BaseSolrFilter {
log.info("SolrDispatchFilter.init() done"); log.info("SolrDispatchFilter.init() done");
} }
private ConfigSolr loadConfigSolr(SolrResourceLoader loader) { private NodeConfig loadConfigSolr(SolrResourceLoader loader) {
String solrxmlLocation = System.getProperty("solr.solrxml.location", "solrhome"); String solrxmlLocation = System.getProperty("solr.solrxml.location", "solrhome");
if (solrxmlLocation == null || "solrhome".equalsIgnoreCase(solrxmlLocation)) if (solrxmlLocation == null || "solrhome".equalsIgnoreCase(solrxmlLocation))
return ConfigSolr.fromSolrHome(loader, loader.getInstanceDir()); return SolrXmlConfig.fromSolrHome(loader, loader.getInstanceDir());
if ("zookeeper".equalsIgnoreCase(solrxmlLocation)) { if ("zookeeper".equalsIgnoreCase(solrxmlLocation)) {
String zkHost = System.getProperty("zkHost"); String zkHost = System.getProperty("zkHost");
@ -159,7 +159,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
if (!zkClient.exists("/solr.xml", true)) if (!zkClient.exists("/solr.xml", true))
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not load solr.xml from zookeeper: node not found"); throw new SolrException(ErrorCode.SERVER_ERROR, "Could not load solr.xml from zookeeper: node not found");
byte[] data = zkClient.getData("/solr.xml", null, null, true); byte[] data = zkClient.getData("/solr.xml", null, null, true);
return ConfigSolr.fromInputStream(loader, new ByteArrayInputStream(data)); return SolrXmlConfig.fromInputStream(loader, new ByteArrayInputStream(data));
} catch (Exception e) { } catch (Exception e) {
throw new SolrException(ErrorCode.SERVER_ERROR, "Could not load solr.xml from zookeeper", e); throw new SolrException(ErrorCode.SERVER_ERROR, "Could not load solr.xml from zookeeper", e);
} finally { } finally {
@ -177,7 +177,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
*/ */
protected CoreContainer createCoreContainer() { protected CoreContainer createCoreContainer() {
SolrResourceLoader loader = new SolrResourceLoader(SolrResourceLoader.locateSolrHome()); SolrResourceLoader loader = new SolrResourceLoader(SolrResourceLoader.locateSolrHome());
ConfigSolr config = loadConfigSolr(loader); NodeConfig config = loadConfigSolr(loader);
CoreContainer cores = new CoreContainer(config); CoreContainer cores = new CoreContainer(config);
cores.load(); cores.load();
return cores; return cores;

View File

@ -17,9 +17,6 @@ package org.apache.solr.update;
* limitations under the License. * limitations under the License.
*/ */
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.http.client.HttpClient; import org.apache.http.client.HttpClient;
import org.apache.http.conn.ClientConnectionManager; import org.apache.http.conn.ClientConnectionManager;
import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.CloseableHttpClient;
@ -31,10 +28,13 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil; import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.IOUtils; import org.apache.solr.common.util.IOUtils;
import org.apache.solr.common.util.SolrjNamedThreadFactory; import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.core.NodeConfig;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
public class UpdateShardHandler { public class UpdateShardHandler {
private static Logger log = LoggerFactory.getLogger(UpdateShardHandler.class); private static Logger log = LoggerFactory.getLogger(UpdateShardHandler.class);
@ -46,7 +46,12 @@ public class UpdateShardHandler {
private final CloseableHttpClient client; private final CloseableHttpClient client;
public UpdateShardHandler(ConfigSolr cfg) { @Deprecated
public UpdateShardHandler(NodeConfig cfg) {
this(cfg.getUpdateShardHandlerConfig());
}
public UpdateShardHandler(UpdateShardHandlerConfig cfg) {
clientConnectionManager = new PoolingClientConnectionManager(SchemeRegistryFactory.createSystemDefault()); clientConnectionManager = new PoolingClientConnectionManager(SchemeRegistryFactory.createSystemDefault());
if (cfg != null ) { if (cfg != null ) {

View File

@ -0,0 +1,61 @@
package org.apache.solr.update;
/*
* 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.
*/
public class UpdateShardHandlerConfig {
public static final int DEFAULT_DISTRIBUPDATECONNTIMEOUT = 60000;
public static final int DEFAULT_DISTRIBUPDATESOTIMEOUT = 600000;
public static final int DEFAULT_MAXUPDATECONNECTIONS = 10000;
public static final int DEFAULT_MAXUPDATECONNECTIONSPERHOST = 100;
public static final UpdateShardHandlerConfig DEFAULT
= new UpdateShardHandlerConfig(DEFAULT_MAXUPDATECONNECTIONS, DEFAULT_MAXUPDATECONNECTIONSPERHOST,
DEFAULT_DISTRIBUPDATESOTIMEOUT, DEFAULT_DISTRIBUPDATECONNTIMEOUT);
private final int maxUpdateConnections;
private final int maxUpdateConnectionsPerHost;
private final int distributedSocketTimeout;
private final int distributedConnectionTimeout;
public UpdateShardHandlerConfig(int maxUpdateConnections, int maxUpdateConnectionsPerHost, int distributedSocketTimeout, int distributedConnectionTimeout) {
this.maxUpdateConnections = maxUpdateConnections;
this.maxUpdateConnectionsPerHost = maxUpdateConnectionsPerHost;
this.distributedSocketTimeout = distributedSocketTimeout;
this.distributedConnectionTimeout = distributedConnectionTimeout;
}
public int getMaxUpdateConnectionsPerHost() {
return maxUpdateConnectionsPerHost;
}
public int getMaxUpdateConnections() {
return maxUpdateConnections;
}
public int getDistributedSocketTimeout() {
return distributedSocketTimeout;
}
public int getDistributedConnectionTimeout() {
return distributedConnectionTimeout;
}
}

View File

@ -27,9 +27,10 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.handler.component.HttpShardHandlerFactory; import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.update.UpdateShardHandler; import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.util.MockConfigSolr; import org.apache.solr.update.UpdateShardHandlerConfig;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.junit.Ignore; import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
@ -242,10 +243,10 @@ public class ChaosMonkeyShardSplitTest extends ShardSplitTest {
SolrZkClient zkClient = new SolrZkClient(address, TIMEOUT); SolrZkClient zkClient = new SolrZkClient(address, TIMEOUT);
ZkStateReader reader = new ZkStateReader(zkClient); ZkStateReader reader = new ZkStateReader(zkClient);
LeaderElector overseerElector = new LeaderElector(zkClient); LeaderElector overseerElector = new LeaderElector(zkClient);
UpdateShardHandler updateShardHandler = new UpdateShardHandler(null); UpdateShardHandler updateShardHandler = new UpdateShardHandler(UpdateShardHandlerConfig.DEFAULT);
// TODO: close Overseer // TODO: close Overseer
Overseer overseer = new Overseer( Overseer overseer = new Overseer(new HttpShardHandlerFactory().getShardHandler(), updateShardHandler, "/admin/cores",
new HttpShardHandlerFactory().getShardHandler(), updateShardHandler, "/admin/cores", reader, null, new MockConfigSolr()); reader, null, new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "solr").build());
overseer.close(); overseer.close();
ElectionContext ec = new OverseerElectionContext(zkClient, overseer, ElectionContext ec = new OverseerElectionContext(zkClient, overseer,
address.replaceAll("/", "_")); address.replaceAll("/", "_"));

View File

@ -17,6 +17,37 @@ package org.apache.solr.cloud;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.update.UpdateShardHandlerConfig;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.stats.Snapshot;
import org.apache.solr.util.stats.Timer;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.xml.sax.SAXException;
import javax.xml.parsers.ParserConfigurationException;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -32,37 +63,6 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import javax.xml.parsers.ParserConfigurationException;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.cloud.overseer.OverseerAction;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.util.DefaultSolrThreadFactory;
import org.apache.solr.util.MockConfigSolr;
import org.apache.solr.util.stats.Snapshot;
import org.apache.solr.util.stats.Timer;
import org.apache.solr.util.stats.TimerContext;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;
import org.xml.sax.SAXException;
@Slow @Slow
public class OverseerTest extends SolrTestCaseJ4 { public class OverseerTest extends SolrTestCaseJ4 {
@ -1129,12 +1129,12 @@ public class OverseerTest extends SolrTestCaseJ4 {
overseers.get(overseers.size() -1).close(); overseers.get(overseers.size() -1).close();
overseers.get(overseers.size() -1).getZkStateReader().getZkClient().close(); overseers.get(overseers.size() -1).getZkStateReader().getZkClient().close();
} }
UpdateShardHandler updateShardHandler = new UpdateShardHandler(null); UpdateShardHandler updateShardHandler = new UpdateShardHandler(UpdateShardHandlerConfig.DEFAULT);
updateShardHandlers.add(updateShardHandler); updateShardHandlers.add(updateShardHandler);
HttpShardHandlerFactory httpShardHandlerFactory = new HttpShardHandlerFactory(); HttpShardHandlerFactory httpShardHandlerFactory = new HttpShardHandlerFactory();
httpShardHandlerFactorys.add(httpShardHandlerFactory); httpShardHandlerFactorys.add(httpShardHandlerFactory);
Overseer overseer = new Overseer( Overseer overseer = new Overseer(httpShardHandlerFactory.getShardHandler(), updateShardHandler, "/admin/cores", reader, null,
httpShardHandlerFactory.getShardHandler(), updateShardHandler, "/admin/cores", reader, null, new MockConfigSolr()); new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "").build());
overseers.add(overseer); overseers.add(overseer);
ElectionContext ec = new OverseerElectionContext(zkClient, overseer, ElectionContext ec = new OverseerElectionContext(zkClient, overseer,
address.replaceAll("/", "_")); address.replaceAll("/", "_"));

View File

@ -16,17 +16,13 @@ package org.apache.solr.cloud;
* the License. * the License.
*/ */
import java.io.File; import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.nio.charset.StandardCharsets;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.core.NodeConfig;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.servlet.SolrDispatchFilter;
import org.junit.After; import org.junit.After;
@ -36,7 +32,10 @@ import org.junit.Test;
import org.junit.rules.RuleChain; import org.junit.rules.RuleChain;
import org.junit.rules.TestRule; import org.junit.rules.TestRule;
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule; import java.io.File;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.nio.charset.StandardCharsets;
public class SolrXmlInZkTest extends SolrTestCaseJ4 { public class SolrXmlInZkTest extends SolrTestCaseJ4 {
@ -51,7 +50,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
private ZkStateReader reader; private ZkStateReader reader;
private ConfigSolr cfg; private NodeConfig cfg;
private SolrDispatchFilter solrDispatchFilter; private SolrDispatchFilter solrDispatchFilter;
@ -109,7 +108,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
if (solrDispatchFilter != null) solrDispatchFilter.destroy(); if (solrDispatchFilter != null) solrDispatchFilter.destroy();
solrDispatchFilter = new SolrDispatchFilter(); solrDispatchFilter = new SolrDispatchFilter();
Object obj = method.invoke(solrDispatchFilter, new SolrResourceLoader(null)); Object obj = method.invoke(solrDispatchFilter, new SolrResourceLoader(null));
cfg = (ConfigSolr) obj; cfg = (NodeConfig) obj;
log.info("####SETUP_END " + getTestName()); log.info("####SETUP_END " + getTestName());
} }
@ -130,7 +129,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
try { try {
setUpZkAndDiskXml(true, true); setUpZkAndDiskXml(true, true);
assertEquals("Should have gotten a new port the xml file sent to ZK, overrides the copy on disk", assertEquals("Should have gotten a new port the xml file sent to ZK, overrides the copy on disk",
cfg.getSolrHostPort(), "9045"); cfg.getCloudConfig().getSolrHostPort(), 9045);
} finally { } finally {
closeZK(); closeZK();
} }
@ -141,7 +140,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
try { try {
setUpZkAndDiskXml(true, false); setUpZkAndDiskXml(true, false);
assertEquals("Should have gotten a new port the xml file sent to ZK", assertEquals("Should have gotten a new port the xml file sent to ZK",
cfg.getSolrHostPort(), "9045"); cfg.getCloudConfig().getSolrHostPort(), 9045);
} finally { } finally {
closeZK(); closeZK();
} }
@ -180,7 +179,7 @@ public class SolrXmlInZkTest extends SolrTestCaseJ4 {
try { try {
System.clearProperty("solr.solrxml.location"); System.clearProperty("solr.solrxml.location");
setUpZkAndDiskXml(false, true); setUpZkAndDiskXml(false, true);
assertEquals("Should have gotten the default port", cfg.getSolrHostPort(), "8983"); assertEquals("Should have gotten the default port", cfg.getCloudConfig().getSolrHostPort(), 8983);
} finally { } finally {
closeZK(); closeZK();
} }

View File

@ -19,6 +19,7 @@ package org.apache.solr.cloud;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor; import org.apache.solr.core.CoreDescriptor;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -46,7 +47,11 @@ public class TestLeaderElectionZkExpiry extends SolrTestCaseJ4 {
AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost()); AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
AbstractZkTestCase.makeSolrZkNode(server.getZkHost()); AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
final ZkController zkController = new ZkController(cc, server.getZkAddress(), 15000, 30000, "dummy.host.com", "8984", "/solr", 180000, 180000, true, new CurrentCoreDescriptorProvider() { CloudConfig cloudConfig = new CloudConfig.CloudConfigBuilder("dummy.host.com", 8984, "solr")
.setLeaderConflictResolveWait(180000)
.setLeaderVoteWait(180000)
.build();
final ZkController zkController = new ZkController(cc, server.getZkAddress(), 15000, cloudConfig, new CurrentCoreDescriptorProvider() {
@Override @Override
public List<CoreDescriptor> getCurrentDescriptors() { public List<CoreDescriptor> getCurrentDescriptors() {
return Collections.EMPTY_LIST; return Collections.EMPTY_LIST;

View File

@ -23,14 +23,13 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager; import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor; import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.CoresLocator;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.handler.admin.CoreAdminHandler; import org.apache.solr.handler.admin.CoreAdminHandler;
import org.apache.solr.handler.component.HttpShardHandlerFactory; import org.apache.solr.handler.component.HttpShardHandlerFactory;
import org.apache.solr.update.UpdateShardHandler; import org.apache.solr.update.UpdateShardHandler;
import org.apache.solr.update.UpdateShardHandlerConfig;
import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.CreateMode;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
@ -181,9 +180,10 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
zkClient.close(); zkClient.close();
cc = getCoreContainer(); cc = getCoreContainer();
ZkController zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, 10000, CloudConfig cloudConfig = new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "solr").build();
"127.0.0.1", "8983", "solr", 0, 60000, true, new CurrentCoreDescriptorProvider() { ZkController zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, cloudConfig,
new CurrentCoreDescriptorProvider() {
@Override @Override
public List<CoreDescriptor> getCurrentDescriptors() { public List<CoreDescriptor> getCurrentDescriptors() {
@ -206,7 +206,6 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
} }
@Test
public void testGetHostName() throws Exception { public void testGetHostName() throws Exception {
String zkDir = createTempDir("zkData").toFile().getAbsolutePath(); String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
CoreContainer cc = null; CoreContainer cc = null;
@ -222,8 +221,8 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
ZkController zkController = null; ZkController zkController = null;
try { try {
zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, 10000, CloudConfig cloudConfig = new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "solr").build();
"http://127.0.0.1", "8983", "solr", 0, 60000, true, new CurrentCoreDescriptorProvider() { zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, cloudConfig, new CurrentCoreDescriptorProvider() {
@Override @Override
public List<CoreDescriptor> getCurrentDescriptors() { public List<CoreDescriptor> getCurrentDescriptors() {
@ -263,33 +262,11 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
} }
@Override @Override
public void load() {}; public void load() {}
@Override
public ConfigSolr getConfig() {
return new ConfigSolr(null, null) {
@Override
public CoresLocator getCoresLocator() {
throw new UnsupportedOperationException();
}
@Override
public PluginInfo getShardHandlerFactoryPluginInfo() {
return null;
}
@Override
protected String getProperty(CfgProp key) {
return null;
}
};
}
@Override @Override
public UpdateShardHandler getUpdateShardHandler() { public UpdateShardHandler getUpdateShardHandler() {
return new UpdateShardHandler(null); return new UpdateShardHandler(UpdateShardHandlerConfig.DEFAULT);
} }
} }

View File

@ -46,7 +46,7 @@ public class TestConfigSets extends SolrTestCaseJ4 {
System.setProperty("configsets", configSetsBaseDir); System.setProperty("configsets", configSetsBaseDir);
SolrResourceLoader loader = new SolrResourceLoader(testDirectory.getAbsolutePath()); SolrResourceLoader loader = new SolrResourceLoader(testDirectory.getAbsolutePath());
CoreContainer container = new CoreContainer(ConfigSolr.fromString(loader, solrxml)); CoreContainer container = new CoreContainer(SolrXmlConfig.fromString(loader, solrxml));
container.load(); container.load();
return container; return container;
@ -115,7 +115,7 @@ public class TestConfigSets extends SolrTestCaseJ4 {
System.setProperty("configsets", csd); System.setProperty("configsets", csd);
SolrResourceLoader loader = new SolrResourceLoader(testDirectory.getAbsolutePath()); SolrResourceLoader loader = new SolrResourceLoader(testDirectory.getAbsolutePath());
CoreContainer container = new CoreContainer(ConfigSolr.fromString(loader, solrxml)); CoreContainer container = new CoreContainer(SolrXmlConfig.fromString(loader, solrxml));
container.load(); container.load();
// We initially don't have a /get handler defined // We initially don't have a /get handler defined

View File

@ -74,7 +74,7 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
private CoreContainer init(Path homeDirectory, String xml) throws Exception { private CoreContainer init(Path homeDirectory, String xml) throws Exception {
SolrResourceLoader loader = new SolrResourceLoader(homeDirectory.toString()); SolrResourceLoader loader = new SolrResourceLoader(homeDirectory.toString());
CoreContainer ret = new CoreContainer(ConfigSolr.fromString(loader, xml)); CoreContainer ret = new CoreContainer(SolrXmlConfig.fromString(loader, xml));
ret.load(); ret.load();
return ret; return ret;
} }
@ -203,7 +203,7 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
System.setProperty("configsets", getFile("solr/configsets").getAbsolutePath()); System.setProperty("configsets", getFile("solr/configsets").getAbsolutePath());
final CoreContainer cc = new CoreContainer(ConfigSolr.fromString(resourceLoader, CONFIGSETS_SOLR_XML), cl); final CoreContainer cc = new CoreContainer(SolrXmlConfig.fromString(resourceLoader, CONFIGSETS_SOLR_XML), cl);
CoreDescriptor badcore = new CoreDescriptor(cc, "badcore", "badcore", "configSet", "nosuchconfigset"); CoreDescriptor badcore = new CoreDescriptor(cc, "badcore", "badcore", "configSet", "nosuchconfigset");
cl.add(badcore); cl.add(badcore);
@ -452,7 +452,7 @@ public class TestCoreContainer extends SolrTestCaseJ4 {
System.setProperty("configsets", getFile("solr/configsets").getAbsolutePath()); System.setProperty("configsets", getFile("solr/configsets").getAbsolutePath());
final CoreContainer cc = new CoreContainer(ConfigSolr.fromString(resourceLoader, CONFIGSETS_SOLR_XML), cl); final CoreContainer cc = new CoreContainer(SolrXmlConfig.fromString(resourceLoader, CONFIGSETS_SOLR_XML), cl);
cl.add(new CoreDescriptor(cc, "col_ok", "col_ok", "configSet", "minimal")); cl.add(new CoreDescriptor(cc, "col_ok", "col_ok", "configSet", "minimal"));
cl.add(new CoreDescriptor(cc, "col_bad", "col_bad", "configSet", "bad-mergepolicy")); cl.add(new CoreDescriptor(cc, "col_bad", "col_bad", "configSet", "bad-mergepolicy"));
cc.load(); cc.load();

View File

@ -52,7 +52,7 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 {
if (alternateCoreDir != null) { if (alternateCoreDir != null) {
xmlStr = xmlStr.replace("<solr>", "<solr> <str name=\"coreRootDirectory\">" + alternateCoreDir + "</str> "); xmlStr = xmlStr.replace("<solr>", "<solr> <str name=\"coreRootDirectory\">" + alternateCoreDir + "</str> ");
} }
File tmpFile = new File(solrHomeDirectory, ConfigSolr.SOLR_XML_FILE); File tmpFile = new File(solrHomeDirectory, SolrXmlConfig.SOLR_XML_FILE);
FileUtils.write(tmpFile, xmlStr, IOUtils.UTF_8); FileUtils.write(tmpFile, xmlStr, IOUtils.UTF_8);
} }
@ -400,10 +400,10 @@ public class TestCoreDiscovery extends SolrTestCaseJ4 {
SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath()); SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath());
ConfigSolr config = ConfigSolr.fromString(loader, "<solr><str name=\"coreRootDirectory\">relative</str></solr>"); NodeConfig config = SolrXmlConfig.fromString(loader, "<solr><str name=\"coreRootDirectory\">relative</str></solr>");
assertThat(config.getCoreRootDirectory(), containsString(solrHomeDirectory.getAbsolutePath())); assertThat(config.getCoreRootDirectory(), containsString(solrHomeDirectory.getAbsolutePath()));
ConfigSolr absConfig = ConfigSolr.fromString(loader, "<solr><str name=\"coreRootDirectory\">/absolute</str></solr>"); NodeConfig absConfig = SolrXmlConfig.fromString(loader, "<solr><str name=\"coreRootDirectory\">/absolute</str></solr>");
assertThat(absConfig.getCoreRootDirectory(), not(containsString(solrHomeDirectory.getAbsolutePath()))); assertThat(absConfig.getCoreRootDirectory(), not(containsString(solrHomeDirectory.getAbsolutePath())));
} }
} }

View File

@ -49,6 +49,30 @@ public class TestLazyCores extends SolrTestCaseJ4 {
private File solrHomeDirectory; private File solrHomeDirectory;
private static CoreDescriptor makeCoreDescriptor(CoreContainer cc, String coreName, String instanceDir, String isTransient, String loadOnStartup) {
return new CoreDescriptor(cc, coreName, instanceDir,
CoreDescriptor.CORE_TRANSIENT, isTransient,
CoreDescriptor.CORE_LOADONSTARTUP, loadOnStartup);
}
private static CoresLocator testCores = new ReadOnlyCoresLocator() {
@Override
public List<CoreDescriptor> discover(CoreContainer cc) {
return ImmutableList.of(
new CoreDescriptor(cc, "collection1", "collection1"),
makeCoreDescriptor(cc, "collectionLazy2", "collection2", "true", "true"),
makeCoreDescriptor(cc, "collectionLazy3", "collection3", "on", "false"),
makeCoreDescriptor(cc, "collectionLazy4", "collection4", "false", "false"),
makeCoreDescriptor(cc, "collectionLazy5", "collection5", "false", "true"),
makeCoreDescriptor(cc, "collectionLazy6", "collection6", "true", "false"),
makeCoreDescriptor(cc, "collectionLazy7", "collection7", "true", "false"),
makeCoreDescriptor(cc, "collectionLazy8", "collection8", "true", "false"),
makeCoreDescriptor(cc, "collectionLazy9", "collection9", "true", "false")
);
}
};
private CoreContainer init() throws Exception { private CoreContainer init() throws Exception {
solrHomeDirectory = createTempDir().toFile(); solrHomeDirectory = createTempDir().toFile();
@ -57,7 +81,8 @@ public class TestLazyCores extends SolrTestCaseJ4 {
} }
SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath()); SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath());
return createCoreContainer(new LazyCoreTestConfig(loader)); NodeConfig config = new NodeConfig.NodeConfigBuilder("testNode", loader).setTransientCacheSize(4).build();
return createCoreContainer(config, testCores);
} }
@Test @Test
@ -539,10 +564,10 @@ public class TestLazyCores extends SolrTestCaseJ4 {
FileUtils.write(solrXml, "<solr/>", Charsets.UTF_8.toString()); FileUtils.write(solrXml, "<solr/>", Charsets.UTF_8.toString());
SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath()); SolrResourceLoader loader = new SolrResourceLoader(solrHomeDirectory.getAbsolutePath());
ConfigSolrXml config = (ConfigSolrXml) ConfigSolr.fromFile(loader, solrXml); NodeConfig config = SolrXmlConfig.fromFile(loader, solrXml);
// OK this should succeed, but at the end we should have recorded a series of errors. // OK this should succeed, but at the end we should have recorded a series of errors.
return createCoreContainer(config); return createCoreContainer(config, new CorePropertiesLocator(config.getCoreRootDirectory()));
} }
// We want to see that the core "heals itself" if an un-corrupted file is written to the directory. // We want to see that the core "heals itself" if an un-corrupted file is written to the directory.
@ -619,51 +644,4 @@ public class TestLazyCores extends SolrTestCaseJ4 {
return StringUtils.join(args, File.separator); return StringUtils.join(args, File.separator);
} }
public static class LazyCoreTestConfig extends ConfigSolr {
public LazyCoreTestConfig(SolrResourceLoader loader) {
super(loader, null);
}
static CoreDescriptor makeCoreDescriptor(CoreContainer cc, String coreName, String instanceDir, String isTransient, String loadOnStartup) {
return new CoreDescriptor(cc, coreName, instanceDir,
CoreDescriptor.CORE_TRANSIENT, isTransient,
CoreDescriptor.CORE_LOADONSTARTUP, loadOnStartup);
}
@Override
public CoresLocator getCoresLocator() {
return new ReadOnlyCoresLocator() {
@Override
public List<CoreDescriptor> discover(CoreContainer cc) {
return ImmutableList.of(
new CoreDescriptor(cc, "collection1", "collection1"),
makeCoreDescriptor(cc, "collectionLazy2", "collection2", "true", "true"),
makeCoreDescriptor(cc, "collectionLazy3", "collection3", "on", "false"),
makeCoreDescriptor(cc, "collectionLazy4", "collection4", "false", "false"),
makeCoreDescriptor(cc, "collectionLazy5", "collection5", "false", "true"),
makeCoreDescriptor(cc, "collectionLazy6", "collection6", "true", "false"),
makeCoreDescriptor(cc, "collectionLazy7", "collection7", "true", "false"),
makeCoreDescriptor(cc, "collectionLazy8", "collection8", "true", "false"),
makeCoreDescriptor(cc, "collectionLazy9", "collection9", "true", "false")
);
}
};
}
@Override
public PluginInfo getShardHandlerFactoryPluginInfo() {
return null;
}
@Override
protected String getProperty(CfgProp key) {
switch (key) {
case SOLR_TRANSIENTCACHESIZE:
return "4";
}
return null;
}
}
} }

View File

@ -22,8 +22,10 @@ import org.apache.commons.io.FileUtils;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.update.UpdateShardHandlerConfig;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule; import org.junit.Rule;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import org.junit.rules.RuleChain; import org.junit.rules.RuleChain;
@ -63,7 +65,9 @@ public class TestSolrXml extends SolrTestCaseJ4 {
File testSrcRoot = new File(SolrTestCaseJ4.TEST_HOME()); File testSrcRoot = new File(SolrTestCaseJ4.TEST_HOME());
FileUtils.copyFile(new File(testSrcRoot, "solr-50-all.xml"), new File(solrHome, "solr.xml")); FileUtils.copyFile(new File(testSrcRoot, "solr-50-all.xml"), new File(solrHome, "solr.xml"));
ConfigSolr cfg = ConfigSolr.fromSolrHome(loader, solrHome.getAbsolutePath()); NodeConfig cfg = SolrXmlConfig.fromSolrHome(loader, solrHome.getAbsolutePath());
CloudConfig ccfg = cfg.getCloudConfig();
UpdateShardHandlerConfig ucfg = cfg.getUpdateShardHandlerConfig();
assertEquals("core admin handler class", "testAdminHandler", cfg.getCoreAdminHandlerClass()); assertEquals("core admin handler class", "testAdminHandler", cfg.getCoreAdminHandlerClass());
assertEquals("collection handler class", "testCollectionsHandler", cfg.getCollectionsHandlerClass()); assertEquals("collection handler class", "testCollectionsHandler", cfg.getCollectionsHandlerClass());
@ -71,13 +75,21 @@ public class TestSolrXml extends SolrTestCaseJ4 {
assertEquals("core load threads", 11, cfg.getCoreLoadThreadCount()); assertEquals("core load threads", 11, cfg.getCoreLoadThreadCount());
assertThat("core root dir", cfg.getCoreRootDirectory(), containsString("testCoreRootDirectory")); assertThat("core root dir", cfg.getCoreRootDirectory(), containsString("testCoreRootDirectory"));
assertEquals("distrib conn timeout", 22, cfg.getDistributedConnectionTimeout()); assertEquals("distrib conn timeout", 22, cfg.getDistributedConnectionTimeout());
assertEquals("distrib conn timeout", 22, cfg.getUpdateShardHandlerConfig().getDistributedConnectionTimeout());
assertEquals("distrib socket timeout", 33, cfg.getDistributedSocketTimeout()); assertEquals("distrib socket timeout", 33, cfg.getDistributedSocketTimeout());
assertEquals("distrib socket timeout", 33, cfg.getUpdateShardHandlerConfig().getDistributedSocketTimeout());
assertEquals("max update conn", 3, cfg.getMaxUpdateConnections()); assertEquals("max update conn", 3, cfg.getMaxUpdateConnections());
assertEquals("max update conn", 3, cfg.getUpdateShardHandlerConfig().getMaxUpdateConnections());
assertEquals("max update conn/host", 37, cfg.getMaxUpdateConnectionsPerHost()); assertEquals("max update conn/host", 37, cfg.getMaxUpdateConnectionsPerHost());
assertEquals("host", "testHost", cfg.getHost()); assertEquals("max update conn/host", 37, cfg.getUpdateShardHandlerConfig().getMaxUpdateConnectionsPerHost());
assertEquals("zk host context", "testHostContext", cfg.getZkHostContext()); assertEquals("distrib conn timeout", 22, ucfg.getDistributedConnectionTimeout());
assertEquals("solr host port", "44", cfg.getSolrHostPort()); assertEquals("distrib socket timeout", 33, ucfg.getDistributedSocketTimeout());
assertEquals("leader vote wait", 55, cfg.getLeaderVoteWait()); assertEquals("max update conn", 3, ucfg.getMaxUpdateConnections());
assertEquals("max update conn/host", 37, ucfg.getMaxUpdateConnectionsPerHost());
assertEquals("host", "testHost", ccfg.getHost());
assertEquals("zk host context", "testHostContext", ccfg.getSolrHostContext());
assertEquals("solr host port", 44, ccfg.getSolrHostPort());
assertEquals("leader vote wait", 55, ccfg.getLeaderVoteWait());
assertEquals("logging class", "testLoggingClass", cfg.getLogWatcherConfig().getLoggingClass()); assertEquals("logging class", "testLoggingClass", cfg.getLogWatcherConfig().getLoggingClass());
assertEquals("log watcher", true, cfg.getLogWatcherConfig().isEnabled()); assertEquals("log watcher", true, cfg.getLogWatcherConfig().isEnabled());
assertEquals("log watcher size", 88, cfg.getLogWatcherConfig().getWatcherSize()); assertEquals("log watcher size", 88, cfg.getLogWatcherConfig().getWatcherSize());
@ -86,8 +98,8 @@ public class TestSolrXml extends SolrTestCaseJ4 {
assertEquals("shardLib", "testSharedLib", cfg.getSharedLibDirectory()); assertEquals("shardLib", "testSharedLib", cfg.getSharedLibDirectory());
assertEquals("schema cache", true, cfg.hasSchemaCache()); assertEquals("schema cache", true, cfg.hasSchemaCache());
assertEquals("trans cache size", 66, cfg.getTransientCacheSize()); assertEquals("trans cache size", 66, cfg.getTransientCacheSize());
assertEquals("zk client timeout", 77, cfg.getZkClientTimeout()); assertEquals("zk client timeout", 77, ccfg.getZkClientTimeout());
assertEquals("zk host", "testZkHost", cfg.getZkHost()); assertEquals("zk host", "testZkHost", ccfg.getZkHost());
} }
// Test a few property substitutions that happen to be in solr-50-all.xml. // Test a few property substitutions that happen to be in solr-50-all.xml.
@ -102,19 +114,23 @@ public class TestSolrXml extends SolrTestCaseJ4 {
File testSrcRoot = new File(SolrTestCaseJ4.TEST_HOME()); File testSrcRoot = new File(SolrTestCaseJ4.TEST_HOME());
FileUtils.copyFile(new File(testSrcRoot, "solr-50-all.xml"), new File(solrHome, "solr.xml")); FileUtils.copyFile(new File(testSrcRoot, "solr-50-all.xml"), new File(solrHome, "solr.xml"));
ConfigSolr cfg = ConfigSolr.fromSolrHome(loader, solrHome.getAbsolutePath()); NodeConfig cfg = SolrXmlConfig.fromSolrHome(loader, solrHome.getAbsolutePath());
assertThat(cfg.getCoreRootDirectory(), containsString("myCoreRoot")); assertThat(cfg.getCoreRootDirectory(), containsString("myCoreRoot"));
assertEquals("solr host port", "8888", cfg.getSolrHostPort()); assertEquals("solr host port", 8888, cfg.getCloudConfig().getSolrHostPort());
assertEquals("schema cache", false, cfg.hasSchemaCache()); assertEquals("schema cache", false, cfg.hasSchemaCache());
} }
public void testExplicitNullGivesDefaults() throws IOException { public void testExplicitNullGivesDefaults() throws IOException {
// 2 diff options, one where the default is in fact null, and one where it isn't String solrXml = "<solr>" +
String solrXml = "<solr><solrcloud><null name=\"host\"/><null name=\"leaderVoteWait\"/></solrcloud></solr>"; "<solrcloud>" +
"<str name=\"host\">host</str>" +
"<int name=\"hostPort\">8983</int>" +
"<str name=\"hostContext\">solr</str>" +
"<null name=\"leaderVoteWait\"/>" +
"</solrcloud></solr>";
ConfigSolr cfg = ConfigSolr.fromString(loader, solrXml); NodeConfig cfg = SolrXmlConfig.fromString(loader, solrXml);
assertEquals("host", null, cfg.getHost()); assertEquals("leaderVoteWait", 180000, cfg.getCloudConfig().getLeaderVoteWait());
assertEquals("leaderVoteWait", 180000, cfg.getLeaderVoteWait());
} }
public void testIntAsLongBad() throws IOException { public void testIntAsLongBad() throws IOException {
@ -122,14 +138,14 @@ public class TestSolrXml extends SolrTestCaseJ4 {
String solrXml = "<solr><long name=\"transientCacheSize\">"+bad+"</long></solr>"; String solrXml = "<solr><long name=\"transientCacheSize\">"+bad+"</long></solr>";
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "Value of '%s' can not be parsed as 'int'", bad)); expectedException.expectMessage("transientCacheSize");
ConfigSolr cfg = ConfigSolr.fromString(loader, solrXml); NodeConfig cfg = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testIntAsLongOk() throws IOException { public void testIntAsLongOk() throws IOException {
int ok = random().nextInt(); int ok = random().nextInt();
String solrXml = "<solr><long name=\"transientCacheSize\">"+ok+"</long></solr>"; String solrXml = "<solr><long name=\"transientCacheSize\">"+ok+"</long></solr>";
ConfigSolr cfg = ConfigSolr.fromString(loader, solrXml); NodeConfig cfg = SolrXmlConfig.fromString(loader, solrXml);
assertEquals(ok, cfg.getTransientCacheSize()); assertEquals(ok, cfg.getTransientCacheSize());
} }
@ -139,8 +155,8 @@ public class TestSolrXml extends SolrTestCaseJ4 {
+ "<solrcloud><bool name=\"genericCoreNodeNames\">false</bool></solrcloud>" + "<solrcloud><bool name=\"genericCoreNodeNames\">false</bool></solrcloud>"
+ "</solr>"; + "</solr>";
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("2 instances of <solrcloud> found in solr.xml"); expectedException.expectMessage("Multiple instances of solrcloud section found in solr.xml");
ConfigSolr cfg = ConfigSolr.fromString(loader, solrXml); NodeConfig cfg = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testMultiLoggingSectionError() throws IOException { public void testMultiLoggingSectionError() throws IOException {
@ -149,8 +165,8 @@ public class TestSolrXml extends SolrTestCaseJ4 {
+ "<logging><str name=\"class\">foo</str></logging>" + "<logging><str name=\"class\">foo</str></logging>"
+ "</solr>"; + "</solr>";
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("2 instances of <logging> found in solr.xml"); expectedException.expectMessage("Multiple instances of logging section found in solr.xml");
ConfigSolr cfg = ConfigSolr.fromString(loader, solrXml); NodeConfig cfg = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testMultiLoggingWatcherSectionError() throws IOException { public void testMultiLoggingWatcherSectionError() throws IOException {
@ -161,23 +177,23 @@ public class TestSolrXml extends SolrTestCaseJ4 {
+ "</logging></solr>"; + "</logging></solr>";
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("3 instances of Logging <watcher> found in solr.xml"); expectedException.expectMessage("Multiple instances of logging/watcher section found in solr.xml");
ConfigSolr cfg = ConfigSolr.fromString(loader, solrXml); NodeConfig cfg = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testValidStringValueWhenBoolTypeIsExpected() throws IOException { public void testValidStringValueWhenBoolTypeIsExpected() throws IOException {
boolean genericCoreNodeNames = random().nextBoolean(); boolean schemaCache = random().nextBoolean();
String solrXml = String.format(Locale.ROOT, "<solr><solrcloud><str name=\"genericCoreNodeNames\">%s</str></solrcloud></solr>", genericCoreNodeNames); String solrXml = String.format(Locale.ROOT, "<solr><str name=\"shareSchema\">%s</str></solr>", schemaCache);
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
assertEquals("gen core node names", genericCoreNodeNames, configSolr.getGenericCoreNodeNames()); assertEquals("gen core node names", schemaCache, nodeConfig.hasSchemaCache());
} }
public void testValidStringValueWhenIntTypeIsExpected() throws IOException { public void testValidStringValueWhenIntTypeIsExpected() throws IOException {
int maxUpdateConnections = random().nextInt(); int maxUpdateConnections = random().nextInt();
String solrXml = String.format(Locale.ROOT, "<solr><solrcloud><str name=\"maxUpdateConnections\">%d</str></solrcloud></solr>", maxUpdateConnections); String solrXml = String.format(Locale.ROOT, "<solr><updateshardhandler><str name=\"maxUpdateConnections\">%d</str></updateshardhandler></solr>", maxUpdateConnections);
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
assertEquals("max update conn", maxUpdateConnections, configSolr.getMaxUpdateConnections()); assertEquals("max update conn", maxUpdateConnections, nodeConfig.getUpdateShardHandlerConfig().getMaxUpdateConnections());
} }
public void testFailAtConfigParseTimeWhenIntTypeIsExpectedAndLongTypeIsGiven() throws IOException { public void testFailAtConfigParseTimeWhenIntTypeIsExpectedAndLongTypeIsGiven() throws IOException {
@ -185,26 +201,8 @@ public class TestSolrXml extends SolrTestCaseJ4 {
String solrXml = String.format(Locale.ROOT, "<solr><solrcloud><long name=\"maxUpdateConnections\">%d</long></solrcloud></solr>", val); String solrXml = String.format(Locale.ROOT, "<solr><solrcloud><long name=\"maxUpdateConnections\">%d</long></solrcloud></solr>", val);
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "Value of '%d' can not be parsed as 'int'", val)); expectedException.expectMessage("Error parsing 'maxUpdateConnections'");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
}
public void testFailAtConfigParseTimeWhenBoolTypeIsExpectedAndLongTypeIsGiven() throws IOException {
long val = random().nextLong();
String solrXml = String.format(Locale.ROOT, "<solr><solrcloud><long name=\"genericCoreNodeNames\">%d</long></solrcloud></solr>", val);
expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "Value of '%d' can not be parsed as 'bool'", val));
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml);
}
public void testFailAtConfigParseTimeWhenBoolTypeIsExpectedAndDoubleTypeIsGiven() throws IOException {
String val = ""+random().nextDouble();
String solrXml = String.format(Locale.ROOT, "<solr><solrcloud><double name=\"genericCoreNodeNames\">%s</double></solrcloud></solr>", val);
expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "Value of '%s' can not be parsed as 'bool'", val));
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenBoolTypeIsExpectedAndValueIsInvalidString() throws IOException { public void testFailAtConfigParseTimeWhenBoolTypeIsExpectedAndValueIsInvalidString() throws IOException {
@ -212,7 +210,7 @@ public class TestSolrXml extends SolrTestCaseJ4 {
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("invalid boolean value: NOT_A_BOOLEAN"); expectedException.expectMessage("invalid boolean value: NOT_A_BOOLEAN");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenIntTypeIsExpectedAndBoolTypeIsGiven() throws IOException { public void testFailAtConfigParseTimeWhenIntTypeIsExpectedAndBoolTypeIsGiven() throws IOException {
@ -223,34 +221,34 @@ public class TestSolrXml extends SolrTestCaseJ4 {
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "Value of 'unknown-option' can not be parsed as 'int': \"%s\"", randomBoolean)); expectedException.expectMessage(String.format(Locale.ROOT, "Value of 'unknown-option' can not be parsed as 'int': \"%s\"", randomBoolean));
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenUnrecognizedSolrCloudOptionWasFound() throws IOException { public void testFailAtConfigParseTimeWhenUnrecognizedSolrCloudOptionWasFound() throws IOException {
String solrXml = "<solr><solrcloud><bool name=\"unknown-option\">true</bool></solrcloud></solr>"; String solrXml = "<solr><solrcloud><str name=\"host\">host</str><int name=\"hostPort\">8983</int><str name=\"hostContext\"></str><bool name=\"unknown-option\">true</bool></solrcloud></solr>";
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("<solrcloud> section of solr.xml contains 1 unknown config parameter(s): [unknown-option]"); expectedException.expectMessage("Unknown configuration parameter in <solrcloud> section of solr.xml: unknown-option");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenUnrecognizedSolrOptionWasFound() throws IOException { public void testFailAtConfigParseTimeWhenUnrecognizedSolrOptionWasFound() throws IOException {
String solrXml = "<solr><bool name=\"unknown-bool-option\">true</bool><str name=\"unknown-str-option\">true</str></solr>"; String solrXml = "<solr><bool name=\"unknown-bool-option\">true</bool><str name=\"unknown-str-option\">true</str></solr>";
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("Main section of solr.xml contains 2 unknown config parameter(s): [unknown-bool-option, unknown-str-option]"); expectedException.expectMessage("Unknown configuration value in solr.xml: unknown-bool-option");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenUnrecognizedLoggingOptionWasFound() throws IOException { public void testFailAtConfigParseTimeWhenUnrecognizedLoggingOptionWasFound() throws IOException {
String solrXml = String.format(Locale.ROOT, "<solr><logging><bool name=\"unknown-option\">%s</bool></logging></solr>", random().nextBoolean()); String solrXml = String.format(Locale.ROOT, "<solr><logging><bool name=\"unknown-option\">%s</bool></logging></solr>", random().nextBoolean());
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage("<logging> section of solr.xml contains 1 unknown config parameter(s): [unknown-option]"); expectedException.expectMessage("Unknown value in logwatcher config: unknown-option");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenLoggingConfigParamsAreDuplicated() throws IOException { public void testFailAtConfigParseTimeWhenLoggingConfigParamsAreDuplicated() throws IOException {
@ -264,9 +262,9 @@ public class TestSolrXml extends SolrTestCaseJ4 {
v1, v2); v1, v2);
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "<logging> section of solr.xml contains duplicated 'class' in solr.xml: [%s, %s]", v1, v2)); expectedException.expectMessage("<logging> section of solr.xml contains duplicated 'class'");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
public void testFailAtConfigParseTimeWhenSolrCloudConfigParamsAreDuplicated() throws IOException { public void testFailAtConfigParseTimeWhenSolrCloudConfigParamsAreDuplicated() throws IOException {
@ -283,11 +281,12 @@ public class TestSolrXml extends SolrTestCaseJ4 {
v1, v2, v3); v1, v2, v3);
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "<solrcloud> section of solr.xml contains duplicated 'zkClientTimeout' in solr.xml: [%s, %s, %s]", v1, v2, v3)); expectedException.expectMessage("<solrcloud> section of solr.xml contains duplicated 'zkClientTimeout'");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml); NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
} }
@Ignore
public void testFailAtConfigParseTimeWhenSolrConfigParamsAreDuplicated() throws IOException { public void testFailAtConfigParseTimeWhenSolrConfigParamsAreDuplicated() throws IOException {
String v1 = ""+random().nextInt(); String v1 = ""+random().nextInt();
String v2 = ""+random().nextInt(); String v2 = ""+random().nextInt();
@ -299,9 +298,36 @@ public class TestSolrXml extends SolrTestCaseJ4 {
v1, v2); v1, v2);
expectedException.expect(SolrException.class); expectedException.expect(SolrException.class);
expectedException.expectMessage(String.format(Locale.ROOT, "Main section of solr.xml contains duplicated 'coreLoadThreads' in solr.xml: [%s, %s]", v1, v2)); expectedException.expectMessage("Main section of solr.xml contains duplicated 'coreLoadThreads'");
NodeConfig nodeConfig = SolrXmlConfig.fromString(loader, solrXml);
}
public void testCloudConfigRequiresHost() throws Exception {
expectedException.expect(SolrException.class);
expectedException.expectMessage("solrcloud section missing required entry 'host'");
SolrXmlConfig.fromString(loader, "<solr><solrcloud></solrcloud></solr>");
}
public void testCloudConfigRequiresHostPort() throws Exception {
expectedException.expect(SolrException.class);
expectedException.expectMessage("solrcloud section missing required entry 'hostPort'");
SolrXmlConfig.fromString(loader, "<solr><solrcloud><str name=\"host\">host</str></solrcloud></solr>");
}
public void testCloudConfigRequiresHostContext() throws Exception {
expectedException.expect(SolrException.class);
expectedException.expectMessage("solrcloud section missing required entry 'hostContext'");
SolrXmlConfig.fromString(loader, "<solr><solrcloud><str name=\"host\">host</str><int name=\"hostPort\">8983</int></solrcloud></solr>");
ConfigSolr configSolr = ConfigSolr.fromString(loader, solrXml);
} }
} }

View File

@ -33,10 +33,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.UpdateParams; import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoresLocator;
import org.apache.solr.core.PluginInfo;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrEventListener; import org.apache.solr.core.SolrEventListener;
import org.apache.solr.search.SolrIndexSearcher; import org.apache.solr.search.SolrIndexSearcher;
@ -73,23 +70,7 @@ public class SolrCmdDistributorTest extends BaseDistributedSearchTestCase {
private UpdateShardHandler updateShardHandler; private UpdateShardHandler updateShardHandler;
public SolrCmdDistributorTest() throws ParserConfigurationException, IOException, SAXException { public SolrCmdDistributorTest() throws ParserConfigurationException, IOException, SAXException {
updateShardHandler = new UpdateShardHandler(new ConfigSolr(null, null) { updateShardHandler = new UpdateShardHandler(UpdateShardHandlerConfig.DEFAULT);
@Override
public CoresLocator getCoresLocator() {
return null;
}
@Override
public PluginInfo getShardHandlerFactoryPluginInfo() {
return null;
}
@Override
protected String getProperty(CfgProp key) {
return null;
}
});
stress = 0; stress = 0;
} }

View File

@ -1,48 +0,0 @@
package org.apache.solr.util;
/*
* 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.
*/
import org.apache.solr.core.ConfigSolr;
import org.apache.solr.core.CoresLocator;
import org.apache.solr.core.PluginInfo;
/**
*
*/
public class MockConfigSolr extends ConfigSolr {
public MockConfigSolr() {
super(null, null);
}
@Override
public CoresLocator getCoresLocator() {
return null;
}
@Override
public PluginInfo getShardHandlerFactoryPluginInfo() {
return null;
}
@Override
protected String getProperty(CfgProp key) {
return null;
}
}

View File

@ -46,9 +46,11 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.util.ObjectReleaseTracker; import org.apache.solr.common.util.ObjectReleaseTracker;
import org.apache.solr.common.util.XML; import org.apache.solr.common.util.XML;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.core.CoresLocator;
import org.apache.solr.core.NodeConfig;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor; import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.SolrXmlConfig;
import org.apache.solr.core.SolrConfig; import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.SolrResourceLoader;
@ -309,14 +311,14 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
*/ */
public static void setupNoCoreTest(File solrHome, String xmlStr) throws Exception { public static void setupNoCoreTest(File solrHome, String xmlStr) throws Exception {
File tmpFile = new File(solrHome, ConfigSolr.SOLR_XML_FILE); File tmpFile = new File(solrHome, SolrXmlConfig.SOLR_XML_FILE);
if (xmlStr == null) { if (xmlStr == null) {
xmlStr = "<solr></solr>"; xmlStr = "<solr></solr>";
} }
FileUtils.write(tmpFile, xmlStr, IOUtils.UTF_8); FileUtils.write(tmpFile, xmlStr, IOUtils.UTF_8);
SolrResourceLoader loader = new SolrResourceLoader(solrHome.getAbsolutePath()); SolrResourceLoader loader = new SolrResourceLoader(solrHome.getAbsolutePath());
h = new TestHarness(ConfigSolr.fromFile(loader, new File(solrHome, "solr.xml"))); h = new TestHarness(SolrXmlConfig.fromFile(loader, new File(solrHome, "solr.xml")));
lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION, "2.2"); lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION, "2.2");
} }
@ -598,15 +600,17 @@ public abstract class SolrTestCaseJ4 extends LuceneTestCase {
return h.getCoreContainer(); return h.getCoreContainer();
} }
public static CoreContainer createCoreContainer(ConfigSolr config) { public static CoreContainer createCoreContainer(NodeConfig config, CoresLocator locator) {
testSolrHome = config.getSolrResourceLoader().getInstanceDir(); testSolrHome = config.getSolrResourceLoader().getInstanceDir();
h = new TestHarness(config); h = new TestHarness(config, locator);
lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION, "2.2"); lrf = h.getRequestFactory("standard", 0, 20, CommonParams.VERSION, "2.2");
return h.getCoreContainer(); return h.getCoreContainer();
} }
public static CoreContainer createCoreContainer(String coreName, String dataDir, String solrConfig, String schema) { public static CoreContainer createCoreContainer(String coreName, String dataDir, String solrConfig, String schema) {
CoreContainer cc = createCoreContainer(new TestHarness.TestConfigSolr(coreName, dataDir, solrConfig, schema)); NodeConfig nodeConfig = TestHarness.buildTestNodeConfig(new SolrResourceLoader(SolrResourceLoader.locateSolrHome()));
CoresLocator locator = new TestHarness.TestCoresLocator(coreName, dataDir, solrConfig, schema);
CoreContainer cc = createCoreContainer(nodeConfig, locator);
h.coreName = coreName; h.coreName = coreName;
return cc; return cc;
} }

View File

@ -23,14 +23,16 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams; import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.NamedList.NamedListEntry; import org.apache.solr.common.util.NamedList.NamedListEntry;
import org.apache.solr.core.ConfigSolr; import org.apache.solr.core.CloudConfig;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.CoreDescriptor; import org.apache.solr.core.CoreDescriptor;
import org.apache.solr.core.CorePropertiesLocator;
import org.apache.solr.core.CoresLocator; import org.apache.solr.core.CoresLocator;
import org.apache.solr.core.PluginInfo; import org.apache.solr.core.NodeConfig;
import org.apache.solr.core.SolrConfig; import org.apache.solr.core.SolrConfig;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.core.SolrXmlConfig;
import org.apache.solr.handler.UpdateRequestHandler; import org.apache.solr.handler.UpdateRequestHandler;
import org.apache.solr.request.LocalSolrQueryRequest; import org.apache.solr.request.LocalSolrQueryRequest;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
@ -41,6 +43,7 @@ import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.IndexSchema; import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.IndexSchemaFactory; import org.apache.solr.schema.IndexSchemaFactory;
import org.apache.solr.servlet.DirectSolrConnection; import org.apache.solr.servlet.DirectSolrConnection;
import org.apache.solr.update.UpdateShardHandlerConfig;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@ -130,8 +133,8 @@ public class TestHarness extends BaseTestHarness {
* @param indexSchema schema resource name * @param indexSchema schema resource name
*/ */
public TestHarness(String coreName, String dataDir, String solrConfig, String indexSchema) { public TestHarness(String coreName, String dataDir, String solrConfig, String indexSchema) {
this(new TestConfigSolr(new SolrResourceLoader(SolrResourceLoader.locateSolrHome()), this(buildTestNodeConfig(new SolrResourceLoader(SolrResourceLoader.locateSolrHome())),
coreName, dataDir, solrConfig, indexSchema)); new TestCoresLocator(coreName, dataDir, solrConfig, indexSchema));
this.coreName = (coreName == null) ? SolrTestCaseJ4.DEFAULT_TEST_CORENAME : coreName; this.coreName = (coreName == null) ? SolrTestCaseJ4.DEFAULT_TEST_CORENAME : coreName;
} }
@ -154,56 +157,41 @@ public class TestHarness extends BaseTestHarness {
* @param solrXml the text of a solrxml * @param solrXml the text of a solrxml
*/ */
public TestHarness(SolrResourceLoader loader, String solrXml) { public TestHarness(SolrResourceLoader loader, String solrXml) {
this(ConfigSolr.fromString(loader, solrXml)); this(SolrXmlConfig.fromString(loader, solrXml));
}
public TestHarness(NodeConfig nodeConfig) {
this(nodeConfig, new CorePropertiesLocator(nodeConfig.getCoreRootDirectory()));
} }
/** /**
* Create a TestHarness using a specific config * Create a TestHarness using a specific config
* @param config the ConfigSolr to use * @param config the ConfigSolr to use
*/ */
public TestHarness(ConfigSolr config) { public TestHarness(NodeConfig config, CoresLocator coresLocator) {
container = new CoreContainer(config); container = new CoreContainer(config, coresLocator);
container.load(); container.load();
updater = new UpdateRequestHandler(); updater = new UpdateRequestHandler();
updater.init(null); updater.init(null);
} }
public static class TestConfigSolr extends ConfigSolr { public static NodeConfig buildTestNodeConfig(SolrResourceLoader loader) {
CloudConfig cloudConfig = new CloudConfig.CloudConfigBuilder(System.getProperty("host"),
final CoresLocator locator; Integer.getInteger("hostPort", 8983),
System.getProperty("hostContext", ""))
public TestConfigSolr(String coreName, String dataDir, String solrConfig, String schema) { .setZkClientTimeout(Integer.getInteger("zkClientTimeout", 30000))
this(new SolrResourceLoader(SolrResourceLoader.locateSolrHome()), coreName, dataDir, solrConfig, schema); .build();
} if (System.getProperty("zkHost") == null)
cloudConfig = null;
public TestConfigSolr(SolrResourceLoader loader, String coreName, String dataDir, String solrConfig, String schema) { UpdateShardHandlerConfig updateShardHandlerConfig
super(loader); = new UpdateShardHandlerConfig(UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONS,
locator = new TestCoresLocator(coreName, dataDir, solrConfig, schema); UpdateShardHandlerConfig.DEFAULT_MAXUPDATECONNECTIONSPERHOST,
} 30000, 30000);
return new NodeConfig.NodeConfigBuilder("testNode", loader)
@Override .setUseSchemaCache(Boolean.getBoolean("shareSchema"))
public CoresLocator getCoresLocator() { .setCloudConfig(cloudConfig)
return locator; .setUpdateShardHandlerConfig(updateShardHandlerConfig)
} .build();
@Override
public PluginInfo getShardHandlerFactoryPluginInfo() {
return null;
}
@Override
protected String getProperty(CfgProp key) {
switch (key) {
case SOLR_HOST: return System.getProperty("host");
case SOLR_HOSTPORT: return System.getProperty("hostPort", "");
case SOLR_HOSTCONTEXT: return System.getProperty("hostContext", "");
case SOLR_DISTRIBUPDATESOTIMEOUT: return "30000";
case SOLR_ZKCLIENTTIMEOUT: return System.getProperty("zkClientTimeout", "30000");
case SOLR_DISTRIBUPDATECONNTIMEOUT: return "30000";
case SOLR_SHARESCHEMA: return System.getProperty("shareSchema", "false");
}
return null;
}
} }
public static class TestCoresLocator extends ReadOnlyCoresLocator { public static class TestCoresLocator extends ReadOnlyCoresLocator {