mirror of https://github.com/apache/lucene.git
SOLR-6533,SOLR-6654
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1636862 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
fd3be6e81a
commit
30db71bbde
|
@ -205,6 +205,8 @@ New Features
|
|||
facet.pivot which refers to a 'tag' local param in one or more stats.field params.
|
||||
(hossman, Vitaliy Zhovtyuk)
|
||||
|
||||
SOLR-6533: Support editing common solrconfig.xml values (Noble Paul)
|
||||
|
||||
Bug Fixes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -20,18 +20,17 @@ package org.apache.solr.cloud;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.ConnectException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.NetworkInterface;
|
||||
import java.net.SocketException;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.net.URLEncoder;
|
||||
import java.net.UnknownHostException;
|
||||
import java.text.MessageFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Enumeration;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
|
@ -42,11 +41,10 @@ import java.util.Set;
|
|||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.http.NoHttpResponseException;
|
||||
import org.apache.http.conn.ConnectTimeoutException;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
|
||||
import org.apache.solr.common.SolrException;
|
||||
|
@ -72,9 +70,11 @@ import org.apache.solr.common.cloud.ZooKeeperException;
|
|||
import org.apache.solr.common.params.CollectionParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.URLUtil;
|
||||
import org.apache.solr.core.CloseHook;
|
||||
import org.apache.solr.core.CoreContainer;
|
||||
import org.apache.solr.core.CoreDescriptor;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.core.SolrResourceLoader;
|
||||
import org.apache.solr.handler.component.ShardHandler;
|
||||
import org.apache.solr.update.UpdateLog;
|
||||
import org.apache.solr.update.UpdateShardHandler;
|
||||
|
@ -83,9 +83,9 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.apache.zookeeper.KeeperException.ConnectionLossException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.apache.zookeeper.KeeperException.SessionExpiredException;
|
||||
import org.apache.zookeeper.WatchedEvent;
|
||||
import org.apache.zookeeper.Watcher;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.noggit.JSONParser;
|
||||
import org.noggit.ObjectBuilder;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -200,11 +200,10 @@ public final class ZkController {
|
|||
|
||||
// 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>();
|
||||
|
||||
|
||||
public ZkController(final CoreContainer cc, String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout, String localHost, String locaHostPort,
|
||||
String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
|
||||
throws InterruptedException, TimeoutException, IOException
|
||||
{
|
||||
String localHostContext, int leaderVoteWait, int leaderConflictResolveWait, boolean genericCoreNodeNames, final CurrentCoreDescriptorProvider registerOnReconnect)
|
||||
throws InterruptedException, TimeoutException, IOException {
|
||||
|
||||
if (cc == null) throw new IllegalArgumentException("CoreContainer cannot be null.");
|
||||
this.cc = cc;
|
||||
|
@ -214,40 +213,41 @@ public final class ZkController {
|
|||
// solr.xml to indicate the root context, instead of hostContext=""
|
||||
// which means the default of "solr"
|
||||
localHostContext = trimLeadingAndTrailingSlashes(localHostContext);
|
||||
|
||||
|
||||
this.zkServerAddress = zkServerAddress;
|
||||
this.localHostPort = locaHostPort;
|
||||
this.localHostContext = localHostContext;
|
||||
this.hostName = normalizeHostName(localHost);
|
||||
this.nodeName = generateNodeName(this.hostName,
|
||||
this.localHostPort,
|
||||
this.localHostContext);
|
||||
this.nodeName = generateNodeName(this.hostName,
|
||||
this.localHostPort,
|
||||
this.localHostContext);
|
||||
|
||||
this.leaderVoteWait = leaderVoteWait;
|
||||
this.leaderConflictResolveWait = leaderConflictResolveWait;
|
||||
|
||||
|
||||
this.clientTimeout = zkClientTimeout;
|
||||
DefaultConnectionStrategy strat = new DefaultConnectionStrategy();
|
||||
String zkACLProviderClass = cc.getConfig().getZkACLProviderClass();
|
||||
ZkACLProvider zkACLProvider = null;
|
||||
if (zkACLProviderClass != null && zkACLProviderClass.trim().length() > 0) {
|
||||
zkACLProvider = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
|
||||
zkACLProvider = cc.getResourceLoader().newInstance(zkACLProviderClass, ZkACLProvider.class);
|
||||
} else {
|
||||
zkACLProvider = new DefaultZkACLProvider();
|
||||
}
|
||||
|
||||
|
||||
String zkCredentialProviderClass = cc.getConfig().getZkCredentialProviderClass();
|
||||
if (zkCredentialProviderClass != null && zkCredentialProviderClass.trim().length() > 0) {
|
||||
strat.setZkCredentialsToAddAutomatically(cc.getResourceLoader().newInstance(zkCredentialProviderClass, ZkCredentialsProvider.class));
|
||||
} else {
|
||||
strat.setZkCredentialsToAddAutomatically(new DefaultZkCredentialsProvider());
|
||||
}
|
||||
|
||||
addOnReconnectListener(getConfigDirListener());
|
||||
|
||||
zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout,
|
||||
zkClientConnectTimeout, strat,
|
||||
// on reconnect, reload cloud info
|
||||
new OnReconnect() {
|
||||
|
||||
|
||||
@Override
|
||||
public void command() {
|
||||
log.info("ZooKeeper session re-connected ... refreshing core states after session expiration.");
|
||||
|
@ -260,32 +260,32 @@ public final class ZkController {
|
|||
// he is involved in the sync, and he certainly may not be
|
||||
// ExecutorUtil.shutdownAndAwaitTermination(cc.getCmdDistribExecutor());
|
||||
// we need to create all of our lost watches
|
||||
|
||||
|
||||
// seems we dont need to do this again...
|
||||
// Overseer.createClientNodes(zkClient, getNodeName());
|
||||
|
||||
|
||||
cc.cancelCoreRecoveries();
|
||||
|
||||
|
||||
registerAllCoresAsDown(registerOnReconnect, false);
|
||||
|
||||
|
||||
if (!zkRunOnly) {
|
||||
ElectionContext context = new OverseerElectionContext(zkClient,
|
||||
overseer, getNodeName());
|
||||
|
||||
|
||||
ElectionContext prevContext = overseerElector.getContext();
|
||||
if (prevContext != null) {
|
||||
prevContext.cancelElection();
|
||||
}
|
||||
|
||||
|
||||
overseerElector.setup(context);
|
||||
overseerElector.joinElection(context, true);
|
||||
}
|
||||
|
||||
|
||||
zkStateReader.createClusterStateWatchersAndUpdate();
|
||||
|
||||
|
||||
// we have to register as live first to pick up docs in the buffer
|
||||
createEphemeralLiveNode();
|
||||
|
||||
|
||||
List<CoreDescriptor> descriptors = registerOnReconnect
|
||||
.getCurrentDescriptors();
|
||||
// re register all descriptors
|
||||
|
@ -314,7 +314,7 @@ public final class ZkController {
|
|||
listener.command();
|
||||
} catch (Exception exc) {
|
||||
// not much we can do here other than warn in the log
|
||||
log.warn("Error when notifying OnReconnect listener "+listener+" after session re-connected.", exc);
|
||||
log.warn("Error when notifying OnReconnect listener " + listener + " after session re-connected.", exc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -330,20 +330,20 @@ public final class ZkController {
|
|||
SolrException.ErrorCode.SERVER_ERROR, "", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}, new BeforeReconnect() {
|
||||
|
||||
@Override
|
||||
public void command() {
|
||||
try {
|
||||
ZkController.this.overseer.close();
|
||||
} catch (Exception e) {
|
||||
log.error("Error trying to stop any Overseer threads", e);
|
||||
}
|
||||
markAllAsNotLeader(registerOnReconnect);
|
||||
}
|
||||
}, zkACLProvider);
|
||||
|
||||
|
||||
@Override
|
||||
public void command() {
|
||||
try {
|
||||
ZkController.this.overseer.close();
|
||||
} catch (Exception e) {
|
||||
log.error("Error trying to stop any Overseer threads", e);
|
||||
}
|
||||
markAllAsNotLeader(registerOnReconnect);
|
||||
}
|
||||
}, zkACLProvider);
|
||||
|
||||
this.overseerJobQueue = Overseer.getInQueue(zkClient);
|
||||
this.overseerCollectionQueue = Overseer.getCollectionQueue(zkClient);
|
||||
this.overseerRunningMap = Overseer.getRunningMap(zkClient);
|
||||
|
@ -352,9 +352,9 @@ public final class ZkController {
|
|||
cmdExecutor = new ZkCmdExecutor(zkClientTimeout);
|
||||
leaderElector = new LeaderElector(zkClient);
|
||||
zkStateReader = new ZkStateReader(zkClient);
|
||||
|
||||
|
||||
this.baseURL = zkStateReader.getBaseUrlForNodeName(this.nodeName);
|
||||
|
||||
|
||||
init(registerOnReconnect);
|
||||
}
|
||||
|
||||
|
@ -1179,7 +1179,7 @@ public final class ZkController {
|
|||
return true;
|
||||
}
|
||||
|
||||
public void unregister(String coreName, CoreDescriptor cd)
|
||||
public void unregister(String coreName, CoreDescriptor cd, String configLocation)
|
||||
throws InterruptedException, KeeperException {
|
||||
final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
|
||||
final String collection = cd.getCloudDescriptor().getCollectionName();
|
||||
|
@ -1200,6 +1200,10 @@ public final class ZkController {
|
|||
boolean removeWatch = true;
|
||||
// if there is no SolrCore which is a member of this collection, remove the watch
|
||||
for (SolrCore solrCore : cc.getCores()) {
|
||||
if (((ZkSolrResourceLoader)solrCore.getResourceLoader()).getCollectionZkPath().equals(configLocation))
|
||||
configLocation = null; //if a core uses this config dir , then set it to null
|
||||
|
||||
|
||||
CloudDescriptor cloudDesc = solrCore.getCoreDescriptor()
|
||||
.getCloudDescriptor();
|
||||
if (cloudDesc != null
|
||||
|
@ -1216,6 +1220,13 @@ public final class ZkController {
|
|||
ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
|
||||
ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
|
||||
overseerJobQueue.offer(ZkStateReader.toJSON(m));
|
||||
|
||||
if(configLocation != null) {
|
||||
synchronized (confDirectoryWatchers) {
|
||||
log.info("This conf directory is no more watched {0}",configLocation);
|
||||
confDirectoryWatchers.remove(configLocation);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void createCollection(String collection) throws KeeperException,
|
||||
|
@ -2098,4 +2109,159 @@ public final class ZkController {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Persists a config file to ZooKeeper using optimistic concurrency.
|
||||
*
|
||||
* @return true on success
|
||||
*/
|
||||
public static boolean persistConfigResourceToZooKeeper( SolrResourceLoader loader, int znodeVersion , String resourceName, byte[] content, boolean createIfNotExists) {
|
||||
final ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader)loader;
|
||||
final ZkController zkController = zkLoader.getZkController();
|
||||
final SolrZkClient zkClient = zkController.getZkClient();
|
||||
final String resourceLocation = zkLoader.getCollectionZkPath() + "/" + resourceName;
|
||||
String errMsg = "Failed to persist resource at {0} - version mismatch";
|
||||
try {
|
||||
try {
|
||||
zkClient.setData(resourceLocation , content,znodeVersion, true);
|
||||
} catch (NoNodeException e) {
|
||||
if(createIfNotExists){
|
||||
try {
|
||||
zkClient.create(resourceLocation,content, CreateMode.PERSISTENT,true);
|
||||
} catch (KeeperException.NodeExistsException nee) {
|
||||
log.info(MessageFormat.format(errMsg,resourceLocation));
|
||||
throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation) + ", retry.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} catch (KeeperException.BadVersionException bve){
|
||||
log.info(MessageFormat.format(errMsg,resourceLocation));
|
||||
throw new ResourceModifiedInZkException(ErrorCode.CONFLICT, MessageFormat.format(errMsg,resourceLocation) + ", retry.");
|
||||
} catch (Exception e) {
|
||||
if (e instanceof InterruptedException) {
|
||||
Thread.currentThread().interrupt(); // Restore the interrupted status
|
||||
}
|
||||
final String msg = "Error persisting resource at " + resourceLocation;
|
||||
log.error(msg, e);
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public static class ResourceModifiedInZkException extends SolrException {
|
||||
public ResourceModifiedInZkException(ErrorCode code, String msg) {
|
||||
super(code, msg);
|
||||
}
|
||||
}
|
||||
|
||||
public void unRegisterConfListener(Runnable listener) {
|
||||
if(listener == null) return;
|
||||
synchronized (confDirectoryWatchers){
|
||||
for (Set<Runnable> runnables : confDirectoryWatchers.values()) {
|
||||
if(runnables != null) runnables.remove(listener);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**This will give a callback to the listener whenever a child is modified in the
|
||||
* conf directory. It is the responsibility of the listener to check if the individual
|
||||
* item of interest has been modified. When the last core which was interested in
|
||||
* this conf directory is gone the listeners will be removed automatically.
|
||||
*/
|
||||
public void registerConfListenerForCore(String confDir,SolrCore core, final Runnable listener){
|
||||
if(listener==null) throw new NullPointerException("listener cannot be null");
|
||||
synchronized (confDirectoryWatchers){
|
||||
if(confDirectoryWatchers.containsKey(confDir)){
|
||||
confDirectoryWatchers.get(confDir).add(listener);
|
||||
core.addCloseHook(new CloseHook() {
|
||||
@Override
|
||||
public void preClose(SolrCore core) {
|
||||
unRegisterConfListener(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postClose(SolrCore core) { }
|
||||
});
|
||||
|
||||
|
||||
} else {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,"This conf directory is not valid");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String , Set<Runnable>> confDirectoryWatchers = new HashMap<>();
|
||||
void watchZKConfDir(final String zkDir) {
|
||||
|
||||
if(!confDirectoryWatchers.containsKey(zkDir)){
|
||||
confDirectoryWatchers.put(zkDir,new HashSet<Runnable>());
|
||||
}else{
|
||||
//it's already watched
|
||||
return;
|
||||
}
|
||||
|
||||
Watcher watcher = new Watcher() {
|
||||
@Override
|
||||
public void process(WatchedEvent event) {
|
||||
try {
|
||||
synchronized (confDirectoryWatchers) {
|
||||
// if this is not among directories to be watched then don't set the watcher anymore
|
||||
if(!confDirectoryWatchers.containsKey(zkDir)) return;
|
||||
}
|
||||
|
||||
if (event.getType() == Event.EventType.NodeChildrenChanged) {
|
||||
synchronized (confDirectoryWatchers) {
|
||||
final Set<Runnable> listeners = confDirectoryWatchers.get(zkDir);
|
||||
if (listeners != null) {
|
||||
new Thread() {
|
||||
@Override
|
||||
public synchronized void run() {
|
||||
//running in a separate thread so that the zk event thread is not
|
||||
// unnecessarily held up
|
||||
for (Runnable listener : listeners) listener.run();
|
||||
}
|
||||
}.start();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
} finally {
|
||||
if (Event.EventType.None.equals(event.getType())) {
|
||||
return;
|
||||
} else {
|
||||
setConfWatcher(zkDir,this);
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
setConfWatcher(zkDir,watcher);
|
||||
}
|
||||
|
||||
private void setConfWatcher(String zkDir, Watcher watcher) {
|
||||
try {
|
||||
zkClient.getChildren(zkDir,watcher,true);
|
||||
} catch (KeeperException e) {
|
||||
log.error("failed to set watcher for conf dir {} ", zkDir);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.interrupted();
|
||||
log.error("failed to set watcher for conf dir {} ", zkDir);
|
||||
}
|
||||
}
|
||||
|
||||
public OnReconnect getConfigDirListener() {
|
||||
return new OnReconnect() {
|
||||
@Override
|
||||
public void command() {
|
||||
synchronized (confDirectoryWatchers){
|
||||
for (String s : confDirectoryWatchers.keySet()) {
|
||||
watchZKConfDir(s);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.solr.common.cloud.ZooKeeperException;
|
|||
import org.apache.solr.core.SolrResourceLoader;
|
||||
import org.apache.solr.schema.ZkIndexSchemaReader;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
/**
|
||||
* ResourceLoader that works with ZooKeeper.
|
||||
|
@ -46,6 +47,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
|
|||
super(instanceDir);
|
||||
this.zkController = zooKeeperController;
|
||||
collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
|
||||
zkController.watchZKConfDir(collectionZkPath);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -61,6 +63,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
|
|||
super(instanceDir, parent, coreProperties);
|
||||
this.zkController = zooKeeperController;
|
||||
collectionZkPath = ZkController.CONFIGS_ZKNODE + "/" + collection;
|
||||
zkController.watchZKConfDir(collectionZkPath);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -78,8 +81,9 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
|
|||
String file = collectionZkPath + "/" + resource;
|
||||
try {
|
||||
if (zkController.pathExists(file)) {
|
||||
byte[] bytes = zkController.getZkClient().getData(file, null, null, true);
|
||||
return new ByteArrayInputStream(bytes);
|
||||
Stat stat = new Stat();
|
||||
byte[] bytes = zkController.getZkClient().getData(file, null, stat, true);
|
||||
return new ZkByteArrayInputStream(bytes, stat);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Error opening " + file, e);
|
||||
|
@ -98,6 +102,24 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
|
|||
return is;
|
||||
}
|
||||
|
||||
public static class ZkByteArrayInputStream extends ByteArrayInputStream{
|
||||
|
||||
private final Stat stat;
|
||||
public ZkByteArrayInputStream(byte[] buf, Stat stat) {
|
||||
super(buf);
|
||||
this.stat = stat;
|
||||
|
||||
}
|
||||
|
||||
public ZkByteArrayInputStream(byte[] buf, int offset, int length, Stat stat) {
|
||||
super(buf, offset, length);
|
||||
this.stat = stat;
|
||||
}
|
||||
public Stat getStat(){
|
||||
return stat;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getConfigDir() {
|
||||
throw new ZooKeeperException(
|
||||
|
|
|
@ -52,6 +52,7 @@ import java.text.ParseException;
|
|||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.SortedSet;
|
||||
|
@ -102,7 +103,7 @@ public class Config {
|
|||
* @param is the resource as a SAX InputSource
|
||||
* @param prefix an optional prefix that will be preprended to all non-absolute xpath expressions
|
||||
*/
|
||||
public Config(SolrResourceLoader loader, String name, InputSource is, String prefix, boolean subProps) throws ParserConfigurationException, IOException, SAXException
|
||||
public Config(SolrResourceLoader loader, String name, InputSource is, String prefix, boolean substituteProps) throws ParserConfigurationException, IOException, SAXException
|
||||
{
|
||||
if( loader == null ) {
|
||||
loader = new SolrResourceLoader( null );
|
||||
|
@ -138,8 +139,8 @@ public class Config {
|
|||
// some XML parsers are broken and don't close the byte stream (but they should according to spec)
|
||||
IOUtils.closeQuietly(is.getByteStream());
|
||||
}
|
||||
if (subProps) {
|
||||
DOMUtil.substituteProperties(doc, loader.getCoreProperties());
|
||||
if (substituteProps) {
|
||||
DOMUtil.substituteProperties(doc, getSubstituteProperties());
|
||||
}
|
||||
} catch (ParserConfigurationException e) {
|
||||
SolrException.log(log, "Exception during parsing file: " + name, e);
|
||||
|
@ -152,7 +153,11 @@ public class Config {
|
|||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected Properties getSubstituteProperties() {
|
||||
return loader.getCoreProperties();
|
||||
}
|
||||
|
||||
public Config(SolrResourceLoader loader, String name, Document doc) {
|
||||
this.prefix = null;
|
||||
this.doc = doc;
|
||||
|
@ -207,7 +212,7 @@ public class Config {
|
|||
}
|
||||
|
||||
public void substituteProperties() {
|
||||
DOMUtil.substituteProperties(doc, loader.getCoreProperties());
|
||||
DOMUtil.substituteProperties(doc, getSubstituteProperties());
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -0,0 +1,257 @@
|
|||
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 java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.text.MessageFormat;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.common.util.StrUtils;
|
||||
import org.noggit.CharArr;
|
||||
import org.noggit.JSONParser;
|
||||
import org.noggit.JSONWriter;
|
||||
import org.noggit.ObjectBuilder;
|
||||
|
||||
public class ConfigOverlay {
|
||||
private final int znodeVersion ;
|
||||
private Map<String, Object> data;
|
||||
private Map<String,Object> props;
|
||||
private Map<String,Object> userProps;
|
||||
|
||||
public ConfigOverlay(Map<String,Object> jsonObj, int znodeVersion){
|
||||
if(jsonObj == null) jsonObj= Collections.EMPTY_MAP;
|
||||
this.znodeVersion = znodeVersion;
|
||||
data = Collections.unmodifiableMap(jsonObj);
|
||||
props = (Map<String, Object>) data.get("props");
|
||||
if(props == null) props= Collections.EMPTY_MAP;
|
||||
userProps = (Map<String, Object>) data.get("userProps");
|
||||
if(userProps == null) userProps= Collections.EMPTY_MAP;
|
||||
|
||||
}
|
||||
public Object getXPathProperty(String xpath){
|
||||
return getXPathProperty(xpath,true);
|
||||
}
|
||||
|
||||
public Object getXPathProperty(String xpath, boolean onlyPrimitive) {
|
||||
List<String> hierarchy = checkEditable(xpath, true, false);
|
||||
if(hierarchy == null) return null;
|
||||
return getObjectByPath(props, onlyPrimitive, hierarchy);
|
||||
}
|
||||
|
||||
public static Object getObjectByPath(Map root, boolean onlyPrimitive, List<String> hierarchy) {
|
||||
Map obj = root;
|
||||
for (int i = 0; i < hierarchy.size(); i++) {
|
||||
String s = hierarchy.get(i);
|
||||
if(i < hierarchy.size()-1){
|
||||
obj = (Map) obj.get(s);
|
||||
if(obj == null) return null;
|
||||
} else {
|
||||
Object val = obj.get(s);
|
||||
if (onlyPrimitive && val instanceof Map) {
|
||||
return null;
|
||||
}
|
||||
return val;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
public ConfigOverlay setUserProperty(String key, Object val){
|
||||
Map copy = new LinkedHashMap(userProps);
|
||||
copy.put(key,val);
|
||||
Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
|
||||
jsonObj.put("userProps", copy);
|
||||
return new ConfigOverlay(jsonObj, znodeVersion);
|
||||
}
|
||||
public ConfigOverlay unsetUserProperty(String key){
|
||||
if(!userProps.containsKey(key)) return this;
|
||||
Map copy = new LinkedHashMap(userProps);
|
||||
copy.remove(key);
|
||||
Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
|
||||
jsonObj.put("userProps", copy);
|
||||
return new ConfigOverlay(jsonObj, znodeVersion);
|
||||
}
|
||||
|
||||
public ConfigOverlay setProperty(String name, Object val) {
|
||||
List<String> hierarchy = checkEditable(name,false, true);
|
||||
Map deepCopy = getDeepCopy(props);
|
||||
Map obj = deepCopy;
|
||||
for (int i = 0; i < hierarchy.size(); i++) {
|
||||
String s = hierarchy.get(i);
|
||||
if (i < hierarchy.size()-1) {
|
||||
if(obj.get(s) == null || (!(obj.get(s) instanceof Map))) {
|
||||
obj.put(s, new LinkedHashMap<>());
|
||||
}
|
||||
obj = (Map) obj.get(s);
|
||||
} else {
|
||||
obj.put(s,val);
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
|
||||
jsonObj.put("props", deepCopy);
|
||||
|
||||
return new ConfigOverlay(jsonObj, znodeVersion);
|
||||
}
|
||||
|
||||
|
||||
|
||||
private Map getDeepCopy(Map map) {
|
||||
return (Map) ZkStateReader.fromJSON(ZkStateReader.toJSON(map));
|
||||
}
|
||||
|
||||
public static final String NOT_EDITABLE = "''{0}'' is not an editable property";
|
||||
|
||||
private List<String> checkEditable(String propName, boolean isXPath, boolean failOnError) {
|
||||
LinkedList<String> hierarchy = new LinkedList<>();
|
||||
if(!isEditableProp(propName, isXPath,hierarchy)) {
|
||||
if(failOnError) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, MessageFormat.format( NOT_EDITABLE,propName));
|
||||
else return null;
|
||||
}
|
||||
return hierarchy;
|
||||
|
||||
}
|
||||
|
||||
public ConfigOverlay unsetProperty(String name) {
|
||||
List<String> hierarchy = checkEditable(name,false, true);
|
||||
Map deepCopy = getDeepCopy(props);
|
||||
Map obj = deepCopy;
|
||||
for (int i = 0; i < hierarchy.size(); i++) {
|
||||
String s = hierarchy.get(i);
|
||||
if (i < hierarchy.size()-1) {
|
||||
if(obj.get(s) == null || (!(obj.get(s) instanceof Map))) {
|
||||
return this;
|
||||
}
|
||||
obj = (Map) obj.get(s);
|
||||
} else {
|
||||
obj.remove(s);
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, Object> jsonObj = new LinkedHashMap<>(this.data);
|
||||
jsonObj.put("props", deepCopy);
|
||||
|
||||
return new ConfigOverlay(jsonObj, znodeVersion);
|
||||
}
|
||||
|
||||
public byte[] toByteArray() {
|
||||
return ZkStateReader.toJSON(data);
|
||||
}
|
||||
|
||||
|
||||
public int getZnodeVersion(){
|
||||
return znodeVersion;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
CharArr out = new CharArr();
|
||||
try {
|
||||
new JSONWriter(out, 2).write(data);
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return out.toString();
|
||||
}
|
||||
|
||||
public Map toOutputFormat() {
|
||||
Map result = new LinkedHashMap();
|
||||
result.put("version",znodeVersion);
|
||||
result.putAll(data);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
public static final String RESOURCE_NAME = "configoverlay.json";
|
||||
|
||||
private static final Long XML_ATTR = 0L;
|
||||
private static final Long XML_NODE = 1L;
|
||||
|
||||
private static Map editable_prop_map ;
|
||||
public static final String MAPPING = "{ updateHandler : {" +
|
||||
" autoCommit : { maxDocs:1, maxTime:1, openSearcher:1 }," +
|
||||
" autoSoftCommit : { maxDocs:1, maxTime :1}," +
|
||||
" commitWithin : {softCommit:1}," +
|
||||
" commitIntervalLowerBound:1," +
|
||||
" indexWriter : {closeWaitsForMerges:1}" +
|
||||
" }," +
|
||||
" query : {" +
|
||||
" filterCache : {class:0, size:0, initialSize:0 , autowarmCount:0 , regenerator:0}," +
|
||||
" queryResultCache :{class:0, size:0, initialSize:0,autowarmCount:0,regenerator:0}," +
|
||||
" documentCache :{class:0, size:0, initialSize:0 ,autowarmCount:0,regenerator:0}," +
|
||||
" fieldValueCache :{class:0, size:0, initialSize:0 ,autowarmCount:0,regenerator:0}" +
|
||||
"}}";
|
||||
static{
|
||||
try {
|
||||
editable_prop_map = (Map)new ObjectBuilder(new JSONParser(new StringReader(
|
||||
MAPPING))).getObject();
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "error parsing mapping ", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public static boolean isEditableProp(String path, boolean isXpath, List<String> hierarchy) {
|
||||
List<String> parts = StrUtils.splitSmart(path, isXpath? '/':'.');
|
||||
Object obj = editable_prop_map;
|
||||
for (int i = 0; i < parts.size(); i++) {
|
||||
String part = parts.get(i);
|
||||
boolean isAttr = isXpath && part.startsWith("@");
|
||||
if(isAttr){
|
||||
part = part.substring(1);
|
||||
}
|
||||
if(hierarchy != null) hierarchy.add(part);
|
||||
if(obj ==null) return false;
|
||||
if(i == parts.size()-1) {
|
||||
if (obj instanceof Map) {
|
||||
Map map = (Map) obj;
|
||||
if(isXpath && isAttr){
|
||||
return XML_ATTR.equals(map.get(part));
|
||||
} else {
|
||||
return XML_ATTR.equals( map.get(part)) || XML_NODE.equals(map.get(part));
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
obj = ((Map) obj).get(part);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
public Map<String, String> getEditableSubProperties(String xpath) {
|
||||
Object o = getObjectByPath(props,false,StrUtils.splitSmart(xpath,'/'));
|
||||
if (o instanceof Map) {
|
||||
return (Map) o;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, Object> getUserProps() {
|
||||
return userProps;
|
||||
}
|
||||
}
|
|
@ -34,6 +34,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
|
||||
import org.apache.solr.cloud.ZkController;
|
||||
import org.apache.solr.cloud.ZkSolrResourceLoader;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
|
@ -683,6 +684,7 @@ public class CoreContainer {
|
|||
// cancel recovery in cloud mode
|
||||
core.getSolrCoreState().cancelRecovery();
|
||||
}
|
||||
String collectionZkPath = core.getResourceLoader() instanceof ZkSolrResourceLoader ? ((ZkSolrResourceLoader)core.getResourceLoader()).getCollectionZkPath() : null;
|
||||
|
||||
core.unloadOnClose(deleteIndexDir, deleteDataDir, deleteInstanceDir);
|
||||
if (close)
|
||||
|
@ -690,7 +692,7 @@ public class CoreContainer {
|
|||
|
||||
if (zkSys.getZkController() != null) {
|
||||
try {
|
||||
zkSys.getZkController().unregister(name, cd);
|
||||
zkSys.getZkController().unregister(name, cd, collectionZkPath);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted while unregistering core [" + name + "] from cloud state");
|
||||
|
|
|
@ -0,0 +1,25 @@
|
|||
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 java.util.Map;
|
||||
|
||||
public interface MapSerializable {
|
||||
public Map<String, Object> toMap();
|
||||
}
|
|
@ -29,7 +29,7 @@ import static java.util.Collections.unmodifiableMap;
|
|||
* An Object which represents a Plugin of any type
|
||||
*
|
||||
*/
|
||||
public class PluginInfo {
|
||||
public class PluginInfo implements MapSerializable{
|
||||
public final String name, className, type;
|
||||
public final NamedList initArgs;
|
||||
public final Map<String, String> attributes;
|
||||
|
@ -92,6 +92,28 @@ public class PluginInfo {
|
|||
List<PluginInfo> l = getChildren(type);
|
||||
return l.isEmpty() ? null:l.get(0);
|
||||
}
|
||||
public Map<String,Object> toMap(){
|
||||
LinkedHashMap m = new LinkedHashMap(attributes);
|
||||
if(initArgs!=null ) m.putAll(initArgs.asMap(3));
|
||||
if(children != null){
|
||||
for (PluginInfo child : children) {
|
||||
Object old = m.get(child.name);
|
||||
if(old == null){
|
||||
m.put(child.name, child.toMap());
|
||||
} else if (old instanceof List) {
|
||||
List list = (List) old;
|
||||
list.add(child.toMap());
|
||||
} else {
|
||||
ArrayList l = new ArrayList();
|
||||
l.add(old);
|
||||
l.add(child.toMap());
|
||||
m.put(child.name,l);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
return m;
|
||||
}
|
||||
|
||||
/**Filter children by type
|
||||
* @param type The type name. must not be null
|
||||
|
|
|
@ -18,12 +18,14 @@
|
|||
package org.apache.solr.core;
|
||||
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.apache.solr.cloud.ZkSolrResourceLoader;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||
import org.apache.solr.handler.component.SearchComponent;
|
||||
import org.apache.solr.request.SolrRequestHandler;
|
||||
import org.apache.solr.response.QueryResponseWriter;
|
||||
|
@ -42,6 +44,8 @@ import org.apache.solr.update.processor.UpdateRequestProcessorChain;
|
|||
import org.apache.solr.util.DOMUtil;
|
||||
import org.apache.solr.util.FileUtils;
|
||||
import org.apache.solr.util.RegexFileFilter;
|
||||
import org.noggit.JSONParser;
|
||||
import org.noggit.ObjectBuilder;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.w3c.dom.Node;
|
||||
|
@ -54,6 +58,8 @@ import javax.xml.xpath.XPathConstants;
|
|||
import java.io.File;
|
||||
import java.io.FileFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.EnumSet;
|
||||
|
@ -62,6 +68,8 @@ import java.util.LinkedHashMap;
|
|||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -76,7 +84,7 @@ import static org.apache.solr.core.SolrConfig.PluginOpts.REQUIRE_NAME;
|
|||
* configuration data for a a Solr instance -- typically found in
|
||||
* "solrconfig.xml".
|
||||
*/
|
||||
public class SolrConfig extends Config {
|
||||
public class SolrConfig extends Config implements MapSerializable{
|
||||
|
||||
public static final Logger log = LoggerFactory.getLogger(SolrConfig.class);
|
||||
|
||||
|
@ -165,6 +173,7 @@ public class SolrConfig extends Config {
|
|||
public SolrConfig(SolrResourceLoader loader, String name, InputSource is)
|
||||
throws ParserConfigurationException, IOException, SAXException {
|
||||
super(loader, name, is, "/config/");
|
||||
getOverlay();//just in case it is not initialized
|
||||
initLibs();
|
||||
luceneMatchVersion = getLuceneVersion("luceneMatchVersion");
|
||||
String indexConfigPrefix;
|
||||
|
@ -242,48 +251,7 @@ public class SolrConfig extends Config {
|
|||
}
|
||||
maxWarmingSearchers = getInt("query/maxWarmingSearchers",Integer.MAX_VALUE);
|
||||
slowQueryThresholdMillis = getInt("query/slowQueryThresholdMillis", -1);
|
||||
|
||||
loadPluginInfo(SolrRequestHandler.class,"requestHandler",
|
||||
REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
|
||||
loadPluginInfo(QParserPlugin.class,"queryParser",
|
||||
REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
|
||||
loadPluginInfo(QueryResponseWriter.class,"queryResponseWriter",
|
||||
REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
|
||||
loadPluginInfo(ValueSourceParser.class,"valueSourceParser",
|
||||
REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
|
||||
loadPluginInfo(TransformerFactory.class,"transformer",
|
||||
REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
|
||||
loadPluginInfo(SearchComponent.class,"searchComponent",
|
||||
REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK);
|
||||
|
||||
// TODO: WTF is up with queryConverter???
|
||||
// it aparently *only* works as a singleton? - SOLR-4304
|
||||
// and even then -- only if there is a single SpellCheckComponent
|
||||
// because of queryConverter.setIndexAnalyzer
|
||||
loadPluginInfo(QueryConverter.class,"queryConverter",
|
||||
REQUIRE_NAME, REQUIRE_CLASS);
|
||||
|
||||
// this is hackish, since it picks up all SolrEventListeners,
|
||||
// regardless of when/how/why they are used (or even if they are
|
||||
// declared outside of the appropriate context) but there's no nice
|
||||
// way around that in the PluginInfo framework
|
||||
loadPluginInfo(SolrEventListener.class, "//listener",
|
||||
REQUIRE_CLASS, MULTI_OK);
|
||||
|
||||
loadPluginInfo(DirectoryFactory.class,"directoryFactory",
|
||||
REQUIRE_CLASS);
|
||||
loadPluginInfo(IndexDeletionPolicy.class,indexConfigPrefix+"/deletionPolicy",
|
||||
REQUIRE_CLASS);
|
||||
loadPluginInfo(CodecFactory.class,"codecFactory",
|
||||
REQUIRE_CLASS);
|
||||
loadPluginInfo(IndexReaderFactory.class,"indexReaderFactory",
|
||||
REQUIRE_CLASS);
|
||||
loadPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain",
|
||||
MULTI_OK);
|
||||
loadPluginInfo(UpdateLog.class,"updateHandler/updateLog");
|
||||
loadPluginInfo(IndexSchemaFactory.class,"schemaFactory",
|
||||
REQUIRE_CLASS);
|
||||
loadPluginInfo(RestManager.class, "restManager");
|
||||
for (SolrPluginInfo plugin : plugins) loadPluginInfo(plugin);
|
||||
updateHandlerInfo = loadUpdatehandlerInfo();
|
||||
|
||||
multipartUploadLimitKB = getInt(
|
||||
|
@ -302,7 +270,6 @@ public class SolrConfig extends Config {
|
|||
addHttpRequestToContext = getBool(
|
||||
"requestDispatcher/requestParsers/@addHttpRequestToContext", false );
|
||||
|
||||
loadPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK);
|
||||
List<PluginInfo> argsInfos = pluginStore.get(InitParams.class.getName()) ;
|
||||
if(argsInfos!=null){
|
||||
Map<String,InitParams> argsMap = new HashMap<>();
|
||||
|
@ -317,6 +284,71 @@ public class SolrConfig extends Config {
|
|||
solrRequestParsers = new SolrRequestParsers(this);
|
||||
Config.log.info("Loaded SolrConfig: " + name);
|
||||
}
|
||||
|
||||
public static List<SolrPluginInfo> plugins = ImmutableList.<SolrPluginInfo>builder()
|
||||
.add(new SolrPluginInfo(SolrRequestHandler.class, "requestHandler", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
|
||||
.add(new SolrPluginInfo(QParserPlugin.class, "queryParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
|
||||
.add(new SolrPluginInfo(QueryResponseWriter.class, "queryResponseWriter", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
|
||||
.add(new SolrPluginInfo(ValueSourceParser.class, "valueSourceParser", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
|
||||
.add(new SolrPluginInfo(TransformerFactory.class, "transformer", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
|
||||
.add(new SolrPluginInfo(SearchComponent.class, "searchComponent", REQUIRE_NAME, REQUIRE_CLASS, MULTI_OK))
|
||||
// TODO: WTF is up with queryConverter???
|
||||
// it aparently *only* works as a singleton? - SOLR-4304
|
||||
// and even then -- only if there is a single SpellCheckComponent
|
||||
// because of queryConverter.setIndexAnalyzer
|
||||
.add(new SolrPluginInfo(QueryConverter.class, "queryConverter", REQUIRE_NAME, REQUIRE_CLASS))
|
||||
// this is hackish, since it picks up all SolrEventListeners,
|
||||
// regardless of when/how/why they are used (or even if they are
|
||||
// declared outside of the appropriate context) but there's no nice
|
||||
// way around that in the PluginInfo framework
|
||||
.add(new SolrPluginInfo(SolrEventListener.class, "//listener", REQUIRE_CLASS, MULTI_OK))
|
||||
.add(new SolrPluginInfo(DirectoryFactory.class, "directoryFactory", REQUIRE_CLASS))
|
||||
.add(new SolrPluginInfo(IndexDeletionPolicy.class, "indexConfig/deletionPolicy", REQUIRE_CLASS))
|
||||
.add(new SolrPluginInfo(CodecFactory.class, "codecFactory", REQUIRE_CLASS))
|
||||
.add(new SolrPluginInfo(IndexReaderFactory.class, "indexReaderFactory", REQUIRE_CLASS))
|
||||
.add(new SolrPluginInfo(UpdateRequestProcessorChain.class,"updateRequestProcessorChain", MULTI_OK))
|
||||
.add(new SolrPluginInfo(UpdateLog.class,"updateHandler/updateLog"))
|
||||
.add(new SolrPluginInfo(IndexSchemaFactory.class, "schemaFactory", REQUIRE_CLASS))
|
||||
.add(new SolrPluginInfo(RestManager.class, "restManager"))
|
||||
.add(new SolrPluginInfo(InitParams.class, InitParams.TYPE, MULTI_OK))
|
||||
.build();
|
||||
|
||||
public static class SolrPluginInfo{
|
||||
|
||||
public final Class clazz;
|
||||
public final String tag;
|
||||
public final Set<PluginOpts> options;
|
||||
|
||||
|
||||
private SolrPluginInfo(Class clz, String tag, PluginOpts... opts) {
|
||||
this.clazz = clz;
|
||||
this.tag = tag;
|
||||
this.options= opts == null? Collections.EMPTY_SET : EnumSet.of(NOOP, opts);
|
||||
}
|
||||
}
|
||||
|
||||
private static ConfigOverlay getConfigOverlay(SolrResourceLoader loader) {
|
||||
InputStream in = null;
|
||||
try {
|
||||
in = loader.openResource(ConfigOverlay.RESOURCE_NAME);
|
||||
} catch (IOException e) {
|
||||
//no problem no overlay.json file
|
||||
return new ConfigOverlay(Collections.EMPTY_MAP,0);
|
||||
}
|
||||
|
||||
try {
|
||||
int version = 0; //will be always 0 for file based resourceloader
|
||||
if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
|
||||
version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
|
||||
}
|
||||
Map m = (Map) ObjectBuilder.getVal(new JSONParser(new InputStreamReader(in)));
|
||||
return new ConfigOverlay(m,version);
|
||||
} catch (Exception e) {
|
||||
throw new SolrException(ErrorCode.SERVER_ERROR,"Error reading config overlay",e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private Map<String,InitParams> initParams = Collections.emptyMap();
|
||||
public Map<String, InitParams> getInitParams() {
|
||||
return initParams;
|
||||
|
@ -333,20 +365,19 @@ public class SolrConfig extends Config {
|
|||
getBool("updateHandler/commitWithin/softCommit",true));
|
||||
}
|
||||
|
||||
private void loadPluginInfo(Class clazz, String tag, PluginOpts... opts) {
|
||||
EnumSet<PluginOpts> options = EnumSet.<PluginOpts>of(NOOP, opts);
|
||||
boolean requireName = options.contains(REQUIRE_NAME);
|
||||
boolean requireClass = options.contains(REQUIRE_CLASS);
|
||||
private void loadPluginInfo(SolrPluginInfo pluginInfo) {
|
||||
boolean requireName = pluginInfo.options.contains(REQUIRE_NAME);
|
||||
boolean requireClass = pluginInfo.options.contains(REQUIRE_CLASS);
|
||||
|
||||
List<PluginInfo> result = readPluginInfos(tag, requireName, requireClass);
|
||||
List<PluginInfo> result = readPluginInfos(pluginInfo.tag, requireName, requireClass);
|
||||
|
||||
if (1 < result.size() && ! options.contains(MULTI_OK)) {
|
||||
if (1 < result.size() && ! pluginInfo.options.contains(MULTI_OK)) {
|
||||
throw new SolrException
|
||||
(SolrException.ErrorCode.SERVER_ERROR,
|
||||
"Found " + result.size() + " configuration sections when at most "
|
||||
+ "1 is allowed matching expression: " + tag);
|
||||
+ "1 is allowed matching expression: " + pluginInfo.tag);
|
||||
}
|
||||
if(!result.isEmpty()) pluginStore.put(clazz.getName(),result);
|
||||
if(!result.isEmpty()) pluginStore.put(pluginInfo.clazz.getName(),result);
|
||||
}
|
||||
|
||||
public List<PluginInfo> readPluginInfos(String tag, boolean requireName, boolean requireClass) {
|
||||
|
@ -411,7 +442,7 @@ public class SolrConfig extends Config {
|
|||
return httpCachingConfig;
|
||||
}
|
||||
|
||||
public static class JmxConfiguration {
|
||||
public static class JmxConfiguration implements MapSerializable{
|
||||
public boolean enabled = false;
|
||||
public String agentId;
|
||||
public String serviceUrl;
|
||||
|
@ -434,9 +465,18 @@ public class SolrConfig extends Config {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> toMap() {
|
||||
LinkedHashMap map = new LinkedHashMap();
|
||||
map.put("agentId",agentId);
|
||||
map.put("serviceUrl",serviceUrl);
|
||||
map.put("rootName",rootName);
|
||||
return map;
|
||||
}
|
||||
}
|
||||
|
||||
public static class HttpCachingConfig {
|
||||
public static class HttpCachingConfig implements MapSerializable{
|
||||
|
||||
/** config xpath prefix for getting HTTP Caching options */
|
||||
private final static String CACHE_PRE
|
||||
|
@ -445,7 +485,15 @@ public class SolrConfig extends Config {
|
|||
/** For extracting Expires "ttl" from <cacheControl> config */
|
||||
private final static Pattern MAX_AGE
|
||||
= Pattern.compile("\\bmax-age=(\\d+)");
|
||||
|
||||
|
||||
@Override
|
||||
public Map<String, Object> toMap() {
|
||||
return ZkNodeProps.makeMap("never304",never304,
|
||||
"etagSeed",etagSeed,
|
||||
"lastModFrom",lastModFrom.name().toLowerCase(Locale.ROOT),
|
||||
"cacheControl",cacheControlHeader);
|
||||
}
|
||||
|
||||
public static enum LastModFrom {
|
||||
OPENTIME, DIRLASTMOD, BOGUS;
|
||||
|
||||
|
@ -505,7 +553,7 @@ public class SolrConfig extends Config {
|
|||
public LastModFrom getLastModFrom() { return lastModFrom; }
|
||||
}
|
||||
|
||||
public static class UpdateHandlerInfo{
|
||||
public static class UpdateHandlerInfo implements MapSerializable{
|
||||
public final String className;
|
||||
public final int autoCommmitMaxDocs,autoCommmitMaxTime,commitIntervalLowerBound,
|
||||
autoSoftCommmitMaxDocs,autoSoftCommmitMaxTime;
|
||||
|
@ -531,7 +579,29 @@ public class SolrConfig extends Config {
|
|||
this.autoSoftCommmitMaxTime = autoSoftCommmitMaxTime;
|
||||
|
||||
this.commitWithinSoftCommit = commitWithinSoftCommit;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public Map<String, Object> toMap() {
|
||||
LinkedHashMap result = new LinkedHashMap();
|
||||
result.put("class",className);
|
||||
result.put("autoCommmitMaxDocs",autoCommmitMaxDocs);
|
||||
result.put("indexWriterCloseWaitsForMerges",indexWriterCloseWaitsForMerges);
|
||||
result.put("openSearcher",openSearcher);
|
||||
result.put("commitIntervalLowerBound",commitIntervalLowerBound);
|
||||
result.put("commitWithinSoftCommit",commitWithinSoftCommit);
|
||||
result.put("autoCommit", ZkNodeProps.makeMap(
|
||||
"maxDocs", autoCommmitMaxDocs,
|
||||
"maxTime",autoCommmitMaxTime,
|
||||
"commitIntervalLowerBound", commitIntervalLowerBound
|
||||
));
|
||||
result.put("autoSoftCommit" ,
|
||||
ZkNodeProps.makeMap("maxDocs", autoSoftCommmitMaxDocs,
|
||||
"maxTime",autoSoftCommmitMaxTime));
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
// public Map<String, List<PluginInfo>> getUpdateProcessorChainInfo() { return updateProcessorChainInfo; }
|
||||
|
@ -619,5 +689,99 @@ public class SolrConfig extends Config {
|
|||
return enableRemoteStreams;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(String path) {
|
||||
return getInt(path, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getInt(String path, int def) {
|
||||
Object v = overlay.getXPathProperty(path);
|
||||
|
||||
Object val = overlay.getXPathProperty(path);
|
||||
if (val != null) return Integer.parseInt(val.toString());
|
||||
return super.getInt(path, def);
|
||||
}
|
||||
@Override
|
||||
public boolean getBool(String path, boolean def) {
|
||||
Object val = overlay.getXPathProperty(path);
|
||||
if (val != null) return Boolean.parseBoolean(val.toString());
|
||||
return super.getBool(path, def);
|
||||
}
|
||||
@Override
|
||||
public Map<String, Object> toMap() {
|
||||
LinkedHashMap result = new LinkedHashMap();
|
||||
result.put("luceneMatchVersion",luceneMatchVersion);
|
||||
result.put("updateHandler", getUpdateHandlerInfo().toMap());
|
||||
Map m = new LinkedHashMap();
|
||||
result.put("query", m);
|
||||
m.put("useFilterForSortedQuery", useFilterForSortedQuery);
|
||||
m.put("queryResultWindowSize", queryResultWindowSize);
|
||||
m.put("queryResultMaxDocsCached", queryResultMaxDocsCached);
|
||||
m.put("enableLazyFieldLoading", enableLazyFieldLoading);
|
||||
m.put("maxBooleanClauses", booleanQueryMaxClauseCount);
|
||||
|
||||
for (SolrPluginInfo plugin : plugins) {
|
||||
List<PluginInfo> infos = getPluginInfos(plugin.clazz.getName());
|
||||
if(infos == null || infos.isEmpty()) continue;
|
||||
String tag = plugin.tag;
|
||||
tag = tag.replace("/","");
|
||||
if(plugin.options.contains(PluginOpts.REQUIRE_NAME)){
|
||||
LinkedHashMap items = new LinkedHashMap();
|
||||
for (PluginInfo info : infos) items.put(info.name, info.toMap());
|
||||
result.put(tag,items);
|
||||
} else {
|
||||
if(plugin.options.contains(MULTI_OK)){
|
||||
ArrayList<Map> l = new ArrayList<>();
|
||||
for (PluginInfo info : infos) l.add(info.toMap());
|
||||
result.put(tag,l);
|
||||
} else {
|
||||
result.put(tag, infos.get(0).toMap());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
addCacheConfig(m,filterCacheConfig,queryResultCacheConfig,documentCacheConfig,fieldValueCacheConfig);
|
||||
if(jmxConfig != null) result.put("jmx",jmxConfig.toMap());
|
||||
m = new LinkedHashMap();
|
||||
result.put("requestDispatcher", m);
|
||||
m.put("handleSelect",handleSelect);
|
||||
if(httpCachingConfig!=null) m.put("httpCaching", httpCachingConfig.toMap());
|
||||
m.put("requestParsers", ZkNodeProps.makeMap("multipartUploadLimitKB",multipartUploadLimitKB,
|
||||
"formUploadLimitKB",formUploadLimitKB,
|
||||
"addHttpRequestToContext",addHttpRequestToContext));
|
||||
if(indexConfig != null) result.put("indexConfig",indexConfig.toMap());
|
||||
|
||||
//TODO there is more to add
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
private void addCacheConfig(Map queryMap, CacheConfig... cache) {
|
||||
if(cache==null)return;
|
||||
for (CacheConfig config : cache) if(config !=null) queryMap.put(config.getNodeName(),config.toMap());
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Properties getSubstituteProperties() {
|
||||
Map<String, Object> p = getOverlay().getUserProps();
|
||||
if(p==null || p.isEmpty()) return super.getSubstituteProperties();
|
||||
Properties result = new Properties(super.getSubstituteProperties());
|
||||
result.putAll(p);
|
||||
return result;
|
||||
}
|
||||
private ConfigOverlay overlay;
|
||||
|
||||
public ConfigOverlay getOverlay() {
|
||||
if(overlay ==null) {
|
||||
overlay = getConfigOverlay(getResourceLoader());
|
||||
}
|
||||
return overlay;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.solr.common.util.SimpleOrderedMap;
|
|||
import org.apache.solr.core.DirectoryFactory.DirContext;
|
||||
import org.apache.solr.handler.RequestHandlerBase;
|
||||
import org.apache.solr.handler.SnapPuller;
|
||||
import org.apache.solr.handler.SolrConfigHandler;
|
||||
import org.apache.solr.handler.UpdateRequestHandler;
|
||||
import org.apache.solr.handler.admin.ShowFileRequestHandler;
|
||||
import org.apache.solr.handler.component.DebugComponent;
|
||||
|
@ -808,6 +809,8 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
|
|||
reqHandlers = new RequestHandlers(this);
|
||||
List<PluginInfo> implicitReqHandlerInfo = new ArrayList<>();
|
||||
UpdateRequestHandler.addImplicits(implicitReqHandlerInfo);
|
||||
SolrConfigHandler.addImplicits(implicitReqHandlerInfo);
|
||||
|
||||
reqHandlers.initHandlersFromConfig(solrConfig, implicitReqHandlerInfo);
|
||||
|
||||
// Handle things that should eventually go away
|
||||
|
|
|
@ -53,8 +53,10 @@ import java.io.Closeable;
|
|||
import java.io.File;
|
||||
import java.io.FileFilter;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URI;
|
||||
|
@ -819,4 +821,36 @@ public class SolrResourceLoader implements ResourceLoader,Closeable
|
|||
public List<SolrInfoMBean> getInfoMBeans(){
|
||||
return Collections.unmodifiableList(infoMBeans);
|
||||
}
|
||||
|
||||
|
||||
public static void persistConfLocally(SolrResourceLoader loader, String resourceName, byte[] content) {
|
||||
// Persist locally
|
||||
File managedSchemaFile = new File(loader.getConfigDir(), resourceName);
|
||||
OutputStreamWriter writer = null;
|
||||
try {
|
||||
File parentDir = managedSchemaFile.getParentFile();
|
||||
if ( ! parentDir.isDirectory()) {
|
||||
if ( ! parentDir.mkdirs()) {
|
||||
final String msg = "Can't create managed schema directory " + parentDir.getAbsolutePath();
|
||||
log.error(msg);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
|
||||
}
|
||||
}
|
||||
final FileOutputStream out = new FileOutputStream(managedSchemaFile);
|
||||
out.write(content);
|
||||
log.info("Upgraded to managed schema at " + managedSchemaFile.getPath());
|
||||
} catch (IOException e) {
|
||||
final String msg = "Error persisting managed schema " + managedSchemaFile;
|
||||
log.error(msg, e);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
|
||||
} finally {
|
||||
org.apache.commons.io.IOUtils.closeQuietly(writer);
|
||||
try {
|
||||
FileUtils.sync(managedSchemaFile);
|
||||
} catch (IOException e) {
|
||||
final String msg = "Error syncing the managed schema file " + managedSchemaFile;
|
||||
log.error(msg, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,345 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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 java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.text.MessageFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.lucene.analysis.util.ResourceLoader;
|
||||
import org.apache.lucene.analysis.util.ResourceLoaderAware;
|
||||
import org.apache.solr.cloud.ZkController;
|
||||
import org.apache.solr.cloud.ZkSolrResourceLoader;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.CoreAdminParams;
|
||||
import org.apache.solr.common.params.MapSolrParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.ContentStream;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.StrUtils;
|
||||
import org.apache.solr.core.CloseHook;
|
||||
import org.apache.solr.core.ConfigOverlay;
|
||||
import org.apache.solr.core.PluginInfo;
|
||||
import org.apache.solr.core.SolrConfig;
|
||||
import org.apache.solr.core.SolrCore;
|
||||
import org.apache.solr.core.SolrInfoMBean;
|
||||
import org.apache.solr.core.SolrResourceLoader;
|
||||
import org.apache.solr.request.LocalSolrQueryRequest;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.request.SolrRequestHandler;
|
||||
import org.apache.solr.response.SolrQueryResponse;
|
||||
import org.apache.solr.schema.SchemaManager;
|
||||
import org.apache.solr.util.CommandOperation;
|
||||
import org.apache.solr.util.plugin.SolrCoreAware;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static java.text.MessageFormat.format;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
|
||||
import static org.apache.solr.core.ConfigOverlay.NOT_EDITABLE;
|
||||
import static org.apache.solr.core.PluginInfo.DEFAULTS;
|
||||
|
||||
public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAware{
|
||||
public static final Logger log = LoggerFactory.getLogger(SolrConfigHandler.class);
|
||||
public static final boolean configEditing_disabled = Boolean.getBoolean("disable.configEdit");
|
||||
|
||||
@Override
|
||||
public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
|
||||
|
||||
setWt(req, "json");
|
||||
String httpMethod = (String) req.getContext().get("httpMethod");
|
||||
Command command = new Command(req, rsp, httpMethod);
|
||||
if("POST".equals(httpMethod)){
|
||||
if(configEditing_disabled) throw new SolrException(SolrException.ErrorCode.FORBIDDEN," solrconfig editing is not enabled");
|
||||
command.handlePOST();
|
||||
} else {
|
||||
command.handleGET();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void inform(final SolrCore core) {
|
||||
if( ! (core.getResourceLoader() instanceof ZkSolrResourceLoader)) return;
|
||||
final ZkSolrResourceLoader zkSolrResourceLoader = (ZkSolrResourceLoader) core.getResourceLoader();
|
||||
if(zkSolrResourceLoader != null){
|
||||
Runnable listener = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
if(core.isClosed()) return;
|
||||
Stat stat = zkSolrResourceLoader.getZkController().getZkClient().exists((zkSolrResourceLoader).getCollectionZkPath() + "/" + ConfigOverlay.RESOURCE_NAME, null, true);
|
||||
if(stat == null) return;
|
||||
if (stat.getVersion() > core.getSolrConfig().getOverlay().getZnodeVersion()) {
|
||||
core.getCoreDescriptor().getCoreContainer().reload(core.getName());
|
||||
}
|
||||
} catch (KeeperException.NoNodeException nne){
|
||||
//no problem
|
||||
} catch (KeeperException e) {
|
||||
log.error("error refreshing solrconfig ", e);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().isInterrupted();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
zkSolrResourceLoader.getZkController().registerConfListenerForCore(zkSolrResourceLoader.getCollectionZkPath(), core,listener);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
private static class Command{
|
||||
private final SolrQueryRequest req;
|
||||
private final SolrQueryResponse resp;
|
||||
private final String method;
|
||||
|
||||
private Command(SolrQueryRequest req, SolrQueryResponse resp, String httpMethod) {
|
||||
this.req = req;
|
||||
this.resp = resp;
|
||||
this.method = httpMethod;
|
||||
}
|
||||
|
||||
private void handleGET() {
|
||||
String path = (String) req.getContext().get("path");
|
||||
if(path == null) path="/config";
|
||||
if("/config/overlay".equals(path)){
|
||||
resp.add("overlay", req.getCore().getSolrConfig().getOverlay().toOutputFormat());
|
||||
return;
|
||||
} else {
|
||||
List<String> parts =StrUtils.splitSmart(path, '/');
|
||||
if(parts.get(0).isEmpty()) parts.remove(0);
|
||||
if(parts.size() == 1) {
|
||||
resp.add("solrConfig", req.getCore().getSolrConfig().toMap());
|
||||
} else{
|
||||
Map<String, Object> m = req.getCore().getSolrConfig().toMap();
|
||||
resp.add("solrConfig", ZkNodeProps.makeMap(parts.get(1),m.get(parts.get(1))));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void handlePOST() throws IOException {
|
||||
Iterable<ContentStream> streams = req.getContentStreams();
|
||||
if(streams == null ){
|
||||
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
|
||||
}
|
||||
try {
|
||||
for (ContentStream stream : streams) {
|
||||
runCommandsTillSuccess(stream);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
resp.setException(e);
|
||||
resp.add(CommandOperation.ERR_MSGS, singletonList(SchemaManager.getErrorStr(e)));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void runCommandsTillSuccess(ContentStream stream) throws IOException {
|
||||
for (;;) {
|
||||
try {
|
||||
handleCommands(stream);
|
||||
break;
|
||||
} catch (ZkController.ResourceModifiedInZkException e) {
|
||||
log.info(e.getMessage());
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void handleCommands( ContentStream stream) throws IOException {
|
||||
ConfigOverlay overlay = req.getCore().getSolrConfig().getOverlay();
|
||||
List<CommandOperation> ops = CommandOperation.parse(stream.getReader());
|
||||
for (CommandOperation op : ops) {
|
||||
if(SET_PROPERTY.equals( op.name) ){
|
||||
overlay = applySetProp(op, overlay);
|
||||
}else if(UNSET_PROPERTY.equals(op.name)){
|
||||
overlay = applyUnset(op,overlay);
|
||||
}else if(SET_USER_PROPERTY.equals(op.name)){
|
||||
overlay = applySetUserProp(op ,overlay);
|
||||
}else if(UNSET_USER_PROPERTY.equals(op.name)){
|
||||
overlay = applyUnsetUserProp(op, overlay);
|
||||
}
|
||||
}
|
||||
List errs = CommandOperation.captureErrors(ops);
|
||||
if (!errs.isEmpty()) {
|
||||
resp.add(CommandOperation.ERR_MSGS,errs);
|
||||
return;
|
||||
}
|
||||
|
||||
SolrResourceLoader loader = req.getCore().getResourceLoader();
|
||||
if (loader instanceof ZkSolrResourceLoader) {
|
||||
ZkController.persistConfigResourceToZooKeeper(loader,overlay.getZnodeVersion(),
|
||||
ConfigOverlay.RESOURCE_NAME,overlay.toByteArray(),true);
|
||||
|
||||
String collectionName = req.getCore().getCoreDescriptor().getCloudDescriptor().getCollectionName();
|
||||
Map map = ZkNodeProps.makeMap(CoreAdminParams.ACTION, CollectionParams.CollectionAction.RELOAD.toString() ,
|
||||
CollectionParams.NAME, collectionName);
|
||||
|
||||
SolrQueryRequest solrQueryRequest = new LocalSolrQueryRequest(req.getCore(), new MapSolrParams(map));
|
||||
SolrQueryResponse tmpResp = new SolrQueryResponse();
|
||||
try {
|
||||
//doing a collection reload
|
||||
req.getCore().getCoreDescriptor().getCoreContainer().getCollectionsHandler().handleRequestBody(solrQueryRequest,tmpResp);
|
||||
} catch (Exception e) {
|
||||
String msg = MessageFormat.format("Unable to reload collection {0}", collectionName);
|
||||
log.error(msg);
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
|
||||
}
|
||||
|
||||
} else {
|
||||
SolrResourceLoader.persistConfLocally(loader, ConfigOverlay.RESOURCE_NAME, overlay.toByteArray());
|
||||
req.getCore().getCoreDescriptor().getCoreContainer().reload(req.getCore().getName());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private ConfigOverlay applySetUserProp(CommandOperation op, ConfigOverlay overlay) {
|
||||
Map<String, Object> m = op.getDataMap();
|
||||
if(op.hasError()) return overlay;
|
||||
for (Map.Entry<String, Object> e : m.entrySet()) {
|
||||
String name = e.getKey();
|
||||
Object val = e.getValue();
|
||||
overlay = overlay.setUserProperty(name, val);
|
||||
}
|
||||
return overlay;
|
||||
}
|
||||
|
||||
private ConfigOverlay applyUnsetUserProp(CommandOperation op, ConfigOverlay overlay) {
|
||||
List<String> name = op.getStrs(CommandOperation.ROOT_OBJ);
|
||||
if(op.hasError()) return overlay;
|
||||
for (String o : name) {
|
||||
if(!overlay.getUserProps().containsKey(o)) {
|
||||
op.addError(format("No such property ''{0}''", name));
|
||||
} else {
|
||||
overlay = overlay.unsetUserProperty(o);
|
||||
}
|
||||
}
|
||||
return overlay;
|
||||
}
|
||||
|
||||
|
||||
|
||||
private ConfigOverlay applyUnset(CommandOperation op, ConfigOverlay overlay) {
|
||||
List<String> name = op.getStrs(CommandOperation.ROOT_OBJ);
|
||||
if(op.hasError()) return overlay;
|
||||
|
||||
for (String o : name) {
|
||||
if(!ConfigOverlay.isEditableProp(o, false, null)) {
|
||||
op.addError(format(NOT_EDITABLE, name));
|
||||
} else {
|
||||
overlay = overlay.unsetProperty(o);
|
||||
}
|
||||
}
|
||||
return overlay;
|
||||
}
|
||||
|
||||
private ConfigOverlay applySetProp(CommandOperation op, ConfigOverlay overlay) {
|
||||
Map<String, Object> m = op.getDataMap();
|
||||
if(op.hasError()) return overlay;
|
||||
for (Map.Entry<String, Object> e : m.entrySet()) {
|
||||
String name = e.getKey();
|
||||
Object val = e.getValue();
|
||||
if(!ConfigOverlay.isEditableProp(name, false, null)) {
|
||||
op.addError(format(NOT_EDITABLE, name));
|
||||
continue;
|
||||
}
|
||||
overlay = overlay.setProperty(name, val);
|
||||
}
|
||||
return overlay;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static void setWt(SolrQueryRequest req, String wt){
|
||||
SolrParams params = req.getParams();
|
||||
if( params.get(CommonParams.WT) != null ) return;//wt is set by user
|
||||
Map<String,String> map = new HashMap<>(1);
|
||||
map.put(CommonParams.WT, wt);
|
||||
map.put("indent", "true");
|
||||
req.setParams(SolrParams.wrapDefaults(params, new MapSolrParams(map)));
|
||||
}
|
||||
|
||||
|
||||
public static void addImplicits(List<PluginInfo> infoList){
|
||||
Map m = makeMap("name", "/config", "class", SolrConfigHandler.class.getName());
|
||||
infoList.add(new PluginInfo(SolrRequestHandler.TYPE, m, new NamedList<>(singletonMap(DEFAULTS, new NamedList())), null));
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public SolrRequestHandler getSubHandler(String path) {
|
||||
if(subPaths.contains(path)) return this;
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
private static Set<String> subPaths = new HashSet<>(Arrays.asList("/overlay",
|
||||
"/query","/jmx","/requestDispatcher"));
|
||||
static {
|
||||
for (SolrConfig.SolrPluginInfo solrPluginInfo : SolrConfig.plugins) subPaths.add("/"+solrPluginInfo.tag.replaceAll("/",""));
|
||||
|
||||
}
|
||||
|
||||
//////////////////////// SolrInfoMBeans methods //////////////////////
|
||||
|
||||
|
||||
@Override
|
||||
public String getDescription() {
|
||||
return "Edit solrconfig.xml";
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String getVersion() {
|
||||
return getClass().getPackage().getSpecificationVersion();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Category getCategory() {
|
||||
return Category.OTHER;
|
||||
}
|
||||
|
||||
|
||||
|
||||
public static final String SET_PROPERTY = "set-property";
|
||||
public static final String UNSET_PROPERTY = "unset-property";
|
||||
public static final String SET_USER_PROPERTY = "set-user-property";
|
||||
public static final String UNSET_USER_PROPERTY = "unset-user-property";
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -587,7 +587,7 @@ public class CoreAdminHandler extends RequestHandlerBase {
|
|||
catch (Exception ex) {
|
||||
if (coreContainer.isZooKeeperAware() && dcore != null && !preExisitingZkEntry) {
|
||||
try {
|
||||
coreContainer.getZkController().unregister(dcore.getName(), dcore);
|
||||
coreContainer.getZkController().unregister(dcore.getName(), dcore,null);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
SolrException.log(log, null, e);
|
||||
|
|
|
@ -58,7 +58,7 @@ public class SolrConfigRestApi extends Application {
|
|||
*/
|
||||
@Override
|
||||
public synchronized Restlet createInboundRoot() {
|
||||
|
||||
/*
|
||||
log.info("createInboundRoot started for /config");
|
||||
|
||||
router.attachDefault(RestManager.ManagedEndpoint.class);
|
||||
|
@ -70,6 +70,7 @@ public class SolrConfigRestApi extends Application {
|
|||
|
||||
log.info("createInboundRoot complete for /config");
|
||||
|
||||
return router;
|
||||
return router;*/
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,9 +17,14 @@
|
|||
|
||||
package org.apache.solr.search;
|
||||
|
||||
import org.apache.solr.common.util.StrUtils;
|
||||
import org.apache.solr.core.MapSerializable;
|
||||
import org.w3c.dom.Node;
|
||||
import org.w3c.dom.NodeList;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.solr.common.SolrException;
|
||||
|
@ -36,7 +41,7 @@ import javax.xml.xpath.XPathConstants;
|
|||
*
|
||||
*
|
||||
*/
|
||||
public class CacheConfig {
|
||||
public class CacheConfig implements MapSerializable{
|
||||
private String nodeName;
|
||||
|
||||
private Class<? extends SolrCache> clazz;
|
||||
|
@ -70,7 +75,7 @@ public class CacheConfig {
|
|||
if (nodes==null || nodes.getLength()==0) return null;
|
||||
CacheConfig[] configs = new CacheConfig[nodes.getLength()];
|
||||
for (int i=0; i<nodes.getLength(); i++) {
|
||||
configs[i] = getConfig(solrConfig, nodes.item(i));
|
||||
configs[i] = getConfig(solrConfig, nodes.item(i).getNodeName(), DOMUtil.toMap(nodes.item(i).getAttributes()), configPath);
|
||||
}
|
||||
return configs;
|
||||
}
|
||||
|
@ -78,15 +83,29 @@ public class CacheConfig {
|
|||
|
||||
public static CacheConfig getConfig(SolrConfig solrConfig, String xpath) {
|
||||
Node node = solrConfig.getNode(xpath, false);
|
||||
return getConfig(solrConfig, node);
|
||||
if(node == null) {
|
||||
Map<String, String> m = solrConfig.getOverlay().getEditableSubProperties(xpath);
|
||||
if(m==null) return null;
|
||||
List<String> parts = StrUtils.splitSmart(xpath, '/');
|
||||
return getConfig(solrConfig,parts.get(parts.size()-1) , Collections.EMPTY_MAP,xpath);
|
||||
}
|
||||
return getConfig(solrConfig, node.getNodeName(),DOMUtil.toMap(node.getAttributes()), xpath);
|
||||
}
|
||||
|
||||
|
||||
public static CacheConfig getConfig(SolrConfig solrConfig, Node node) {
|
||||
if (node==null) return null;
|
||||
public static CacheConfig getConfig(SolrConfig solrConfig, String nodeName, Map<String,String> attrs, String xpath) {
|
||||
CacheConfig config = new CacheConfig();
|
||||
config.nodeName = node.getNodeName();
|
||||
config.args = DOMUtil.toMap(node.getAttributes());
|
||||
config.nodeName = nodeName;
|
||||
config.args = attrs;
|
||||
|
||||
Map<String, String> map = solrConfig.getOverlay().getEditableSubProperties(xpath);
|
||||
if(map != null){
|
||||
HashMap<String, String> mapCopy = new HashMap<>(config.args);
|
||||
for (Map.Entry<String, String> e : map.entrySet()) {
|
||||
mapCopy.put(e.getKey(),String.valueOf(e.getValue()));
|
||||
}
|
||||
config.args = mapCopy;
|
||||
}
|
||||
String nameAttr = config.args.get("name"); // OPTIONAL
|
||||
if (nameAttr==null) {
|
||||
config.args.put("name",config.nodeName);
|
||||
|
@ -94,6 +113,7 @@ public class CacheConfig {
|
|||
|
||||
SolrResourceLoader loader = solrConfig.getResourceLoader();
|
||||
config.cacheImpl = config.args.get("class");
|
||||
if(config.cacheImpl == null) config.cacheImpl = "solr.LRUCache";
|
||||
config.regenImpl = config.args.get("regenerator");
|
||||
config.clazz = loader.findClass(config.cacheImpl, SolrCache.class);
|
||||
if (config.regenImpl != null) {
|
||||
|
@ -116,4 +136,15 @@ public class CacheConfig {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> toMap() {
|
||||
Map result = Collections.unmodifiableMap(args);
|
||||
return result;
|
||||
}
|
||||
|
||||
public String getNodeName() {
|
||||
return nodeName;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -344,7 +344,7 @@ public class SolrDispatchFilter extends BaseSolrFilter {
|
|||
|
||||
// Handle /schema/* and /config/* paths via Restlet
|
||||
if( path.equals("/schema") || path.startsWith("/schema/")
|
||||
|| path.equals("/config") || path.startsWith("/config/")) {
|
||||
/*|| path.equals("/config") || path.startsWith("/config/")*/) {
|
||||
solrReq = parser.parse(core, path, req);
|
||||
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(solrReq, new SolrQueryResponse()));
|
||||
if( path.equals(req.getServletPath()) ) {
|
||||
|
|
|
@ -24,7 +24,9 @@ import org.apache.lucene.util.PrintStreamInfoStream;
|
|||
import org.apache.lucene.util.Version;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.SolrException.ErrorCode;
|
||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.core.MapSerializable;
|
||||
import org.apache.solr.core.SolrConfig;
|
||||
import org.apache.solr.core.PluginInfo;
|
||||
import org.apache.solr.schema.IndexSchema;
|
||||
|
@ -36,12 +38,13 @@ import java.io.File;
|
|||
import java.io.FileOutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* This config object encapsulates IndexWriter config params,
|
||||
* defined in the <indexConfig> section of solrconfig.xml
|
||||
*/
|
||||
public class SolrIndexConfig {
|
||||
public class SolrIndexConfig implements MapSerializable {
|
||||
public static final Logger log = LoggerFactory.getLogger(SolrIndexConfig.class);
|
||||
|
||||
final String defaultMergePolicyClassName;
|
||||
|
@ -173,6 +176,19 @@ public class SolrIndexConfig {
|
|||
|
||||
checkIntegrityAtMerge = solrConfig.getBool(prefix + "/checkIntegrityAtMerge", def.checkIntegrityAtMerge);
|
||||
}
|
||||
@Override
|
||||
public Map<String, Object> toMap() {
|
||||
Map<String, Object> m = ZkNodeProps.makeMap("maxBufferedDocs", maxBufferedDocs,
|
||||
"maxMergeDocs", maxMergeDocs,
|
||||
"maxIndexingThreads", maxIndexingThreads,
|
||||
"mergeFactor", mergeFactor,
|
||||
"ramBufferSizeMB", ramBufferSizeMB,
|
||||
"writeLockTimeout", writeLockTimeout,
|
||||
"lockType", lockType);
|
||||
if(mergeSchedulerInfo != null) m.put("mergeScheduler",mergeSchedulerInfo.toMap());
|
||||
if(mergePolicyInfo != null) m.put("mergeScheduler",mergePolicyInfo.toMap());
|
||||
return m;
|
||||
}
|
||||
|
||||
/*
|
||||
* Assert that assertCondition is true.
|
||||
|
|
|
@ -56,13 +56,13 @@ public class CommandOperation {
|
|||
if (commandData instanceof Map) {
|
||||
return (Map) commandData;
|
||||
}
|
||||
addError(MessageFormat.format("The command {0} should have the values as a json object {key:val} format", name));
|
||||
addError(MessageFormat.format("The command ''{0}'' should have the values as a json object {key:val} format", name));
|
||||
return Collections.EMPTY_MAP;
|
||||
}
|
||||
|
||||
private Object getRootPrimitive(){
|
||||
if (commandData instanceof Map) {
|
||||
errors.add(MessageFormat.format("The value has to be a string for command : {1}",name));
|
||||
errors.add(MessageFormat.format("The value has to be a string for command : ''{0}'' ",name));
|
||||
return null;
|
||||
}
|
||||
return commandData;
|
||||
|
@ -99,7 +99,12 @@ public class CommandOperation {
|
|||
* single value collection is returned
|
||||
*/
|
||||
public List<String> getStrs(String key, List<String> def){
|
||||
Object v = getMapVal(key);
|
||||
Object v = null;
|
||||
if(ROOT_OBJ.equals(key)) {
|
||||
v = getRootPrimitive();
|
||||
} else {
|
||||
v = getMapVal(key);
|
||||
}
|
||||
if(v == null){
|
||||
return def;
|
||||
} else {
|
||||
|
|
|
@ -48,4 +48,11 @@
|
|||
<requestHandler name="/update" class="solr.UpdateRequestHandler"/>
|
||||
<requestHandler name="/replication" class="solr.ReplicationHandler" startup="lazy" />
|
||||
|
||||
<requestHandler name="/dump" class="DumpRequestHandler" initParams="a">
|
||||
<lst name="defaults">
|
||||
<str name="a">${my.custom.variable.a:A}</str>
|
||||
<str name="b">${my.custom.variable.b:B}</str>
|
||||
</lst>
|
||||
</requestHandler>
|
||||
|
||||
</config>
|
||||
|
|
|
@ -0,0 +1,65 @@
|
|||
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 java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
import static org.apache.solr.core.ConfigOverlay.isEditableProp;
|
||||
|
||||
public class TestConfigOverlay extends LuceneTestCase {
|
||||
|
||||
public void testPaths() {
|
||||
assertTrue(isEditableProp("updateHandler/autoCommit/maxDocs", true, null));
|
||||
assertTrue(isEditableProp("updateHandler/autoCommit/maxTime", true, null));
|
||||
assertTrue(isEditableProp("updateHandler/autoCommit/openSearcher", true, null));
|
||||
assertTrue(isEditableProp("updateHandler/autoCommit/maxDocs", true, null));
|
||||
assertTrue(isEditableProp("updateHandler/autoCommit/maxTime", true, null));
|
||||
assertTrue(isEditableProp("updateHandler/commitWithin/softCommit", true, null));
|
||||
assertTrue(isEditableProp("updateHandler/indexWriter/closeWaitsForMerges", true, null));
|
||||
|
||||
assertTrue(isEditableProp("updateHandler.autoCommit.maxDocs", false, null));
|
||||
assertTrue(isEditableProp("updateHandler.autoCommit.maxTime", false, null));
|
||||
assertTrue(isEditableProp("updateHandler.autoCommit.openSearcher", false, null));
|
||||
assertTrue(isEditableProp("updateHandler.autoCommit.maxDocs", false, null));
|
||||
assertTrue(isEditableProp("updateHandler.autoCommit.maxTime", false, null));
|
||||
assertTrue(isEditableProp("updateHandler.commitWithin.softCommit", false, null));
|
||||
assertTrue(isEditableProp("updateHandler.indexWriter.closeWaitsForMerges", false, null));
|
||||
|
||||
assertTrue(isEditableProp("updateHandler/commitIntervalLowerBound", true, null));
|
||||
assertFalse(isEditableProp("updateHandler/commitIntervalLowerBound1", true, null));
|
||||
assertTrue(isEditableProp("query.filterCache.initialSize", false, null));
|
||||
assertFalse(isEditableProp("query.filterCache", false, null));
|
||||
assertTrue(isEditableProp("query/filterCache/@initialSize", true, null));
|
||||
assertFalse(isEditableProp("query/filterCache/@initialSize1", true, null));
|
||||
}
|
||||
|
||||
public void testSetProperty(){
|
||||
ConfigOverlay overlay = new ConfigOverlay(Collections.EMPTY_MAP,0);
|
||||
overlay = overlay.setProperty("query.filterCache.initialSize",100);
|
||||
assertEquals(100, overlay.getXPathProperty("query/filterCache/@initialSize"));
|
||||
Map<String, String> map = overlay.getEditableSubProperties("query/filterCache");
|
||||
assertNotNull(map);
|
||||
assertEquals(1,map.size());
|
||||
assertEquals(100,map.get("initialSize"));
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,145 @@
|
|||
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 java.io.File;
|
||||
import java.io.StringReader;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.util.RestTestBase;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.eclipse.jetty.servlet.ServletHolder;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.noggit.JSONParser;
|
||||
import org.noggit.ObjectBuilder;
|
||||
import org.restlet.ext.servlet.ServerServlet;
|
||||
|
||||
import static org.apache.solr.core.ConfigOverlay.getObjectByPath;
|
||||
|
||||
public class TestSolrConfigHandler extends RestTestBase {
|
||||
private static File tmpSolrHome;
|
||||
private static File tmpConfDir;
|
||||
|
||||
private static final String collection = "collection1";
|
||||
private static final String confDir = collection + "/conf";
|
||||
|
||||
|
||||
@Before
|
||||
public void before() throws Exception {
|
||||
tmpSolrHome = createTempDir().toFile();
|
||||
tmpConfDir = new File(tmpSolrHome, confDir);
|
||||
FileUtils.copyDirectory(new File(TEST_HOME()), tmpSolrHome.getAbsoluteFile());
|
||||
|
||||
final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
|
||||
final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
|
||||
solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
|
||||
extraServlets.put(solrRestApi, "/schema/*"); // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
|
||||
|
||||
System.setProperty("managed.schema.mutable", "true");
|
||||
System.setProperty("enable.update.log", "false");
|
||||
|
||||
createJettyAndHarness(tmpSolrHome.getAbsolutePath(), "solrconfig-managed-schema.xml", "schema-rest.xml",
|
||||
"/solr", true, extraServlets);
|
||||
}
|
||||
|
||||
@After
|
||||
public void after() throws Exception {
|
||||
if (jetty != null) {
|
||||
jetty.stop();
|
||||
jetty = null;
|
||||
}
|
||||
server = null;
|
||||
restTestHarness = null;
|
||||
}
|
||||
|
||||
|
||||
public void testProperty() throws Exception{
|
||||
String payload= "{\n" +
|
||||
" 'set-property' : { 'updateHandler.autoCommit.maxDocs':100, 'updateHandler.autoCommit.maxTime':10 } \n" +
|
||||
" }";
|
||||
RestTestHarness harness = restTestHarness;
|
||||
String response = harness.post("/config?wt=json", SolrTestCaseJ4.json(payload));
|
||||
Map map = (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
|
||||
assertNull(response, map.get("errors"));
|
||||
|
||||
Map m = (Map) getRespMap("/config/overlay?wt=json" ,harness).get("overlay");
|
||||
Map props = (Map) m.get("props");
|
||||
assertNotNull(props);
|
||||
assertEquals("100", String.valueOf(getObjectByPath(props, true, ImmutableList.of("updateHandler", "autoCommit", "maxDocs")) ));
|
||||
assertEquals("10", String.valueOf(getObjectByPath(props, true, ImmutableList.of("updateHandler", "autoCommit", "maxTime")) ));
|
||||
|
||||
m = (Map) getRespMap("/config?wt=json" ,harness).get("solrConfig");
|
||||
assertNotNull(m);
|
||||
|
||||
assertEquals( "100",String.valueOf(getObjectByPath(m, true, ImmutableList.of("updateHandler", "autoCommit", "maxDocs"))));
|
||||
assertEquals( "10",String.valueOf(getObjectByPath(m, true, ImmutableList.of("updateHandler", "autoCommit", "maxTime"))));
|
||||
payload= "{\n" +
|
||||
" 'unset-property' : 'updateHandler.autoCommit.maxDocs'} \n" +
|
||||
" }";
|
||||
response = harness.post("/config?wt=json", SolrTestCaseJ4.json(payload));
|
||||
map = (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
|
||||
assertNull(response, map.get("errors"));
|
||||
|
||||
m = (Map) getRespMap("/config/overlay?wt=json" ,harness).get("overlay");
|
||||
props = (Map) m.get("props");
|
||||
assertNotNull(props);
|
||||
assertNull(getObjectByPath(props, true, ImmutableList.of("updateHandler", "autoCommit", "maxDocs")));
|
||||
assertEquals("10", String.valueOf(getObjectByPath(props, true, ImmutableList.of("updateHandler", "autoCommit", "maxTime"))));
|
||||
}
|
||||
|
||||
public void testUserProp() throws Exception{
|
||||
String payload= "{\n" +
|
||||
" 'set-user-property' : { 'my.custom.variable.a':'MODIFIEDA'," +
|
||||
" 'my.custom.variable.b':'MODIFIEDB' } \n" +
|
||||
" }";
|
||||
RestTestHarness harness = restTestHarness;
|
||||
String response = harness.post("/config?wt=json", SolrTestCaseJ4.json(payload));
|
||||
Map map = (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
|
||||
assertNull(response, map.get("errors"));
|
||||
|
||||
Map m = (Map) getRespMap("/config/overlay?wt=json" ,harness).get("overlay");
|
||||
Map props = (Map) m.get("userProps");
|
||||
assertNotNull(props);
|
||||
assertEquals(props.get("my.custom.variable.a"), "MODIFIEDA");
|
||||
assertEquals(props.get("my.custom.variable.b"),"MODIFIEDB");
|
||||
|
||||
m = (Map) getRespMap("/dump?wt=json&json.nl=map&initArgs=true" ,harness).get("initArgs");
|
||||
|
||||
m = (Map) m.get(PluginInfo.DEFAULTS);
|
||||
assertEquals("MODIFIEDA", m.get("a"));
|
||||
assertEquals("MODIFIEDB", m.get("b"));
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
public static Map getRespMap(String path, RestTestHarness restHarness) throws Exception {
|
||||
String response = restHarness.query(path);
|
||||
return (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,182 @@
|
|||
package org.apache.solr.handler;
|
||||
|
||||
/*
|
||||
* 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 java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.MessageFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.solr.SolrTestCaseJ4;
|
||||
import org.apache.solr.client.solrj.SolrServer;
|
||||
import org.apache.solr.client.solrj.impl.HttpSolrServer;
|
||||
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.core.ConfigOverlay;
|
||||
import org.apache.solr.util.RESTfulServerProvider;
|
||||
import org.apache.solr.util.RestTestHarness;
|
||||
import org.noggit.JSONParser;
|
||||
import org.noggit.ObjectBuilder;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.apache.solr.core.ConfigOverlay.getObjectByPath;
|
||||
import static org.apache.solr.rest.schema.TestBulkSchemaAPI.getAsMap;
|
||||
import static org.noggit.ObjectBuilder.getVal;
|
||||
|
||||
|
||||
public class TestSolrConfigHandlerConcurrent extends AbstractFullDistribZkTestBase {
|
||||
|
||||
|
||||
static final Logger log = LoggerFactory.getLogger(TestSolrConfigHandlerConcurrent.class);
|
||||
private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
|
||||
|
||||
private void setupHarnesses() {
|
||||
for (final SolrServer client : clients) {
|
||||
RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
|
||||
@Override
|
||||
public String getBaseURL() {
|
||||
return ((HttpSolrServer)client).getBaseURL();
|
||||
}
|
||||
});
|
||||
restTestHarnesses.add(harness);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void doTest() throws Exception {
|
||||
|
||||
Map editable_prop_map = (Map) new ObjectBuilder(new JSONParser(new StringReader(
|
||||
ConfigOverlay.MAPPING))).getObject();
|
||||
Map caches = (Map) editable_prop_map.get("query");
|
||||
|
||||
setupHarnesses();
|
||||
List<Thread> threads = new ArrayList<>(caches.size());
|
||||
final List<List> collectErrors = new ArrayList<>();
|
||||
|
||||
for (Object o : caches.entrySet()) {
|
||||
final Map.Entry e = (Map.Entry) o;
|
||||
Thread t = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
ArrayList errs = new ArrayList();
|
||||
collectErrors.add(errs);
|
||||
invokeBulkCall((String)e.getKey() , errs, (Map) e.getValue());
|
||||
} catch (IOException e) {
|
||||
e.printStackTrace();
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
|
||||
}
|
||||
};
|
||||
threads.add(t);
|
||||
t.start();
|
||||
}
|
||||
|
||||
|
||||
for (Thread thread : threads) thread.join();
|
||||
|
||||
boolean success = true;
|
||||
|
||||
for (List e : collectErrors) {
|
||||
if(!e.isEmpty()){
|
||||
success = false;
|
||||
log.error(e.toString());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
assertTrue(collectErrors.toString(), success);
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
private void invokeBulkCall(String cacheName, List<String> errs, Map val) throws Exception {
|
||||
|
||||
String payload = "{" +
|
||||
"'set-property' : {'query.CACHENAME.size':'CACHEVAL1'," +
|
||||
" 'query.CACHENAME.initialSize':'CACHEVAL2'}," +
|
||||
"'set-property': {'query.CACHENAME.autowarmCount' : 'CACHEVAL3'}" +
|
||||
"}";
|
||||
|
||||
Set<String> errmessages = new HashSet<>();
|
||||
for(int i =1;i<2;i++){//make it ahigher number
|
||||
RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
|
||||
payload = payload.replaceAll("CACHENAME" , cacheName);
|
||||
String val1 = String.valueOf(10 * i + 1);
|
||||
payload = payload.replace("CACHEVAL1", val1);
|
||||
String val2 = String.valueOf(10 * i + 2);
|
||||
payload = payload.replace("CACHEVAL2", val2);
|
||||
String val3 = String.valueOf(10 * i + 3);
|
||||
payload = payload.replace("CACHEVAL3", val3);
|
||||
|
||||
String response = publisher.post("/config?wt=json", SolrTestCaseJ4.json(payload));
|
||||
Map map = (Map) getVal(new JSONParser(new StringReader(response)));
|
||||
Object errors = map.get("errors");
|
||||
if(errors!= null){
|
||||
errs.add(new String(ZkStateReader.toJSON(errors), StandardCharsets.UTF_8));
|
||||
return;
|
||||
}
|
||||
|
||||
//get another node
|
||||
RestTestHarness harness = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
|
||||
long startTime = System.nanoTime();
|
||||
boolean success = false;
|
||||
long maxTimeoutSeconds = 10;
|
||||
while ( TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutSeconds) {
|
||||
Thread.sleep(100);
|
||||
errmessages.clear();
|
||||
Map respMap = getAsMap("/config/overlay?wt=json", harness);
|
||||
Map m = (Map) respMap.get("overlay");
|
||||
if(m!= null) m = (Map) m.get("props");
|
||||
if(m == null) {
|
||||
errmessages.add(MessageFormat.format( "overlay does not exist for cache: {} , iteration: {} response {} ", cacheName, i, respMap.toString()));
|
||||
continue;
|
||||
}
|
||||
|
||||
|
||||
Object o = getObjectByPath(m, true, asList("query", cacheName, "size"));
|
||||
if(!val1.equals(o)) errmessages.add(MessageFormat.format("'size' property not set, expected = {}, actual {}", val1,o));
|
||||
|
||||
o = getObjectByPath(m, true, asList("query", cacheName, "initialSize"));
|
||||
if(!val2.equals(o)) errmessages.add(MessageFormat.format("'initialSize' property not set, expected = {}, actual {}", val2,o));
|
||||
|
||||
o = getObjectByPath(m, true, asList("query", cacheName, "autowarmCount"));
|
||||
if(!val3.equals(o)) errmessages.add(MessageFormat.format("'autowarmCount' property not set, expected = {}, actual {}", val3,o));
|
||||
if(errmessages.isEmpty()) break;
|
||||
}
|
||||
if(!errmessages.isEmpty()) {
|
||||
errs.addAll(errmessages);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -26,6 +26,7 @@ import org.apache.solr.common.util.NamedList;
|
|||
import org.apache.solr.core.SolrResourceLoader;
|
||||
import org.apache.solr.rest.ManagedResourceStorage.StorageIO;
|
||||
import org.apache.solr.rest.schema.analysis.ManagedWordSetResource;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.noggit.JSONUtil;
|
||||
import org.restlet.Request;
|
||||
|
@ -74,6 +75,7 @@ public class TestRestManager extends SolrRestletTestBase {
|
|||
* Test RestManager initialization and handling of registered ManagedResources.
|
||||
*/
|
||||
@Test
|
||||
@Ignore
|
||||
public void testManagedResourceRegistrationAndInitialization() throws Exception {
|
||||
// first, we need to register some ManagedResources, which is done with the registry
|
||||
// provided by the SolrResourceLoader
|
||||
|
@ -178,7 +180,7 @@ public class TestRestManager extends SolrRestletTestBase {
|
|||
*/
|
||||
|
||||
// no pre-existing managed config components
|
||||
assertJQ("/config/managed", "/managedResources==[]");
|
||||
// assertJQ("/config/managed", "/managedResources==[]");
|
||||
|
||||
// add a ManagedWordSetResource for managing protected words (for stemming)
|
||||
String newEndpoint = "/schema/analysis/protwords/english";
|
||||
|
@ -223,7 +225,7 @@ public class TestRestManager extends SolrRestletTestBase {
|
|||
assertJDelete(newEndpoint, "/responseHeader/status==0");
|
||||
|
||||
// make sure it's really gone
|
||||
assertJQ("/config/managed", "/managedResources==[]");
|
||||
// assertJQ("/config/managed", "/managedResources==[]");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -215,7 +215,11 @@ public class TestBulkSchemaAPI extends RestTestBase {
|
|||
}
|
||||
|
||||
public static Map getRespMap(RestTestHarness restHarness) throws Exception {
|
||||
String response = restHarness.query("/schema?wt=json");
|
||||
return getAsMap("/schema?wt=json", restHarness);
|
||||
}
|
||||
|
||||
public static Map getAsMap(String uri, RestTestHarness restHarness) throws Exception {
|
||||
String response = restHarness.query(uri);
|
||||
return (Map) ObjectBuilder.getVal(new JSONParser(new StringReader(response)));
|
||||
}
|
||||
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.solr.common.StringUtils;
|
|||
import org.apache.solr.common.cloud.ZkClientConnectionStrategy.ZkUpdate;
|
||||
import org.apache.solr.common.util.ExecutorUtil;
|
||||
import org.apache.solr.common.util.SolrjNamedThreadFactory;
|
||||
import org.apache.zookeeper.AsyncCallback;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
|
|
|
@ -417,7 +417,9 @@ public class TestHarness extends BaseTestHarness {
|
|||
for (int i = 0; i < q.length; i += 2) {
|
||||
entries[i/2] = new NamedListEntry<>(q[i], q[i+1]);
|
||||
}
|
||||
return new LocalSolrQueryRequest(TestHarness.this.getCore(), new NamedList(entries));
|
||||
NamedList nl = new NamedList(entries);
|
||||
if(nl.get("wt" ) == null) nl.add("wt","xml");
|
||||
return new LocalSolrQueryRequest(TestHarness.this.getCore(), nl);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue