Merge branch 'SOLR-8782'

This commit is contained in:
Alan Woodward 2016-03-08 10:53:39 +00:00
commit 406a1635c1
6 changed files with 662 additions and 369 deletions

View File

@ -22,6 +22,10 @@ See the Quick Start guide at http://lucene.apache.org/solr/quickstart.html
================== 6.1.0 ==================
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
----------------------

View File

@ -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>

View File

@ -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>

View File

@ -21,90 +21,80 @@ import java.util.List;
import org.apache.lucene.util.LuceneTestCase.Slow;
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.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.Create;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.SplitShard;
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.SolrInputDocument;
import org.apache.solr.common.cloud.Slice;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Tests the Cloud Collections API.
*/
@Slow
public class CollectionsAPIAsyncDistributedZkTest extends AbstractFullDistribZkTestBase {
public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
private static final int MAX_TIMEOUT_SECONDS = 60;
public CollectionsAPIAsyncDistributedZkTest() {
sliceCount = 1;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(2)
.addConfig("conf1", TEST_PATH().resolve("configsets").resolve("cloud-minimal").resolve("conf"))
.configure();
}
@Test
@ShardsFixed(num = 1)
public void testSolrJAPICalls() throws Exception {
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
Create createCollectionRequest = new Create()
.setCollectionName("testasynccollectioncreation")
.setNumShards(1)
.setConfigName("conf1")
.setAsyncId("1001");
createCollectionRequest.process(client);
RequestStatusState state = getRequestStateAfterCompletion("1001", MAX_TIMEOUT_SECONDS, client);
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
createCollectionRequest = new Create()
.setCollectionName("testasynccollectioncreation")
.setNumShards(1)
.setConfigName("conf1")
.setAsyncId("1002");
createCollectionRequest.process(client);
state = getRequestStateAfterCompletion("1002", MAX_TIMEOUT_SECONDS, client);
assertSame("Recreating a collection with the same should have failed.", RequestStatusState.FAILED, state);
CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
.setCollectionName("testasynccollectioncreation")
.setShardName("shard1")
.setAsyncId("1003");
client.request(addReplica);
state = getRequestStateAfterCompletion("1003", MAX_TIMEOUT_SECONDS, client);
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);
}
final CloudSolrClient client = cluster.getSolrClient();
RequestStatusState state = new Create()
.setCollectionName("testasynccollectioncreation")
.setNumShards(1)
.setReplicationFactor(1)
.setConfigName("conf1")
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
state = new Create()
.setCollectionName("testasynccollectioncreation")
.setNumShards(1)
.setConfigName("conf1")
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("Recreating a collection with the same should have failed.", RequestStatusState.FAILED, state);
state = new CollectionAdminRequest.AddReplica()
.setCollectionName("testasynccollectioncreation")
.setShardName("shard1")
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("Add replica did not complete", RequestStatusState.COMPLETED, state);
state = new SplitShard()
.setCollectionName("testasynccollectioncreation")
.setShardName("shard1")
.processAndWait(client, MAX_TIMEOUT_SECONDS * 2);
assertEquals("Shard split did not complete. Last recorded state: " + state, RequestStatusState.COMPLETED, state);
}
@Test
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)
.setNumShards(1)
.setRouterName("implicit")
.setShards("shard1")
.setConfigName("conf1")
.setAsyncId("42");
CollectionAdminResponse response = createCollectionRequest.process(cloudClient);
assertEquals("42", response.getResponse().get("requestid"));
RequestStatusState state = getRequestStateAfterCompletion("42", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("CreateCollection task did not complete!", RequestStatusState.COMPLETED, state);
//Add a few documents to shard1
@ -116,59 +106,48 @@ public class CollectionsAPIAsyncDistributedZkTest extends AbstractFullDistribZkT
doc.addField("_route_", "shard1");
docs.add(doc);
}
cloudClient.add(collection, docs);
cloudClient.commit(collection);
client.add(collection, docs);
client.commit(collection);
SolrQuery query = new SolrQuery("*:*");
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();
reloadCollection.setCollectionName(collection).setAsyncId("43");
response = reloadCollection.process(cloudClient);
assertEquals("43", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("43", MAX_TIMEOUT_SECONDS, cloudClient);
state = new CollectionAdminRequest.Reload()
.setCollectionName(collection)
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("ReloadCollection did not complete", RequestStatusState.COMPLETED, state);
CollectionAdminRequest.CreateShard createShard = new CollectionAdminRequest.CreateShard()
state = new CollectionAdminRequest.CreateShard()
.setCollectionName(collection)
.setShardName("shard2")
.setAsyncId("44");
response = createShard.process(cloudClient);
assertEquals("44", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("44", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("CreateShard did not complete", RequestStatusState.COMPLETED, state);
//Add a doc to shard2 to make sure shard2 was created properly
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", numDocs + 1);
doc.addField("_route_", "shard2");
cloudClient.add(collection, doc);
cloudClient.commit(collection);
client.add(collection, doc);
client.commit(collection);
query = new SolrQuery("*:*");
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)
.setShardName("shard2")
.setAsyncId("45");
response = deleteShard.process(cloudClient);
assertEquals("45", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("45", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("DeleteShard did not complete", RequestStatusState.COMPLETED, state);
CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
state = new CollectionAdminRequest.AddReplica()
.setCollectionName(collection)
.setShardName("shard1")
.setAsyncId("46");
response = addReplica.process(cloudClient);
assertEquals("46", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("46", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("AddReplica did not complete", RequestStatusState.COMPLETED, state);
//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;
while (shard1.getReplicas().size() != 2) {
if (count++ > 1000) {
@ -177,51 +156,40 @@ public class CollectionsAPIAsyncDistributedZkTest extends AbstractFullDistribZkT
Thread.sleep(100);
}
CollectionAdminRequest.CreateAlias createAlias = new CollectionAdminRequest.CreateAlias()
state = new CollectionAdminRequest.CreateAlias()
.setAliasName("myalias")
.setAliasedCollections(collection)
.setAsyncId("47");
response = createAlias.process(cloudClient);
assertEquals("47", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("47", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("CreateAlias did not complete", RequestStatusState.COMPLETED, state);
query = new SolrQuery("*:*");
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")
.setAsyncId("48");
response = deleteAlias.process(cloudClient);
assertEquals("48", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("48", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("DeleteAlias did not complete", RequestStatusState.COMPLETED, state);
try {
cloudClient.query("myalias", query);
client.query("myalias", query);
fail("Alias should not exist");
} catch (SolrException e) {
//expected
}
String replica = shard1.getReplicas().iterator().next().getName();
CollectionAdminRequest.DeleteReplica deleteReplica = new CollectionAdminRequest.DeleteReplica()
state = new CollectionAdminRequest.DeleteReplica()
.setCollectionName(collection)
.setShardName("shard1")
.setReplica(replica)
.setAsyncId("47");
response = deleteReplica.process(cloudClient);
assertEquals("47", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("47", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("DeleteReplica did not complete", RequestStatusState.COMPLETED, state);
CollectionAdminRequest.Delete deleteCollection = new CollectionAdminRequest.Delete()
state = new CollectionAdminRequest.Delete()
.setCollectionName(collection)
.setAsyncId("48");
response = deleteCollection.process(cloudClient);
assertEquals("48", response.getResponse().get("requestid"));
state = getRequestStateAfterCompletion("48", MAX_TIMEOUT_SECONDS, cloudClient);
.processAndWait(client, MAX_TIMEOUT_SECONDS);
assertSame("DeleteCollection did not complete", RequestStatusState.COMPLETED, state);
}
}

View File

@ -161,7 +161,7 @@ public class BasicAuthIntegrationTest extends TestMiniSolrCloudClusterBase {
verifySecurityStatus(cl, baseUrl + authzPrefix, "authorization/permissions[2]/name", "collection-admin-edit", 20);
CollectionAdminRequest.Reload reload = new CollectionAdminRequest.Reload();
reload.setCollectionName(cloudSolrClient.getDefaultCollection());
reload.setCollectionName(defaultCollName);
HttpSolrClient solrClient = new HttpSolrClient(baseUrl);
try {