mirror of https://github.com/apache/lucene.git
Merge branch 'SOLR-8782'
This commit is contained in:
commit
406a1635c1
|
@ -22,6 +22,10 @@ See the Quick Start guide at http://lucene.apache.org/solr/quickstart.html
|
||||||
================== 6.1.0 ==================
|
================== 6.1.0 ==================
|
||||||
Detailed Change List
|
Detailed Change List
|
||||||
----------------------
|
----------------------
|
||||||
|
* SOLR-8782: Add asynchronous sugar methods to the SolrJ Collections API. You
|
||||||
|
can now call .processAsync() to run a method asynchronously, or
|
||||||
|
.processAndWait() to wait for a call to finish without holding HTTP
|
||||||
|
collections open. (Alan Woodward)
|
||||||
|
|
||||||
New Features
|
New Features
|
||||||
----------------------
|
----------------------
|
||||||
|
|
|
@ -0,0 +1,32 @@
|
||||||
|
<?xml version="1.0" encoding="UTF-8" ?>
|
||||||
|
<!--
|
||||||
|
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.
|
||||||
|
-->
|
||||||
|
<schema name="minimal" version="1.1">
|
||||||
|
<types>
|
||||||
|
<fieldType name="string" class="solr.StrField"/>
|
||||||
|
<fieldType name="int" class="solr.TrieIntField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
|
||||||
|
<fieldType name="long" class="solr.TrieLongField" precisionStep="0" omitNorms="true" positionIncrementGap="0"/>
|
||||||
|
</types>
|
||||||
|
<fields>
|
||||||
|
<dynamicField name="*" type="string" indexed="true" stored="true" />
|
||||||
|
<!-- for versioning -->
|
||||||
|
<field name="_version_" type="long" indexed="true" stored="true"/>
|
||||||
|
<field name="_root_" type="int" indexed="true" stored="true" multiValued="false" required="false"/>
|
||||||
|
<field name="id" type="string" indexed="true" stored="true"/>
|
||||||
|
</fields>
|
||||||
|
<uniqueKey>id</uniqueKey>
|
||||||
|
</schema>
|
|
@ -0,0 +1,48 @@
|
||||||
|
<?xml version="1.0" ?>
|
||||||
|
|
||||||
|
<!--
|
||||||
|
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.
|
||||||
|
-->
|
||||||
|
|
||||||
|
<!-- Minimal solrconfig.xml with /select, /admin and /update only -->
|
||||||
|
|
||||||
|
<config>
|
||||||
|
|
||||||
|
<dataDir>${solr.data.dir:}</dataDir>
|
||||||
|
|
||||||
|
<directoryFactory name="DirectoryFactory"
|
||||||
|
class="${solr.directoryFactory:solr.NRTCachingDirectoryFactory}"/>
|
||||||
|
<schemaFactory class="ClassicIndexSchemaFactory"/>
|
||||||
|
|
||||||
|
<luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
|
||||||
|
|
||||||
|
<updateHandler class="solr.DirectUpdateHandler2">
|
||||||
|
<commitWithin>
|
||||||
|
<softCommit>${solr.commitwithin.softcommit:true}</softCommit>
|
||||||
|
</commitWithin>
|
||||||
|
<updateLog></updateLog>
|
||||||
|
</updateHandler>
|
||||||
|
|
||||||
|
<requestHandler name="/select" class="solr.SearchHandler">
|
||||||
|
<lst name="defaults">
|
||||||
|
<str name="echoParams">explicit</str>
|
||||||
|
<str name="indent">true</str>
|
||||||
|
<str name="df">text</str>
|
||||||
|
</lst>
|
||||||
|
|
||||||
|
</requestHandler>
|
||||||
|
</config>
|
||||||
|
|
|
@ -21,90 +21,80 @@ import java.util.List;
|
||||||
|
|
||||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||||
import org.apache.lucene.util.TestUtil;
|
import org.apache.lucene.util.TestUtil;
|
||||||
import org.apache.solr.client.solrj.SolrClient;
|
|
||||||
import org.apache.solr.client.solrj.SolrQuery;
|
import org.apache.solr.client.solrj.SolrQuery;
|
||||||
import org.apache.solr.client.solrj.impl.HttpSolrClient;
|
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
|
import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
|
||||||
import org.apache.solr.client.solrj.request.CollectionAdminRequest.SplitShard;
|
import org.apache.solr.client.solrj.request.CollectionAdminRequest.SplitShard;
|
||||||
import org.apache.solr.client.solrj.response.RequestStatusState;
|
import org.apache.solr.client.solrj.response.RequestStatusState;
|
||||||
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
|
|
||||||
import org.apache.solr.common.SolrException;
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.SolrInputDocument;
|
import org.apache.solr.common.SolrInputDocument;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests the Cloud Collections API.
|
* Tests the Cloud Collections API.
|
||||||
*/
|
*/
|
||||||
@Slow
|
@Slow
|
||||||
public class CollectionsAPIAsyncDistributedZkTest extends AbstractFullDistribZkTestBase {
|
public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
|
||||||
|
|
||||||
private static final int MAX_TIMEOUT_SECONDS = 60;
|
private static final int MAX_TIMEOUT_SECONDS = 60;
|
||||||
|
|
||||||
public CollectionsAPIAsyncDistributedZkTest() {
|
@BeforeClass
|
||||||
sliceCount = 1;
|
public static void setupCluster() throws Exception {
|
||||||
|
configureCluster(2)
|
||||||
|
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
|
||||||
|
.configure();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ShardsFixed(num = 1)
|
|
||||||
public void testSolrJAPICalls() throws Exception {
|
public void testSolrJAPICalls() throws Exception {
|
||||||
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
|
|
||||||
Create createCollectionRequest = new Create()
|
final CloudSolrClient client = cluster.getSolrClient();
|
||||||
.setCollectionName("testasynccollectioncreation")
|
|
||||||
.setNumShards(1)
|
RequestStatusState state = new Create()
|
||||||
.setConfigName("conf1")
|
.setCollectionName("testasynccollectioncreation")
|
||||||
.setAsyncId("1001");
|
.setNumShards(1)
|
||||||
createCollectionRequest.process(client);
|
.setReplicationFactor(1)
|
||||||
|
.setConfigName("conf1")
|
||||||
RequestStatusState state = getRequestStateAfterCompletion("1001", MAX_TIMEOUT_SECONDS, client);
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
|
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
|
||||||
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
|
|
||||||
|
state = new Create()
|
||||||
createCollectionRequest = new Create()
|
.setCollectionName("testasynccollectioncreation")
|
||||||
.setCollectionName("testasynccollectioncreation")
|
.setNumShards(1)
|
||||||
.setNumShards(1)
|
.setConfigName("conf1")
|
||||||
.setConfigName("conf1")
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
.setAsyncId("1002");
|
assertSame("Recreating a collection with the same should have failed.", RequestStatusState.FAILED, state);
|
||||||
createCollectionRequest.process(client);
|
|
||||||
|
state = new CollectionAdminRequest.AddReplica()
|
||||||
state = getRequestStateAfterCompletion("1002", MAX_TIMEOUT_SECONDS, client);
|
.setCollectionName("testasynccollectioncreation")
|
||||||
|
.setShardName("shard1")
|
||||||
assertSame("Recreating a collection with the same should have failed.", RequestStatusState.FAILED, state);
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
|
assertSame("Add replica did not complete", RequestStatusState.COMPLETED, state);
|
||||||
CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
|
|
||||||
.setCollectionName("testasynccollectioncreation")
|
state = new SplitShard()
|
||||||
.setShardName("shard1")
|
.setCollectionName("testasynccollectioncreation")
|
||||||
.setAsyncId("1003");
|
.setShardName("shard1")
|
||||||
client.request(addReplica);
|
.processAndWait(client, MAX_TIMEOUT_SECONDS * 2);
|
||||||
state = getRequestStateAfterCompletion("1003", MAX_TIMEOUT_SECONDS, client);
|
assertEquals("Shard split did not complete. Last recorded state: " + state, RequestStatusState.COMPLETED, state);
|
||||||
assertSame("Add replica did not complete", RequestStatusState.COMPLETED, state);
|
|
||||||
|
|
||||||
SplitShard splitShardRequest = new SplitShard()
|
|
||||||
.setCollectionName("testasynccollectioncreation")
|
|
||||||
.setShardName("shard1")
|
|
||||||
.setAsyncId("1004");
|
|
||||||
splitShardRequest.process(client);
|
|
||||||
|
|
||||||
state = getRequestStateAfterCompletion("1004", MAX_TIMEOUT_SECONDS * 2, client);
|
|
||||||
|
|
||||||
assertEquals("Shard split did not complete. Last recorded state: " + state, RequestStatusState.COMPLETED, state);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testAsyncRequests() throws Exception {
|
public void testAsyncRequests() throws Exception {
|
||||||
String collection = "testAsyncOperations";
|
|
||||||
|
|
||||||
Create createCollectionRequest = new Create()
|
final String collection = "testAsyncOperations";
|
||||||
|
final CloudSolrClient client = cluster.getSolrClient();
|
||||||
|
|
||||||
|
RequestStatusState state = new Create()
|
||||||
.setCollectionName(collection)
|
.setCollectionName(collection)
|
||||||
.setNumShards(1)
|
.setNumShards(1)
|
||||||
.setRouterName("implicit")
|
.setRouterName("implicit")
|
||||||
.setShards("shard1")
|
.setShards("shard1")
|
||||||
.setConfigName("conf1")
|
.setConfigName("conf1")
|
||||||
.setAsyncId("42");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
CollectionAdminResponse response = createCollectionRequest.process(cloudClient);
|
|
||||||
assertEquals("42", response.getResponse().get("requestid"));
|
|
||||||
RequestStatusState state = getRequestStateAfterCompletion("42", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
|
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
//Add a few documents to shard1
|
//Add a few documents to shard1
|
||||||
|
@ -116,59 +106,48 @@ public class CollectionsAPIAsyncDistributedZkTest extends AbstractFullDistribZkT
|
||||||
doc.addField("_route_", "shard1");
|
doc.addField("_route_", "shard1");
|
||||||
docs.add(doc);
|
docs.add(doc);
|
||||||
}
|
}
|
||||||
cloudClient.add(collection, docs);
|
client.add(collection, docs);
|
||||||
cloudClient.commit(collection);
|
client.commit(collection);
|
||||||
|
|
||||||
SolrQuery query = new SolrQuery("*:*");
|
SolrQuery query = new SolrQuery("*:*");
|
||||||
query.set("shards", "shard1");
|
query.set("shards", "shard1");
|
||||||
assertEquals(numDocs, cloudClient.query(collection, query).getResults().getNumFound());
|
assertEquals(numDocs, client.query(collection, query).getResults().getNumFound());
|
||||||
|
|
||||||
CollectionAdminRequest.Reload reloadCollection = new CollectionAdminRequest.Reload();
|
state = new CollectionAdminRequest.Reload()
|
||||||
reloadCollection.setCollectionName(collection).setAsyncId("43");
|
.setCollectionName(collection)
|
||||||
response = reloadCollection.process(cloudClient);
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
assertEquals("43", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("43", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("ReloadCollection did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("ReloadCollection did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
CollectionAdminRequest.CreateShard createShard = new CollectionAdminRequest.CreateShard()
|
state = new CollectionAdminRequest.CreateShard()
|
||||||
.setCollectionName(collection)
|
.setCollectionName(collection)
|
||||||
.setShardName("shard2")
|
.setShardName("shard2")
|
||||||
.setAsyncId("44");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = createShard.process(cloudClient);
|
|
||||||
assertEquals("44", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("44", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("CreateShard did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("CreateShard did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
//Add a doc to shard2 to make sure shard2 was created properly
|
//Add a doc to shard2 to make sure shard2 was created properly
|
||||||
SolrInputDocument doc = new SolrInputDocument();
|
SolrInputDocument doc = new SolrInputDocument();
|
||||||
doc.addField("id", numDocs + 1);
|
doc.addField("id", numDocs + 1);
|
||||||
doc.addField("_route_", "shard2");
|
doc.addField("_route_", "shard2");
|
||||||
cloudClient.add(collection, doc);
|
client.add(collection, doc);
|
||||||
cloudClient.commit(collection);
|
client.commit(collection);
|
||||||
query = new SolrQuery("*:*");
|
query = new SolrQuery("*:*");
|
||||||
query.set("shards", "shard2");
|
query.set("shards", "shard2");
|
||||||
assertEquals(1, cloudClient.query(collection, query).getResults().getNumFound());
|
assertEquals(1, client.query(collection, query).getResults().getNumFound());
|
||||||
|
|
||||||
CollectionAdminRequest.DeleteShard deleteShard = new CollectionAdminRequest.DeleteShard()
|
state = new CollectionAdminRequest.DeleteShard()
|
||||||
.setCollectionName(collection)
|
.setCollectionName(collection)
|
||||||
.setShardName("shard2")
|
.setShardName("shard2")
|
||||||
.setAsyncId("45");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = deleteShard.process(cloudClient);
|
|
||||||
assertEquals("45", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("45", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("DeleteShard did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("DeleteShard did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
|
state = new CollectionAdminRequest.AddReplica()
|
||||||
.setCollectionName(collection)
|
.setCollectionName(collection)
|
||||||
.setShardName("shard1")
|
.setShardName("shard1")
|
||||||
.setAsyncId("46");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = addReplica.process(cloudClient);
|
|
||||||
assertEquals("46", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("46", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("AddReplica did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("AddReplica did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
//cloudClient watch might take a couple of seconds to reflect it
|
//cloudClient watch might take a couple of seconds to reflect it
|
||||||
Slice shard1 = cloudClient.getZkStateReader().getClusterState().getSlice(collection, "shard1");
|
Slice shard1 = client.getZkStateReader().getClusterState().getSlice(collection, "shard1");
|
||||||
int count = 0;
|
int count = 0;
|
||||||
while (shard1.getReplicas().size() != 2) {
|
while (shard1.getReplicas().size() != 2) {
|
||||||
if (count++ > 1000) {
|
if (count++ > 1000) {
|
||||||
|
@ -177,51 +156,40 @@ public class CollectionsAPIAsyncDistributedZkTest extends AbstractFullDistribZkT
|
||||||
Thread.sleep(100);
|
Thread.sleep(100);
|
||||||
}
|
}
|
||||||
|
|
||||||
CollectionAdminRequest.CreateAlias createAlias = new CollectionAdminRequest.CreateAlias()
|
state = new CollectionAdminRequest.CreateAlias()
|
||||||
.setAliasName("myalias")
|
.setAliasName("myalias")
|
||||||
.setAliasedCollections(collection)
|
.setAliasedCollections(collection)
|
||||||
.setAsyncId("47");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = createAlias.process(cloudClient);
|
|
||||||
assertEquals("47", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("47", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("CreateAlias did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("CreateAlias did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
query = new SolrQuery("*:*");
|
query = new SolrQuery("*:*");
|
||||||
query.set("shards", "shard1");
|
query.set("shards", "shard1");
|
||||||
assertEquals(numDocs, cloudClient.query("myalias", query).getResults().getNumFound());
|
assertEquals(numDocs, client.query("myalias", query).getResults().getNumFound());
|
||||||
|
|
||||||
CollectionAdminRequest.DeleteAlias deleteAlias = new CollectionAdminRequest.DeleteAlias()
|
state = new CollectionAdminRequest.DeleteAlias()
|
||||||
.setAliasName("myalias")
|
.setAliasName("myalias")
|
||||||
.setAsyncId("48");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = deleteAlias.process(cloudClient);
|
|
||||||
assertEquals("48", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("48", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("DeleteAlias did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("DeleteAlias did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
cloudClient.query("myalias", query);
|
client.query("myalias", query);
|
||||||
fail("Alias should not exist");
|
fail("Alias should not exist");
|
||||||
} catch (SolrException e) {
|
} catch (SolrException e) {
|
||||||
//expected
|
//expected
|
||||||
}
|
}
|
||||||
|
|
||||||
String replica = shard1.getReplicas().iterator().next().getName();
|
String replica = shard1.getReplicas().iterator().next().getName();
|
||||||
CollectionAdminRequest.DeleteReplica deleteReplica = new CollectionAdminRequest.DeleteReplica()
|
state = new CollectionAdminRequest.DeleteReplica()
|
||||||
.setCollectionName(collection)
|
.setCollectionName(collection)
|
||||||
.setShardName("shard1")
|
.setShardName("shard1")
|
||||||
.setReplica(replica)
|
.setReplica(replica)
|
||||||
.setAsyncId("47");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = deleteReplica.process(cloudClient);
|
|
||||||
assertEquals("47", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("47", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("DeleteReplica did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("DeleteReplica did not complete", RequestStatusState.COMPLETED, state);
|
||||||
|
|
||||||
CollectionAdminRequest.Delete deleteCollection = new CollectionAdminRequest.Delete()
|
state = new CollectionAdminRequest.Delete()
|
||||||
.setCollectionName(collection)
|
.setCollectionName(collection)
|
||||||
.setAsyncId("48");
|
.processAndWait(client, MAX_TIMEOUT_SECONDS);
|
||||||
response = deleteCollection.process(cloudClient);
|
|
||||||
assertEquals("48", response.getResponse().get("requestid"));
|
|
||||||
state = getRequestStateAfterCompletion("48", MAX_TIMEOUT_SECONDS, cloudClient);
|
|
||||||
assertSame("DeleteCollection did not complete", RequestStatusState.COMPLETED, state);
|
assertSame("DeleteCollection did not complete", RequestStatusState.COMPLETED, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -161,7 +161,7 @@ public class BasicAuthIntegrationTest extends TestMiniSolrCloudClusterBase {
|
||||||
verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[2]/name", "collection-admin-edit", 20);
|
verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[2]/name", "collection-admin-edit", 20);
|
||||||
|
|
||||||
CollectionAdminRequest.Reload reload = new CollectionAdminRequest.Reload();
|
CollectionAdminRequest.Reload reload = new CollectionAdminRequest.Reload();
|
||||||
reload.setCollectionName(cloudSolrClient.getDefaultCollection());
|
reload.setCollectionName(defaultCollName);
|
||||||
|
|
||||||
HttpSolrClient solrClient = new HttpSolrClient(baseUrl);
|
HttpSolrClient solrClient = new HttpSolrClient(baseUrl);
|
||||||
try {
|
try {
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
Loading…
Reference in New Issue