mirror of https://github.com/apache/lucene.git
SOLR-8976: Add SolrJ support for REBALANCELEADERS Collections API
This commit is contained in:
parent
1c8164f7c0
commit
5baf311c38
|
@ -62,6 +62,8 @@ New Features
|
||||||
|
|
||||||
* SOLR-8938: Add optional -excluderegex argument to ZkCLI. (Christine Poerschke)
|
* SOLR-8938: Add optional -excluderegex argument to ZkCLI. (Christine Poerschke)
|
||||||
|
|
||||||
|
* SOLR-8976: Add SolrJ support for REBALANCELEADERS Collections API (Anshum Gupta)
|
||||||
|
|
||||||
Bug Fixes
|
Bug Fixes
|
||||||
----------------------
|
----------------------
|
||||||
|
|
||||||
|
|
|
@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.solr.client.solrj.SolrRequest;
|
import org.apache.solr.client.solrj.SolrRequest;
|
||||||
import org.apache.solr.client.solrj.SolrServerException;
|
import org.apache.solr.client.solrj.SolrServerException;
|
||||||
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
import org.apache.solr.client.solrj.impl.CloudSolrClient;
|
||||||
|
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
|
||||||
import org.apache.solr.client.solrj.request.QueryRequest;
|
import org.apache.solr.client.solrj.request.QueryRequest;
|
||||||
import org.apache.solr.common.cloud.Replica;
|
import org.apache.solr.common.cloud.Replica;
|
||||||
import org.apache.solr.common.cloud.Slice;
|
import org.apache.solr.common.cloud.Slice;
|
||||||
|
@ -276,7 +277,21 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
|
||||||
fail("Waited for timeout for preferredLeader assignments to be made and they werent.");
|
fail("Waited for timeout for preferredLeader assignments to be made and they werent.");
|
||||||
}
|
}
|
||||||
//fillExpectedWithCurrent();
|
//fillExpectedWithCurrent();
|
||||||
// Now rebalance the leaders
|
// Now rebalance the leaders randomly using SolrJ or direct call
|
||||||
|
if(random().nextBoolean())
|
||||||
|
rebalanceLeaderUsingSolrJAPI();
|
||||||
|
else
|
||||||
|
rebalanceLeaderUsingDirectCall();
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private void rebalanceLeaderUsingSolrJAPI() throws IOException, SolrServerException {
|
||||||
|
CollectionAdminRequest.RebalanceLeaders rebalanceLeaders = CollectionAdminRequest.rebalanceLeaders(COLLECTION_NAME);
|
||||||
|
rebalanceLeaders.setMaxAtOnce(10)
|
||||||
|
.process(cloudClient);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void rebalanceLeaderUsingDirectCall() throws IOException, SolrServerException {
|
||||||
ModifiableSolrParams params = new ModifiableSolrParams();
|
ModifiableSolrParams params = new ModifiableSolrParams();
|
||||||
params.set("action", CollectionParams.CollectionAction.REBALANCELEADERS.toString());
|
params.set("action", CollectionParams.CollectionAction.REBALANCELEADERS.toString());
|
||||||
|
|
||||||
|
@ -286,6 +301,7 @@ public class TestRebalanceLeaders extends AbstractFullDistribZkTestBase {
|
||||||
SolrRequest request = new QueryRequest(params);
|
SolrRequest request = new QueryRequest(params);
|
||||||
request.setPath("/admin/collections");
|
request.setPath("/admin/collections");
|
||||||
cloudClient.request(request);
|
cloudClient.request(request);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void issuePreferred(String slice, Replica rep) throws IOException, SolrServerException, InterruptedException {
|
void issuePreferred(String slice, Replica rep) throws IOException, SolrServerException, InterruptedException {
|
||||||
|
|
|
@ -490,6 +490,67 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Returns a RebalanceLeaders object to rebalance leaders for a collection
|
||||||
|
*/
|
||||||
|
public static RebalanceLeaders rebalanceLeaders(String collection) {
|
||||||
|
return new RebalanceLeaders(collection);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RebalanceLeaders extends AsyncCollectionAdminRequest {
|
||||||
|
|
||||||
|
protected Integer maxAtOnce;
|
||||||
|
protected Integer maxWaitSeconds;
|
||||||
|
protected String collection;
|
||||||
|
|
||||||
|
public RebalanceLeaders setMaxAtOnce(Integer maxAtOnce) {
|
||||||
|
this.maxAtOnce = maxAtOnce;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RebalanceLeaders setMaxWaitSeconds(Integer maxWaitSeconds) {
|
||||||
|
this.maxWaitSeconds = maxWaitSeconds;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer getMaxAtOnce() {
|
||||||
|
return maxAtOnce;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Integer getMaxWaitSeconds() {
|
||||||
|
return maxWaitSeconds;
|
||||||
|
}
|
||||||
|
|
||||||
|
public RebalanceLeaders(String collection) {
|
||||||
|
super(CollectionAction.REBALANCELEADERS);
|
||||||
|
this.collection = collection;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RebalanceLeaders setAsyncId(String id) {
|
||||||
|
this.asyncId = id;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SolrParams getParams() {
|
||||||
|
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
|
||||||
|
|
||||||
|
params.set(CoreAdminParams.COLLECTION, collection);
|
||||||
|
|
||||||
|
if(this.maxWaitSeconds != null) {
|
||||||
|
params.set("maxWaitSeconds", this.maxWaitSeconds);
|
||||||
|
}
|
||||||
|
|
||||||
|
if(this.maxAtOnce != null) {
|
||||||
|
params.set("maxAtOnce", this.maxAtOnce);
|
||||||
|
}
|
||||||
|
|
||||||
|
return params;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a SolrRequest to delete a collection
|
* Returns a SolrRequest to delete a collection
|
||||||
*/
|
*/
|
||||||
|
|
Loading…
Reference in New Issue