SOLR-7125: Allow upload and download of configs via CloudSolrClient

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1660919 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Alan Woodward 2015-02-19 15:58:55 +00:00
parent ae3f1abbe3
commit f013c438ee
21 changed files with 539 additions and 241 deletions

View File

@ -110,7 +110,10 @@ New Features
* SOLR-6832: Queries be served locally rather than being forwarded to another replica.
(Sachin Goyal, Timothy Potter)
* SOLR-1945 : Add support for child docs in DocumentObjectBinder (Noble Paul, Mark Miller)
* SOLR-1945: Add support for child docs in DocumentObjectBinder (Noble Paul, Mark Miller)
* SOLR-7125: You can upload and download configurations via CloudSolrClient
(Alan Woodward)
Bug Fixes
----------------------

View File

@ -17,14 +17,7 @@
package org.apache.solr.hadoop;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import com.google.common.io.Files;
import org.apache.commons.io.FileUtils;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
@ -34,6 +27,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -42,7 +36,13 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.io.Files;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
/**
* Extracts SolrCloud information from ZooKeeper.
@ -150,7 +150,7 @@ final class ZooKeeperInspector {
configName = props.getStr(ZkController.CONFIGNAME_PROP);
}
if (configName != null && !zkClient.exists(ZkController.CONFIGS_ZKNODE + "/" + configName, true)) {
if (configName != null && !zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true)) {
LOG.error("Specified config does not exist in ZooKeeper:" + configName);
throw new IllegalArgumentException("Specified config does not exist in ZooKeeper:"
+ configName);
@ -181,7 +181,8 @@ final class ZooKeeperInspector {
throws IOException, InterruptedException, KeeperException {
File dir = Files.createTempDir();
dir.deleteOnExit();
ZkController.downloadConfigDir(zkClient, configName, dir);
ZkConfigManager configManager = new ZkConfigManager(zkClient);
configManager.downloadConfigDir(configName, dir.toPath());
File confDir = new File(dir, "conf");
if (!confDir.isDirectory()) {
// create a temporary directory with "conf" subdir and mv the config in there. This is

View File

@ -24,6 +24,7 @@ import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.cloud.Aliases;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.StrUtils;
@ -89,7 +90,7 @@ final class ZooKeeperDownloader {
configName = props.getStr(ZkController.CONFIGNAME_PROP);
}
if (configName != null && !zkClient.exists(ZkController.CONFIGS_ZKNODE + "/" + configName, true)) {
if (configName != null && !zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true)) {
LOG.error("Specified config does not exist in ZooKeeper:" + configName);
throw new IllegalArgumentException("Specified config does not exist in ZooKeeper:"
+ configName);
@ -105,7 +106,8 @@ final class ZooKeeperDownloader {
throws IOException, InterruptedException, KeeperException {
Preconditions.checkArgument(dir.exists());
Preconditions.checkArgument(dir.isDirectory());
ZkController.downloadConfigDir(zkClient, configName, dir);
ZkConfigManager manager = new ZkConfigManager(zkClient);
manager.downloadConfigDir(configName, dir.toPath());
File confDir = new File(dir, "conf");
if (!confDir.isDirectory()) {
// create a temporary directory with "conf" subdir and mv the config in there. This is

View File

@ -44,6 +44,7 @@ import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.RoutingRule;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
@ -2619,7 +2620,7 @@ public class OverseerCollectionProcessor implements Runnable, Closeable {
// if there is only one conf, use that
List<String> configNames = null;
try {
configNames = zkStateReader.getZkClient().getChildren(ZkController.CONFIGS_ZKNODE, null, true);
configNames = zkStateReader.getZkClient().getChildren(ZkConfigManager.CONFIGS_ZKNODE, null, true);
if (configNames != null && configNames.size() == 1) {
configName = configNames.get(0);
// no config set named, but there is only 1 - use it
@ -2635,7 +2636,7 @@ public class OverseerCollectionProcessor implements Runnable, Closeable {
}
private boolean validateConfig(String configName) throws KeeperException, InterruptedException {
return zkStateReader.getZkClient().exists(ZkController.CONFIGS_ZKNODE + "/" + configName, true);
return zkStateReader.getZkClient().exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, true);
}
/**

View File

@ -3,29 +3,28 @@ package org.apache.solr.cloud;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.solr.common.cloud.OnReconnect;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.xml.sax.SAXException;
import javax.xml.parsers.ParserConfigurationException;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
import java.util.List;
import java.util.concurrent.TimeoutException;
@ -210,8 +209,8 @@ public class ZkCLI {
System.out.println("A chroot was specified in zkHost but the znode doesn't exist. ");
System.exit(1);
}
ZkController.uploadConfigDir(zkClient, new File(confDir), confName);
ZkConfigManager configManager = new ZkConfigManager(zkClient);
configManager.uploadConfigDir(Paths.get(confDir), confName);
} else if (line.getOptionValue(CMD).equals(DOWNCONFIG)) {
if (!line.hasOption(CONFDIR) || !line.hasOption(CONFNAME)) {
System.out.println("-" + CONFDIR + " and -" + CONFNAME
@ -220,8 +219,8 @@ public class ZkCLI {
}
String confDir = line.getOptionValue(CONFDIR);
String confName = line.getOptionValue(CONFNAME);
ZkController.downloadConfigDir(zkClient, confName, new File(confDir));
ZkConfigManager configManager = new ZkConfigManager(zkClient);
configManager.downloadConfigDir(confName, Paths.get(confDir));
} else if (line.getOptionValue(CMD).equals(LINKCONFIG)) {
if (!line.hasOption(COLLECTION) || !line.hasOption(CONFNAME)) {
System.out.println("-" + COLLECTION + " and -" + CONFNAME

View File

@ -17,7 +17,6 @@ package org.apache.solr.cloud;
* limitations under the License.
*/
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang.StringUtils;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CoreAdminRequest.WaitForState;
@ -38,6 +37,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkACLProvider;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkCredentialsProvider;
import org.apache.solr.common.cloud.ZkNodeProps;
@ -64,7 +64,6 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetAddress;
@ -72,6 +71,8 @@ import java.net.NetworkInterface;
import java.net.URLEncoder;
import java.net.UnknownHostException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Collection;
@ -97,6 +98,14 @@ import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
/**
* Handle ZooKeeper interactions.
*
@ -118,8 +127,6 @@ public final class ZkController {
private final DistributedMap overseerRunningMap;
private final DistributedMap overseerCompletedMap;
private final DistributedMap overseerFailureMap;
public static final String CONFIGS_ZKNODE = "/configs";
public final static String COLLECTION_PARAM_PREFIX="collection.";
public final static String CONFIGNAME_PROP="configName";
@ -504,7 +511,7 @@ public final class ZkController {
*/
public boolean configFileExists(String collection, String fileName)
throws KeeperException, InterruptedException {
Stat stat = zkClient.exists(CONFIGS_ZKNODE + "/" + collection + "/" + fileName, null, true);
Stat stat = zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + collection + "/" + fileName, null, true);
return stat != null;
}
@ -520,7 +527,7 @@ public final class ZkController {
*/
public byte[] getConfigFileData(String zkConfigName, String fileName)
throws KeeperException, InterruptedException {
String zkPath = CONFIGS_ZKNODE + "/" + zkConfigName + "/" + fileName;
String zkPath = ZkConfigManager.CONFIGS_ZKNODE + "/" + zkConfigName + "/" + fileName;
byte[] bytes = zkClient.getData(zkPath, null, null, true);
if (bytes == null) {
log.error("Config file contains no data:" + zkPath);
@ -1239,14 +1246,6 @@ public final class ZkController {
overseerJobQueue.offer(ZkStateReader.toJSON(m));
}
public void uploadToZK(File dir, String zkPath) throws IOException, KeeperException, InterruptedException {
uploadToZK(zkClient, dir, zkPath);
}
public void uploadConfigDir(File dir, String configName) throws IOException, KeeperException, InterruptedException {
uploadToZK(zkClient, dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
}
// convenience for testing
void printLayoutToStdOut() throws KeeperException, InterruptedException {
zkClient.printLayoutToStdOut();
@ -1356,7 +1355,7 @@ public final class ZkController {
// if there is only one conf, use that
try {
configNames = zkClient.getChildren(CONFIGS_ZKNODE, null,
configNames = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE, null,
true);
} catch (NoNodeException e) {
// just keep trying
@ -1454,41 +1453,8 @@ public final class ZkController {
throw new SolrException(ErrorCode.SERVER_ERROR,
"Could not get shard id for core: " + cd.getName());
}
public static void uploadToZK(SolrZkClient zkClient, File dir, String zkPath) throws IOException, KeeperException, InterruptedException {
File[] files = dir.listFiles();
if (files == null) {
throw new IllegalArgumentException("Illegal directory: " + dir);
}
for(File file : files) {
if (!file.getName().startsWith(".")) {
if (!file.isDirectory()) {
zkClient.makePath(zkPath + "/" + file.getName(), file, false, true);
} else {
uploadToZK(zkClient, file, zkPath + "/" + file.getName());
}
}
}
}
public static void downloadFromZK(SolrZkClient zkClient, String zkPath,
File dir) throws IOException, KeeperException, InterruptedException {
List<String> files = zkClient.getChildren(zkPath, null, true);
for (String file : files) {
List<String> children = zkClient.getChildren(zkPath + "/" + file, null, true);
if (children.size() == 0) {
byte[] data = zkClient.getData(zkPath + "/" + file, null, null, true);
dir.mkdirs();
log.info("Write file " + new File(dir, file));
FileUtils.writeByteArrayToFile(new File(dir, file), data);
} else {
downloadFromZK(zkClient, zkPath + "/" + file, new File(dir, file));
}
}
}
public String getCoreNodeName(CoreDescriptor descriptor){
String coreNodeName = descriptor.getCloudDescriptor().getCoreNodeName();
if (coreNodeName == null && !genericCoreNodeNames) {
@ -1498,14 +1464,6 @@ public final class ZkController {
return coreNodeName;
}
public static void uploadConfigDir(SolrZkClient zkClient, File dir, String configName) throws IOException, KeeperException, InterruptedException {
uploadToZK(zkClient, dir, ZkController.CONFIGS_ZKNODE + "/" + configName);
}
public static void downloadConfigDir(SolrZkClient zkClient, String configName, File dir) throws IOException, KeeperException, InterruptedException {
downloadFromZK(zkClient, ZkController.CONFIGS_ZKNODE + "/" + configName, dir);
}
public void preRegister(CoreDescriptor cd ) {
@ -1735,8 +1693,9 @@ public final class ZkController {
/**
* If in SolrCloud mode, upload config sets for each SolrCore in solr.xml.
*/
public static void bootstrapConf(SolrZkClient zkClient, CoreContainer cc, String solrHome) throws IOException,
KeeperException, InterruptedException {
public static void bootstrapConf(SolrZkClient zkClient, CoreContainer cc, String solrHome) throws IOException {
ZkConfigManager configManager = new ZkConfigManager(zkClient);
//List<String> allCoreNames = cfg.getAllCoreNames();
List<CoreDescriptor> cds = cc.getCoresLocator().discover(cc);
@ -1749,9 +1708,9 @@ public final class ZkController {
if (StringUtils.isEmpty(confName))
confName = coreName;
String instanceDir = cd.getInstanceDir();
File udir = new File(instanceDir, "conf");
Path udir = Paths.get(instanceDir).resolve("conf");
log.info("Uploading directory " + udir + " with name " + confName + " for SolrCore " + coreName);
ZkController.uploadConfigDir(zkClient, udir, confName);
configManager.uploadConfigDir(udir, confName);
}
}
@ -2335,4 +2294,5 @@ public final class ZkController {
}
};
}
}

View File

@ -17,6 +17,15 @@ package org.apache.solr.cloud;
* limitations under the License.
*/
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ZkConfigManager;
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;
import java.io.ByteArrayInputStream;
import java.io.File;
import java.io.IOException;
@ -24,14 +33,6 @@ import java.io.InputStream;
import java.util.List;
import java.util.Properties;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
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,7 +47,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
ZkController zooKeeperController) {
super(instanceDir);
this.zkController = zooKeeperController;
configSetZkPath = ZkController.CONFIGS_ZKNODE + "/" + configSet;
configSetZkPath = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet;
zkController.watchZKConfDir(configSetZkPath);
}
@ -61,7 +62,7 @@ public class ZkSolrResourceLoader extends SolrResourceLoader {
Properties coreProperties, ZkController zooKeeperController) {
super(instanceDir, parent, coreProperties);
this.zkController = zooKeeperController;
configSetZkPath = ZkController.CONFIGS_ZKNODE + "/" + configSet;
configSetZkPath = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet;
zkController.watchZKConfDir(configSetZkPath);
}

View File

@ -21,6 +21,7 @@ import org.apache.solr.cloud.CurrentCoreDescriptorProvider;
import org.apache.solr.cloud.SolrZkServer;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.common.util.ExecutorUtil;
@ -29,8 +30,10 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@ -156,12 +159,13 @@ public class ZkContainer {
}
if(confDir != null) {
File dir = new File(confDir);
if(!dir.isDirectory()) {
Path configPath = Paths.get(confDir);
if (!Files.isDirectory(configPath))
throw new IllegalArgumentException("bootstrap_confdir must be a directory of configuration files");
}
String confName = System.getProperty(ZkController.COLLECTION_PARAM_PREFIX+ZkController.CONFIGNAME_PROP, "configuration1");
zkController.uploadConfigDir(dir, confName);
ZkConfigManager configManager = new ZkConfigManager(zkController.getZkClient());
configManager.uploadConfigDir(configPath, confName);
}

View File

@ -47,7 +47,6 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpClientUtil;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
@ -1234,7 +1233,7 @@ public class SolrCLI {
// test to see if that config exists in ZK
System.out.println("Uploading "+confDir.getAbsolutePath()+
" for config "+confname+" to ZooKeeper at "+cloudSolrClient.getZkHost());
ZkController.uploadConfigDir(cloudSolrClient.getZkStateReader().getZkClient(), confDir, confname);
cloudSolrClient.uploadConfig(confDir.toPath(), confname);
}
String baseUrl = cloudSolrClient.getZkStateReader().getBaseUrlForNodeName(firstLiveNode);

View File

@ -1,13 +1,5 @@
package org.apache.solr.cloud;
import java.io.File;
import java.io.UnsupportedEncodingException;
import java.nio.charset.Charset;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.StringUtils;
import org.apache.solr.common.cloud.DefaultZkACLProvider;
@ -28,6 +20,14 @@ import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.UnsupportedEncodingException;
import java.nio.charset.Charset;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with

View File

@ -22,9 +22,9 @@ import java.io.File;
import org.apache.solr.SolrJettyTestBase;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZooKeeperException;
import org.apache.solr.core.CoreContainer;
import org.apache.solr.util.ExternalPaths;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -141,7 +141,7 @@ public class TestZkChroot extends SolrTestCaseJ4 {
cores = CoreContainer.createAndLoad(home, new File(home, "solr.xml"));
assertTrue(
"solrconfig.xml should have been uploaded to zk to the correct config directory",
zkClient.exists(chroot + ZkController.CONFIGS_ZKNODE + "/"
zkClient.exists(chroot + ZkConfigManager.CONFIGS_ZKNODE + "/"
+ configName + "/solrconfig.xml", true));
} finally {
if (cores != null) cores.shutdown();

View File

@ -25,6 +25,7 @@ import org.apache.solr.SolrJettyTestBase;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.util.ExternalPaths;
@ -109,7 +110,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
ZkCLI.main(args);
assertTrue(zkClient.exists(chroot + ZkController.CONFIGS_ZKNODE
assertTrue(zkClient.exists(chroot + ZkConfigManager.CONFIGS_ZKNODE
+ "/collection1", true));
}
@ -201,7 +202,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
ExternalPaths.TECHPRODUCTS_CONFIGSET, "-confname", confsetname};
ZkCLI.main(args);
assertTrue(zkClient.exists(ZkController.CONFIGS_ZKNODE + "/" + confsetname, true));
assertTrue(zkClient.exists(ZkConfigManager.CONFIGS_ZKNODE + "/" + confsetname, true));
// print help
// ZkCLI.main(new String[0]);
@ -225,7 +226,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
ZkCLI.main(args);
File[] files = confDir.listFiles();
List<String> zkFiles = zkClient.getChildren(ZkController.CONFIGS_ZKNODE + "/" + confsetname, null, true);
List<String> zkFiles = zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE + "/" + confsetname, null, true);
assertEquals(files.length, zkFiles.size());
File sourceConfDir = new File(ExternalPaths.TECHPRODUCTS_CONFIGSET);

View File

@ -18,9 +18,9 @@ package org.apache.solr.cloud;
*/
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrJettyTestBase;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.ConfigSolr;
@ -36,7 +36,6 @@ import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.File;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -167,7 +166,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
String actualConfigName = "firstConfig";
zkClient.makePath(ZkController.CONFIGS_ZKNODE + "/" + actualConfigName, true);
zkClient.makePath(ZkConfigManager.CONFIGS_ZKNODE + "/" + actualConfigName, true);
Map<String,Object> props = new HashMap<>();
props.put("configName", actualConfigName);
@ -207,60 +206,6 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
}
@Test
public void testUploadToCloud() throws Exception {
String zkDir = createTempDir("zkData").toFile().getAbsolutePath();
ZkTestServer server = new ZkTestServer(zkDir);
ZkController zkController = null;
boolean testFinished = false;
CoreContainer cc = null;
try {
server.run();
AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
cc = getCoreContainer();
zkController = new ZkController(cc, server.getZkAddress(),
TIMEOUT, 10000, "127.0.0.1", "8983", "solr", 0, 60000, true, new CurrentCoreDescriptorProvider() {
@Override
public List<CoreDescriptor> getCurrentDescriptors() {
// do nothing
return null;
}
});
String solrHome = SolrJettyTestBase.legacyExampleCollection1SolrHome();
zkController.uploadToZK(new File(solrHome + "/collection1/conf"),
ZkController.CONFIGS_ZKNODE + "/config1");
// uploading again should overwrite, not error...
zkController.uploadToZK(new File(solrHome + "/collection1/conf"),
ZkController.CONFIGS_ZKNODE + "/config1");
if (DEBUG) {
zkController.printLayoutToStdOut();
}
testFinished = true;
} finally {
if (!testFinished & zkController != null) {
zkController.getZkClient().printLayoutToStdOut();
}
if (zkController != null) {
zkController.close();
}
if (cc != null) {
cc.shutdown();
}
server.shutdown();
}
}
@Test
public void testGetHostName() throws Exception {
String zkDir = createTempDir("zkData").toFile().getAbsolutePath();

View File

@ -35,11 +35,11 @@ import org.apache.http.util.EntityUtils;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrConfig;
import org.apache.solr.util.RESTfulServerProvider;
@ -94,7 +94,7 @@ public class TestConfigReload extends AbstractFullDistribZkTestBase {
private void reloadTest() throws Exception {
SolrZkClient client = cloudClient.getZkStateReader().getZkClient();
log.info("live_nodes_count : " + cloudClient.getZkStateReader().getClusterState().getLiveNodes());
String confPath = ZkController.CONFIGS_ZKNODE+"/conf1/";
String confPath = ZkConfigManager.CONFIGS_ZKNODE+"/conf1/";
// checkConfReload(client, confPath + ConfigOverlay.RESOURCE_NAME, "overlay");
checkConfReload(client, confPath + SolrConfig.DEFAULT_CONF_FILE,"config", "/config");

View File

@ -57,6 +57,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.ConnectException;
import java.net.SocketException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@ -73,7 +74,6 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeoutException;
/**
* SolrJ client class to communicate with SolrCloud.
@ -451,19 +451,16 @@ public class CloudSolrClient extends SolrClient {
if (zkStateReader == null) {
ZkStateReader zk = null;
try {
zk = new ZkStateReader(zkHost, zkClientTimeout,
zkConnectTimeout);
zk = new ZkStateReader(zkHost, zkClientTimeout, zkConnectTimeout);
zk.createClusterStateWatchersAndUpdate();
zkStateReader = zk;
} catch (InterruptedException e) {
if (zk != null) zk.close();
zk.close();
Thread.currentThread().interrupt();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
} catch (KeeperException | TimeoutException | IOException e) {
if (zk != null) zk.close();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (KeeperException e) {
zk.close();
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
} catch (Exception e) {
if (zk != null) zk.close();
// do not wrap because clients may be relying on the underlying exception being thrown
@ -478,6 +475,26 @@ public class CloudSolrClient extends SolrClient {
this.parallelUpdates = parallelUpdates;
}
/**
* Upload a set of config files to Zookeeper and give it a name
* @param configPath {@link java.nio.file.Path} to the config files
* @param configName the name of the config
* @throws IOException if an IO error occurs
*/
public void uploadConfig(Path configPath, String configName) throws IOException {
zkStateReader.getConfigManager().uploadConfigDir(configPath, configName);
}
/**
* Download a named config from Zookeeper to a location on the filesystem
* @param configName the name of the config
* @param downloadPath the path to write config files to
* @throws IOException if an I/O exception occurs
*/
public void downloadConfig(String configName, Path downloadPath) throws IOException {
zkStateReader.getConfigManager().downloadConfigDir(configName, downloadPath);
}
private NamedList<Object> directUpdate(AbstractUpdateRequest request, ClusterState clusterState) throws SolrServerException {
UpdateRequest updateRequest = (UpdateRequest) request;
ModifiableSolrParams params = (ModifiableSolrParams) request.getParams();

View File

@ -17,23 +17,6 @@ package org.apache.solr.common.cloud;
* the License.
*/
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.StringReader;
import java.io.StringWriter;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import javax.xml.transform.OutputKeys;
import javax.xml.transform.Source;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import org.apache.commons.io.FileUtils;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.StringUtils;
@ -54,6 +37,22 @@ import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.xml.transform.OutputKeys;
import javax.xml.transform.Source;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.StringReader;
import java.io.StringWriter;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
/**
*
* All Solr ZooKeeper interactions should go through this class rather than
@ -723,4 +722,15 @@ public class SolrZkClient implements Closeable {
return zkHost.contains("/");
}
/**
* Check to see if a Throwable is an InterruptedException, and if it is, set the thread interrupt flag
* @param e the Throwable
* @return the Throwable
*/
public static Throwable checkInterrupted(Throwable e) {
if (e instanceof InterruptedException)
Thread.interrupted();
return e;
}
}

View File

@ -0,0 +1,128 @@
/*
* 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.
*/
package org.apache.solr.common.cloud;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.Collections;
import java.util.List;
/**
* Class that manages named configs in Zookeeper
*/
public class ZkConfigManager {
private static final Logger logger = LoggerFactory.getLogger(ZkConfigManager.class);
/** ZkNode where named configs are stored */
public static final String CONFIGS_ZKNODE = "/configs";
private final SolrZkClient zkClient;
/**
* Creates a new ZkConfigManager
* @param zkClient the {@link SolrZkClient} to use
*/
public ZkConfigManager(SolrZkClient zkClient) {
this.zkClient = zkClient;
}
private void uploadToZK(final Path rootPath, final String zkPath) throws IOException {
if (!Files.exists(rootPath))
throw new IOException("Path " + rootPath + " does not exist");
Files.walkFileTree(rootPath, new SimpleFileVisitor<Path>(){
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
String zkNode = zkPath + "/" + rootPath.relativize(file).toString();
try {
zkClient.makePath(zkNode, file.toFile(), false, true);
} catch (KeeperException | InterruptedException e) {
throw new IOException("Error uploading file " + file.toString() + " to zookeeper path " + zkNode,
SolrZkClient.checkInterrupted(e));
}
return FileVisitResult.CONTINUE;
}
});
}
private void downloadFromZK(String zkPath, Path dir) throws IOException {
try {
List<String> files = zkClient.getChildren(zkPath, null, true);
Files.createDirectories(dir);
for (String file : files) {
List<String> children = zkClient.getChildren(zkPath + "/" + file, null, true);
if (children.size() == 0) {
byte[] data = zkClient.getData(zkPath + "/" + file, null, null, true);
Path filename = dir.resolve(file);
logger.info("Writing file {}", filename);
Files.write(filename, data);
} else {
downloadFromZK(zkPath + "/" + file, dir.resolve(file));
}
}
}
catch (KeeperException | InterruptedException e) {
throw new IOException("Error downloading files from zookeeper path " + zkPath + " to " + dir.toString(),
SolrZkClient.checkInterrupted(e));
}
}
/**
* Upload files from a given path to a config in Zookeeper
* @param dir {@link java.nio.file.Path} to the files
* @param configName the name to give the config
* @throws IOException
* if an I/O error occurs or the path does not exist
*/
public void uploadConfigDir(Path dir, String configName) throws IOException {
uploadToZK(dir, CONFIGS_ZKNODE + "/" + configName);
}
/**
* Download a config from Zookeeper and write it to the filesystem
* @param configName the config to download
* @param dir the {@link Path} to write files under
* @throws IOException
* if an I/O error occurs or the config does not exist
*/
public void downloadConfigDir(String configName, Path dir) throws IOException {
downloadFromZK(CONFIGS_ZKNODE + "/" + configName, dir);
}
public List<String> listConfigs() throws IOException {
try {
return zkClient.getChildren(ZkConfigManager.CONFIGS_ZKNODE, null, true);
}
catch (KeeperException.NoNodeException e) {
return Collections.emptyList();
}
catch (KeeperException | InterruptedException e) {
throw new IOException("Error listing configs", SolrZkClient.checkInterrupted(e));
}
}
}

View File

@ -50,7 +50,6 @@ import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
public class ZkStateReader implements Closeable {
private static Logger log = LoggerFactory.getLogger(ZkStateReader.class);
@ -116,6 +115,8 @@ public class ZkStateReader implements Closeable {
*/
private Map<String , DocCollection> watchedCollectionStates = new ConcurrentHashMap<String, DocCollection>();
private final ZkConfigManager configManager;
//
// convenience methods... should these go somewhere else?
@ -206,11 +207,11 @@ public class ZkStateReader implements Closeable {
private boolean clusterStateUpdateScheduled;
private SolrZkClient zkClient;
private final SolrZkClient zkClient;
private boolean closeClient = false;
private final boolean closeClient;
private ZkCmdExecutor cmdExecutor;
private final ZkCmdExecutor cmdExecutor;
private volatile Aliases aliases = new Aliases();
@ -218,16 +219,15 @@ public class ZkStateReader implements Closeable {
public ZkStateReader(SolrZkClient zkClient) {
this.zkClient = zkClient;
initZkCmdExecutor(zkClient.getZkClientTimeout());
this.cmdExecutor = new ZkCmdExecutor(zkClient.getZkClientTimeout());
this.configManager = new ZkConfigManager(zkClient);
this.closeClient = false;
}
public ZkStateReader(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) throws InterruptedException, TimeoutException, IOException {
closeClient = true;
initZkCmdExecutor(zkClientTimeout);
zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClientConnectTimeout,
public ZkStateReader(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) {
this.zkClient = new SolrZkClient(zkServerAddress, zkClientTimeout, zkClientConnectTimeout,
// on reconnect, reload cloud info
new OnReconnect() {
@Override
public void command() {
try {
@ -242,15 +242,16 @@ public class ZkStateReader implements Closeable {
log.error("", e);
throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
"", e);
}
}
}
});
this.cmdExecutor = new ZkCmdExecutor(zkClientTimeout);
this.configManager = new ZkConfigManager(zkClient);
this.closeClient = true;
}
private void initZkCmdExecutor(int zkClientTimeout) {
// we must retry at least as long as the session timeout
cmdExecutor = new ZkCmdExecutor(zkClientTimeout);
public ZkConfigManager getConfigManager() {
return configManager;
}
// load and publish a new CollectionInfo

View File

@ -0,0 +1,200 @@
/*
* 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.
*/
package org.apache.solr.common.cloud;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.cloud.ZkTestServer;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Id;
import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
public class TestZkConfigManager extends SolrTestCaseJ4 {
private static ZkTestServer zkServer;
@BeforeClass
public static void startZkServer() throws InterruptedException {
zkServer = new ZkTestServer(createTempDir("zkData").toString());
zkServer.run();
}
@AfterClass
public static void shutdownZkServer() throws IOException, InterruptedException {
zkServer.shutdown();
}
@Test
public void testUploadConfig() throws IOException {
zkServer.ensurePathExists("/solr");
try (SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress("/solr"), 10000)) {
ZkConfigManager configManager = new ZkConfigManager(zkClient);
assertEquals(0, configManager.listConfigs().size());
byte[] testdata = "test data".getBytes(Charsets.UTF_8);
Path tempConfig = createTempDir("config");
Files.createFile(tempConfig.resolve("file1"));
Files.write(tempConfig.resolve("file1"), testdata);
Files.createFile(tempConfig.resolve("file2"));
Files.createDirectory(tempConfig.resolve("subdir"));
Files.createFile(tempConfig.resolve("subdir").resolve("file3"));
configManager.uploadConfigDir(tempConfig, "testconfig");
// uploading a directory creates a new config
List<String> configs = configManager.listConfigs();
assertEquals(1, configs.size());
assertEquals("testconfig", configs.get(0));
// check downloading
Path downloadPath = createTempDir("download");
configManager.downloadConfigDir("testconfig", downloadPath);
assertTrue(Files.exists(downloadPath.resolve("file1")));
assertTrue(Files.exists(downloadPath.resolve("file2")));
assertTrue(Files.isDirectory(downloadPath.resolve("subdir")));
assertTrue(Files.exists(downloadPath.resolve("subdir/file3")));
byte[] checkdata = Files.readAllBytes(downloadPath.resolve("file1"));
assertArrayEquals(testdata, checkdata);
// uploading to the same config overwrites
byte[] overwritten = "new test data".getBytes(Charsets.UTF_8);
Files.write(tempConfig.resolve("file1"), overwritten);
configManager.uploadConfigDir(tempConfig, "testconfig");
assertEquals(1, configManager.listConfigs().size());
Path download2 = createTempDir("download2");
configManager.downloadConfigDir("testconfig", download2);
byte[] checkdata2 = Files.readAllBytes(download2.resolve("file1"));
assertArrayEquals(overwritten, checkdata2);
// uploading same files to a new name creates a new config
configManager.uploadConfigDir(tempConfig, "config2");
assertEquals(2, configManager.listConfigs().size());
}
}
@Test
public void testUploadWithACL() throws IOException {
zkServer.ensurePathExists("/acl");
final String readOnlyUsername = "readonly";
final String readOnlyPassword = "readonly";
final String writeableUsername = "writeable";
final String writeablePassword = "writeable";
ZkACLProvider aclProvider = new DefaultZkACLProvider(){
@Override
protected List<ACL> createGlobalACLsToAdd() {
try {
List<ACL> result = new ArrayList<>();
result.add(new ACL(ZooDefs.Perms.ALL, new Id("digest", DigestAuthenticationProvider.generateDigest(writeableUsername + ":" + writeablePassword))));
result.add(new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(readOnlyUsername + ":" + readOnlyPassword))));
return result;
}
catch (NoSuchAlgorithmException e) {
throw new RuntimeException(e);
}
}
};
ZkCredentialsProvider readonly = new DefaultZkCredentialsProvider(){
@Override
protected Collection<ZkCredentials> createCredentials() {
List<ZkCredentials> credentials = new ArrayList<>();
credentials.add(new ZkCredentials("digest", (readOnlyUsername + ":" + readOnlyPassword).getBytes(Charsets.UTF_8)));
return credentials;
}
};
ZkCredentialsProvider writeable = new DefaultZkCredentialsProvider(){
@Override
protected Collection<ZkCredentials> createCredentials() {
List<ZkCredentials> credentials = new ArrayList<>();
credentials.add(new ZkCredentials("digest", (writeableUsername + ":" + writeablePassword).getBytes(Charsets.UTF_8)));
return credentials;
}
};
Path configPath = createTempDir("acl-config");
Files.createFile(configPath.resolve("file1"));
// Start with all-access client
try (SolrZkClient client = buildZkClient(zkServer.getZkAddress("/acl"), aclProvider, writeable)) {
ZkConfigManager configManager = new ZkConfigManager(client);
configManager.uploadConfigDir(configPath, "acltest");
assertEquals(1, configManager.listConfigs().size());
}
// Readonly access client can get the list of configs, but can't upload
try (SolrZkClient client = buildZkClient(zkServer.getZkAddress("/acl"), aclProvider, readonly)) {
ZkConfigManager configManager = new ZkConfigManager(client);
assertEquals(1, configManager.listConfigs().size());
configManager.uploadConfigDir(configPath, "acltest2");
fail ("Should have thrown an ACL exception");
}
catch (IOException e) {
assertEquals(KeeperException.NoAuthException.class, Throwables.getRootCause(e).getClass());
}
// Client with no auth whatsoever can't even get the list of configs
try (SolrZkClient client = new SolrZkClient(zkServer.getZkAddress("/acl"), 10000)) {
ZkConfigManager configManager = new ZkConfigManager(client);
configManager.listConfigs();
fail("Should have thrown an ACL exception");
}
catch (IOException e) {
assertEquals(KeeperException.NoAuthException.class, Throwables.getRootCause(e).getClass());
}
}
static SolrZkClient buildZkClient(String zkAddress, final ZkACLProvider aclProvider,
final ZkCredentialsProvider credentialsProvider) {
return new SolrZkClient(zkAddress, 10000){
@Override
protected ZkCredentialsProvider createZkCredentialsToAddAutomatically() {
return credentialsProvider;
}
@Override
protected ZkACLProvider createZkACLProvider() {
return aclProvider;
}
};
}
}

View File

@ -23,6 +23,7 @@ import org.apache.solr.client.solrj.embedded.SSLConfig;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
@ -167,7 +168,8 @@ public class MiniSolrCloudCluster {
public void uploadConfigDir(File configDir, String configName) throws IOException, KeeperException, InterruptedException {
try(SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
ZkController.uploadConfigDir(zkClient, configDir, configName);
ZkConfigManager manager = new ZkConfigManager(zkClient);
manager.uploadConfigDir(configDir.toPath(), configName);
}
}

View File

@ -17,28 +17,9 @@ package org.apache.solr.cloud;
* the License.
*/
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.UnknownHostException;
import java.nio.channels.SelectionKey;
import java.nio.channels.SocketChannel;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import javax.management.JMException;
import com.google.common.collect.Ordering;
import com.google.common.util.concurrent.AtomicLongMap;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
@ -57,6 +38,25 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.management.JMException;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.UnknownHostException;
import java.nio.channels.SelectionKey;
import java.nio.channels.SocketChannel;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
public class ZkTestServer {
public static final int TICK_TIME = 1000;
@ -396,7 +396,31 @@ public class ZkTestServer {
}
public String getZkAddress() {
return "127.0.0.1:" + zkServer.getLocalPort() + "/solr";
return getZkAddress("/solr");
}
/**
* Get a connection string for this server with a given chroot
* @param chroot the chroot
* @return the connection string
*/
public String getZkAddress(String chroot) {
if (!chroot.startsWith("/"))
chroot = "/" + chroot;
return "127.0.0.1:" + zkServer.getLocalPort() + chroot;
}
/**
* Check that a path exists in this server
* @param path the path to check
* @throws IOException if an IO exception occurs
*/
public void ensurePathExists(String path) throws IOException {
try (SolrZkClient client = new SolrZkClient(getZkHost(), 10000)) {
client.makePath(path, false);
} catch (InterruptedException | KeeperException e) {
throw new IOException("Error checking path " + path, SolrZkClient.checkInterrupted(e));
}
}
public int getPort() {