SOLR-14907: Support single file upload/overwrite in configSet API (#1977)

This commit is contained in:
Houston Putman 2020-10-13 16:51:21 -04:00 committed by GitHub
parent 9594ab3ac0
commit bcd9cbec95
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 252 additions and 14 deletions

View File

@ -160,6 +160,8 @@ New Features
* SOLR-11167: Avoid $SOLR_STOP_WAIT use during 'bin/solr start' if $SOLR_START_WAIT is supplied. * SOLR-11167: Avoid $SOLR_STOP_WAIT use during 'bin/solr start' if $SOLR_START_WAIT is supplied.
(Omar Abdelnabi, Christine Poerschke) (Omar Abdelnabi, Christine Poerschke)
* SOLR-14907: Support single file upload/overwrite in configSet API. (Houston Putman)
Improvements Improvements
--------------------- ---------------------

View File

@ -170,22 +170,52 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
boolean overwritesExisting = zkClient.exists(configPathInZk, true); boolean overwritesExisting = zkClient.exists(configPathInZk, true);
if (overwritesExisting && !req.getParams().getBool(ConfigSetParams.OVERWRITE, false)) { boolean requestIsTrusted = isTrusted(req, coreContainer.getAuthenticationPlugin());
throw new SolrException(ErrorCode.BAD_REQUEST,
"The configuration " + configSetName + " already exists in zookeeper"); // Get upload parameters
} String singleFilePath = req.getParams().get(ConfigSetParams.FILE_PATH, "");
boolean allowOverwrite = req.getParams().getBool(ConfigSetParams.OVERWRITE, false);
boolean cleanup = req.getParams().getBool(ConfigSetParams.CLEANUP, false);
Iterator<ContentStream> contentStreamsIterator = req.getContentStreams().iterator(); Iterator<ContentStream> contentStreamsIterator = req.getContentStreams().iterator();
if (!contentStreamsIterator.hasNext()) { if (!contentStreamsIterator.hasNext()) {
throw new SolrException(ErrorCode.BAD_REQUEST, throw new SolrException(ErrorCode.BAD_REQUEST,
"No stream found for the config data to be uploaded"); "No stream found for the config data to be uploaded");
} }
InputStream inputStream = contentStreamsIterator.next().getStream(); InputStream inputStream = contentStreamsIterator.next().getStream();
// Create a node for the configuration in zookeeper // Only Upload a single file
boolean cleanup = req.getParams().getBool(ConfigSetParams.CLEANUP, false); if (!singleFilePath.isEmpty()) {
String fixedSingleFilePath = singleFilePath;
if (fixedSingleFilePath.charAt(0) == '/') {
fixedSingleFilePath = fixedSingleFilePath.substring(1);
}
if (fixedSingleFilePath.isEmpty()) {
throw new SolrException(ErrorCode.BAD_REQUEST, "The filePath provided for upload, '" + singleFilePath + "', is not valid.");
} else if (cleanup) {
// Cleanup is not allowed while using singleFilePath upload
throw new SolrException(ErrorCode.BAD_REQUEST, "ConfigSet uploads do not allow cleanup=true when filePath is used.");
} else {
try {
// Create a node for the configuration in zookeeper
// For creating the baseZnode, the cleanup parameter is only allowed to be true when singleFilePath is not passed.
createBaseZnode(zkClient, overwritesExisting, requestIsTrusted, configPathInZk);
String filePathInZk = configPathInZk + "/" + fixedSingleFilePath;
zkClient.makePath(filePathInZk, IOUtils.toByteArray(inputStream), CreateMode.PERSISTENT, null, !allowOverwrite, true);
} catch(KeeperException.NodeExistsException nodeExistsException) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"The path " + singleFilePath + " for configSet " + configSetName + " already exists. In order to overwrite, provide overwrite=true.");
}
}
return;
}
if (overwritesExisting && !allowOverwrite) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"The configuration " + configSetName + " already exists in zookeeper");
}
Set<String> filesToDelete; Set<String> filesToDelete;
if (overwritesExisting && cleanup) { if (overwritesExisting && cleanup) {
@ -193,7 +223,10 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
} else { } else {
filesToDelete = Collections.emptySet(); filesToDelete = Collections.emptySet();
} }
createBaseZnode(zkClient, overwritesExisting, isTrusted(req, coreContainer.getAuthenticationPlugin()), cleanup, configPathInZk);
// Create a node for the configuration in zookeeper
// For creating the baseZnode, the cleanup parameter is only allowed to be true when singleFilePath is not passed.
createBaseZnode(zkClient, overwritesExisting, requestIsTrusted, configPathInZk);
ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8); ZipInputStream zis = new ZipInputStream(inputStream, StandardCharsets.UTF_8);
ZipEntry zipEntry = null; ZipEntry zipEntry = null;
@ -213,17 +246,22 @@ public class ConfigSetsHandler extends RequestHandlerBase implements PermissionN
} }
zis.close(); zis.close();
deleteUnusedFiles(zkClient, filesToDelete); deleteUnusedFiles(zkClient, filesToDelete);
// If the request is doing a full trusted overwrite of an untrusted configSet (overwrite=true, cleanup=true), then trust the configSet.
if (cleanup && requestIsTrusted && overwritesExisting && !isCurrentlyTrusted(zkClient, configPathInZk)) {
byte[] baseZnodeData = ("{\"trusted\": true}").getBytes(StandardCharsets.UTF_8);
zkClient.setData(configPathInZk, baseZnodeData, true);
}
} }
private void createBaseZnode(SolrZkClient zkClient, boolean overwritesExisting, boolean requestIsTrusted, boolean cleanup, String configPathInZk) throws KeeperException, InterruptedException { private void createBaseZnode(SolrZkClient zkClient, boolean overwritesExisting, boolean requestIsTrusted, String configPathInZk) throws KeeperException, InterruptedException {
byte[] baseZnodeData = ("{\"trusted\": " + Boolean.toString(requestIsTrusted) + "}").getBytes(StandardCharsets.UTF_8); byte[] baseZnodeData = ("{\"trusted\": " + Boolean.toString(requestIsTrusted) + "}").getBytes(StandardCharsets.UTF_8);
if (overwritesExisting) { if (overwritesExisting) {
if (cleanup && requestIsTrusted) { if (!requestIsTrusted) {
zkClient.setData(configPathInZk, baseZnodeData, true);
} else if (!requestIsTrusted) {
ensureOverwritingUntrustedConfigSet(zkClient, configPathInZk); ensureOverwritingUntrustedConfigSet(zkClient, configPathInZk);
} }
// If the request is trusted and cleanup=true, then the configSet will be set to trusted after the overwriting has been done.
} else { } else {
zkClient.makePath(configPathInZk, baseZnodeData, true); zkClient.makePath(configPathInZk, baseZnodeData, true);
} }

View File

@ -470,6 +470,14 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
assertFalse(isTrusted(zkClient, configsetName, configsetSuffix)); assertFalse(isTrusted(zkClient, configsetName, configsetSuffix));
solrconfigZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml"); solrconfigZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml");
// Was untrusted, overwrite with trusted with cleanup but fail on unzipping.
// Should not set trusted=true
assertEquals(500, uploadBadConfigSet(configsetName, configsetSuffix, "solr", zkClient, true, true));
assertEquals("Expecting version bump",
solrconfigZkVersion, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml"));
assertFalse(isTrusted(zkClient, configsetName, configsetSuffix));
solrconfigZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml");
// Was untrusted, overwrite with trusted with cleanup // Was untrusted, overwrite with trusted with cleanup
assertEquals(0, uploadConfigSet(configsetName, configsetSuffix, "solr", zkClient, true, true)); assertEquals(0, uploadConfigSet(configsetName, configsetSuffix, "solr", zkClient, true, true));
assertTrue("Expecting version bump", assertTrue("Expecting version bump",
@ -510,6 +518,158 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
} }
@Test
public void testSingleFileOverwrite() throws Exception {
String configsetName = "regular";
String configsetSuffix = "testSinglePathOverwrite-1";
uploadConfigSetWithAssertions(configsetName, configsetSuffix, null);
try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
int solrconfigZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml");
ignoreException("The configuration regulartestOverwrite-1 already exists in zookeeper");
assertEquals("Can't overwrite an existing configset unless the overwrite parameter is set",
400, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "solrconfig.xml", false, false));
unIgnoreException("The configuration regulartestOverwrite-1 already exists in zookeeper");
assertEquals("Expecting version to remain equal",
solrconfigZkVersion, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml"));
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "solrconfig.xml", true, false));
assertTrue("Expecting version bump",
solrconfigZkVersion < getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "solrconfig.xml"));
}
}
@Test
public void testNewSingleFile() throws Exception {
String configsetName = "regular";
String configsetSuffix = "testSinglePathNew-1";
uploadConfigSetWithAssertions(configsetName, configsetSuffix, null);
try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/upload/path/solrconfig.xml", false, false));
assertEquals("Expecting first version of new file", 0, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml"));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
}
}
@Test
public void testSingleWithCleanup() throws Exception {
String configsetName = "regular";
String configsetSuffix = "testSinglePathCleanup-1";
uploadConfigSetWithAssertions(configsetName, configsetSuffix, null);
try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
ignoreException("ConfigSet uploads do not allow cleanup=true when filePath is used.");
assertEquals(400, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/upload/path/solrconfig.xml", false, true));
assertFalse("New file should not exist, since the trust check did not succeed.", zkClient.exists("/configs/"+configsetName+configsetSuffix+"/test/upload/path/solrconfig.xml", true));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
unIgnoreException("ConfigSet uploads do not allow cleanup=true when filePath is used.");
}
}
@Test
public void testSingleFileTrusted() throws Exception {
String configsetName = "regular";
String configsetSuffix = "testSinglePathTrusted-1";
uploadConfigSetWithAssertions(configsetName, configsetSuffix, "solr");
try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, "solr", zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/upload/path/solrconfig.xml", true, false));
assertEquals("Expecting first version of new file", 0, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml"));
assertTrue(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
ignoreException("Trying to make an unstrusted ConfigSet update on a trusted configSet");
assertEquals("Can't upload a trusted configset with an untrusted request",
400, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/different/path/solrconfig.xml", true, false));
assertFalse("New file should not exist, since the trust check did not succeed.", zkClient.exists("/configs/"+configsetName+configsetSuffix+"/test/different/path/solrconfig.xml", true));
assertTrue(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
unIgnoreException("Trying to make an unstrusted ConfigSet update on a trusted configSet");
ignoreException("Trying to make an unstrusted ConfigSet update on a trusted configSet");
int extraFileZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml");
assertEquals("Can't upload a trusted configset with an untrusted request",
400, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/upload/path/solrconfig.xml", true, false));
assertEquals("Expecting version to remain equal",
extraFileZkVersion, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml"));
assertTrue(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
unIgnoreException("Trying to make an unstrusted ConfigSet update on a trusted configSet");
}
}
@Test
public void testSingleFileUntrusted() throws Exception {
String configsetName = "regular";
String configsetSuffix = "testSinglePathUntrusted-1";
uploadConfigSetWithAssertions(configsetName, configsetSuffix, null);
try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
// New file with trusted request
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, "solr", zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/upload/path/solrconfig.xml", false, false));
assertEquals("Expecting first version of new file", 0, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml"));
assertFalse(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
// New file with untrusted request
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/different/path/solrconfig.xml", false, false));
assertEquals("Expecting first version of new file", 0, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/different/path/solrconfig.xml"));
assertFalse(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
// Overwrite with trusted request
int extraFileZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/different/path/solrconfig.xml");
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, "solr", zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/different/path/solrconfig.xml", true, false));
assertTrue("Expecting version bump",
extraFileZkVersion < getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/different/path/solrconfig.xml"));
assertFalse(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
// Overwrite with untrusted request
extraFileZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml");
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffix, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/upload/path/solrconfig.xml", true, false));
assertTrue("Expecting version bump",
extraFileZkVersion < getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/upload/path/solrconfig.xml"));
assertFalse(isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
// Make sure that cleanup flag does not result in configSet being trusted.
ignoreException("ConfigSet uploads do not allow cleanup=true when filePath is used.");
extraFileZkVersion = getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/different/path/solrconfig.xml");
assertEquals(400, uploadSingleConfigSetFile(configsetName, configsetSuffix, "solr", zkClient, "solr/configsets/upload/regular/solrconfig.xml", "/test/different/path/solrconfig.xml", true, true));
assertEquals("Expecting version to stay the same",
extraFileZkVersion, getConfigZNodeVersion(zkClient, configsetName, configsetSuffix, "test/different/path/solrconfig.xml"));
assertFalse("The cleanup=true flag allowed for trust overwriting in a filePath upload.", isTrusted(zkClient, configsetName, configsetSuffix));
assertConfigsetFiles(configsetName, configsetSuffix, zkClient);
unIgnoreException("ConfigSet uploads do not allow cleanup=true when filePath is used.");
}
}
@Test
public void testSingleFileNewConfig() throws Exception {
String configsetName = "regular";
String configsetSuffixTrusted = "testSinglePathNewConfig-1";
String configsetSuffixUntrusted = "testSinglePathNewConfig-2";
try (SolrZkClient zkClient = new SolrZkClient(cluster.getZkServer().getZkAddress(),
AbstractZkTestCase.TIMEOUT, 45000, null)) {
// New file with trusted request
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffixTrusted, "solr", zkClient, "solr/configsets/upload/regular/solrconfig.xml", "solrconfig.xml", false, false));
assertEquals("Expecting first version of new file", 0, getConfigZNodeVersion(zkClient, configsetName, configsetSuffixTrusted, "solrconfig.xml"));
assertTrue(isTrusted(zkClient, configsetName, configsetSuffixTrusted));
List<String> children = zkClient.getChildren(String.format(Locale.ROOT,"/configs/%s%s", configsetName, configsetSuffixTrusted), null, true);
assertEquals("The configSet should only have one file uploaded.", 1, children.size());
assertEquals("Incorrect file uploaded.", "solrconfig.xml", children.get(0));
// New file with trusted request
assertEquals(0, uploadSingleConfigSetFile(configsetName, configsetSuffixUntrusted, null, zkClient, "solr/configsets/upload/regular/solrconfig.xml", "solrconfig.xml", false, false));
assertEquals("Expecting first version of new file", 0, getConfigZNodeVersion(zkClient, configsetName, configsetSuffixUntrusted, "solrconfig.xml"));
assertFalse(isTrusted(zkClient, configsetName, configsetSuffixUntrusted));
children = zkClient.getChildren(String.format(Locale.ROOT,"/configs/%s%s", configsetName, configsetSuffixUntrusted), null, true);
assertEquals("The configSet should only have one file uploaded.", 1, children.size());
assertEquals("Incorrect file uploaded.", "solrconfig.xml", children.get(0));
}
}
private boolean isTrusted(SolrZkClient zkClient, String configsetName, String configsetSuffix) throws KeeperException, InterruptedException { private boolean isTrusted(SolrZkClient zkClient, String configsetName, String configsetSuffix) throws KeeperException, InterruptedException {
String configSetZkPath = String.format(Locale.ROOT,"/configs/%s%s", configsetName, configsetSuffix); String configSetZkPath = String.format(Locale.ROOT,"/configs/%s%s", configsetName, configsetSuffix);
byte[] configSetNodeContent = zkClient.getData(configSetZkPath, null, null, true);; byte[] configSetNodeContent = zkClient.getData(configSetZkPath, null, null, true);;
@ -647,7 +807,35 @@ public class TestConfigSetsAPI extends SolrCloudTestCase {
long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status")); long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
return statusCode; return statusCode;
} }
private long uploadBadConfigSet(String configSetName, String suffix, String username,
SolrZkClient zkClient, boolean overwrite, boolean cleanup) throws IOException {
// Read single file from sample configs. This should fail the unzipping
ByteBuffer sampleBadZippedFile = TestSolrConfigHandler.getFileContent(SolrTestCaseJ4.getFile("solr/configsets/upload/regular/solrconfig.xml").getAbsolutePath(), false);
@SuppressWarnings({"rawtypes"})
Map map = postDataAndGetResponse(cluster.getSolrClient(),
cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&name="+configSetName+suffix + (overwrite? "&overwrite=true" : "") + (cleanup? "&cleanup=true" : ""),
sampleBadZippedFile, username);
assertNotNull(map);
long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
return statusCode;
}
private long uploadSingleConfigSetFile(String configSetName, String suffix, String username,
SolrZkClient zkClient, String filePath, String uploadPath, boolean overwrite, boolean cleanup) throws IOException {
// Read single file from sample configs
ByteBuffer sampleConfigFile = TestSolrConfigHandler.getFileContent(SolrTestCaseJ4.getFile(filePath).getAbsolutePath(), false);
@SuppressWarnings({"rawtypes"})
Map map = postDataAndGetResponse(cluster.getSolrClient(),
cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/admin/configs?action=UPLOAD&name="+configSetName+suffix+"&filePath="+uploadPath + (overwrite? "&overwrite=true" : "") + (cleanup? "&cleanup=true" : ""),
sampleConfigFile, username);
assertNotNull(map);
long statusCode = (long) getObjectByPath(map, false, Arrays.asList("responseHeader", "status"));
return statusCode;
}
/** /**
* Create a zip file (in the temp directory) containing all the files within the specified directory * Create a zip file (in the temp directory) containing all the files within the specified directory
* and return the path for the zip file. * and return the path for the zip file.

View File

@ -81,6 +81,7 @@ The output will look like:
== Upload a Configset == Upload a Configset
Upload a configset, which is sent as a zipped file. Upload a configset, which is sent as a zipped file.
A single, non-zipped file can also be uploaded with the `filePath` parameter.
This functionality is enabled by default, but can be disabled via a runtime parameter `-Dconfigset.upload.enabled=false`. Disabling this feature is advisable if you want to expose Solr installation to untrusted users (even though you should never do that!). This functionality is enabled by default, but can be disabled via a runtime parameter `-Dconfigset.upload.enabled=false`. Disabling this feature is advisable if you want to expose Solr installation to untrusted users (even though you should never do that!).
@ -98,10 +99,18 @@ The `upload` command takes the following parameters:
The configset to be created when the upload is complete. This parameter is required. The configset to be created when the upload is complete. This parameter is required.
`overwrite`:: `overwrite`::
If set to `true`, Solr will overwrite an existing configset with the same name (if false, the request will fail). Default is `false`. If set to `true`, Solr will overwrite an existing configset with the same name (if false, the request will fail).
If `filePath` is provided, then this option specifies whether the specified file within the configSet should be overwritten if it already exists.
Default is `false`.
`cleanup`:: `cleanup`::
When overwriting an existing configset (`overwrite=true`), this parameter tells Solr to delete the files in ZooKeeper that existed in the old configset but not in the one being uploaded. Default is `false`. When overwriting an existing configset (`overwrite=true`), this parameter tells Solr to delete the files in ZooKeeper that existed in the old configset but not in the one being uploaded. Default is `false`.
This parameter cannot be set to true when `filePath` is used.
filePath::
This parameter allows the uploading of a single, non-zipped file to the given path under the configSet in ZooKeeper.
This functionality respects the `overwrite` parameter, so a request will fail if the given file path already exists in the configSet and overwrite is set to `false`.
The `cleanup` parameter cannot be set to true when `filePath` is used.
The body of the request should be a zip file that contains the configset. The zip file must be created from within the `conf` directory (i.e., `solrconfig.xml` must be the top level entry in the zip file). The body of the request should be a zip file that contains the configset. The zip file must be created from within the `conf` directory (i.e., `solrconfig.xml` must be the top level entry in the zip file).

View File

@ -26,6 +26,7 @@ public interface ConfigSetParams
public final static String ACTION = "action"; public final static String ACTION = "action";
public final static String OVERWRITE = "overwrite"; public final static String OVERWRITE = "overwrite";
public final static String CLEANUP = "cleanup"; public final static String CLEANUP = "cleanup";
public final static String FILE_PATH = "filePath";
public enum ConfigSetAction { public enum ConfigSetAction {
CREATE, CREATE,