You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by an...@apache.org on 2014/04/01 06:52:54 UTC
svn commit: r1583532 - in /lucene/dev/trunk/solr/core/src:
java/org/apache/solr/cloud/OverseerCollectionProcessor.java
java/org/apache/solr/handler/admin/CollectionsHandler.java
test/org/apache/solr/cloud/AsyncMigrateRouteKeyTest.java
Author: anshum
Date: Tue Apr 1 04:52:54 2014
New Revision: 1583532
URL: http://svn.apache.org/r1583532
Log:
SOLR-5908: Make the REQUESTSTATUS Collection API call non-blocking and non-blocked.
Modified:
lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AsyncMigrateRouteKeyTest.java
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1583532&r1=1583531&r2=1583532&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Tue Apr 1 04:52:54 2014
@@ -471,8 +471,6 @@ public class OverseerCollectionProcessor
processRoleCommand(message, operation);
} else if (ADDREPLICA.isEqual(operation)) {
addReplica(zkStateReader.getClusterState(), message, results);
- } else if (REQUESTSTATUS.equals(operation)) {
- requestStatus(message, results);
} else if (OVERSEERSTATUS.isEqual(operation)) {
getOverseerStatus(message, results);
} else if(LIST.isEqual(operation)) {
@@ -1494,40 +1492,6 @@ public class OverseerCollectionProcessor
} while (srsp != null);
}
- private void requestStatus(ZkNodeProps message, NamedList results) throws KeeperException, InterruptedException {
- log.info("Request status invoked");
- String requestId = message.getStr(REQUESTID);
-
- // Special taskId (-1), clears up the request state maps.
- if(requestId.equals("-1")) {
- completedMap.clear();
- failureMap.clear();
- return;
- }
-
- if(completedMap.contains(requestId)) {
- SimpleOrderedMap success = new SimpleOrderedMap();
- success.add("state", "completed");
- success.add("msg", "found " + requestId + " in completed tasks");
- results.add("status", success);
- } else if (runningMap.contains(requestId)) {
- SimpleOrderedMap success = new SimpleOrderedMap();
- success.add("state", "running");
- success.add("msg", "found " + requestId + " in submitted tasks");
- results.add("status", success);
- } else if (failureMap.contains(requestId)) {
- SimpleOrderedMap success = new SimpleOrderedMap();
- success.add("state", "failed");
- success.add("msg", "found " + requestId + " in failed tasks");
- results.add("status", success);
- } else {
- SimpleOrderedMap failure = new SimpleOrderedMap();
- failure.add("state", "notfound");
- failure.add("msg", "Did not find taskid [" + requestId + "] in any tasks queue");
- results.add("status", failure);
- }
- }
-
private void deleteShard(ClusterState clusterState, ZkNodeProps message, NamedList results) {
log.info("Delete shard invoked");
String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java?rev=1583532&r1=1583531&r2=1583532&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java Tue Apr 1 04:52:54 2014
@@ -265,11 +265,43 @@ public class CollectionsHandler extends
private void handleRequestStatus(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
log.debug("REQUESTSTATUS action invoked: " + req.getParamString());
req.getParams().required().check(REQUESTID);
- Map<String, Object> props = new HashMap<String, Object>();
- props.put(Overseer.QUEUE_OPERATION, OverseerCollectionProcessor.REQUESTSTATUS);
- props.put(REQUESTID, req.getParams().get(REQUESTID));
- ZkNodeProps m = new ZkNodeProps(props);
- handleResponse(OverseerCollectionProcessor.REQUESTSTATUS, m, rsp);
+
+ String requestId = req.getParams().get(REQUESTID);
+
+ if (requestId.equals("-1")) {
+ // Special taskId (-1), clears up the request state maps.
+ if(requestId.equals("-1")) {
+ coreContainer.getZkController().getOverseerCompletedMap().clear();
+ coreContainer.getZkController().getOverseerFailureMap().clear();
+ return;
+ }
+ } else {
+ NamedList<Object> results = new NamedList<>();
+ if (coreContainer.getZkController().getOverseerCompletedMap().contains(requestId)) {
+ SimpleOrderedMap success = new SimpleOrderedMap();
+ success.add("state", "completed");
+ success.add("msg", "found " + requestId + " in completed tasks");
+ results.add("status", success);
+ } else if (coreContainer.getZkController().getOverseerRunningMap().contains(requestId)) {
+ SimpleOrderedMap success = new SimpleOrderedMap();
+ success.add("state", "running");
+ success.add("msg", "found " + requestId + " in submitted tasks");
+ results.add("status", success);
+ } else if (coreContainer.getZkController().getOverseerFailureMap().contains(requestId)) {
+ SimpleOrderedMap success = new SimpleOrderedMap();
+ success.add("state", "failed");
+ success.add("msg", "found " + requestId + " in failed tasks");
+ results.add("status", success);
+ } else {
+ SimpleOrderedMap failure = new SimpleOrderedMap();
+ failure.add("state", "notfound");
+ failure.add("msg", "Did not find taskid [" + requestId + "] in any tasks queue");
+ results.add("status", failure);
+ }
+ SolrResponse response = new OverseerSolrResponse(results);
+
+ rsp.getValues().addAll(response.getResponse());
+ }
}
private void handleResponse(String operation, ZkNodeProps m,
Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AsyncMigrateRouteKeyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AsyncMigrateRouteKeyTest.java?rev=1583532&r1=1583531&r2=1583532&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AsyncMigrateRouteKeyTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/AsyncMigrateRouteKeyTest.java Tue Apr 1 04:52:54 2014
@@ -53,7 +53,11 @@ public class AsyncMigrateRouteKeyTest ex
params = new ModifiableSolrParams();
params.set("action", CollectionParams.CollectionAction.REQUESTSTATUS.toString());
params.set(OverseerCollectionProcessor.REQUESTID, asyncId);
- message = sendStatusRequestWithRetry(params, 10);
+ // This task takes long enough to run. Also check for the current state of the task to be running.
+ message = sendStatusRequestWithRetry(params, 2);
+ assertEquals("found " + asyncId + " in submitted tasks", message);
+ // Now wait until the task actually completes successfully/fails.
+ message = sendStatusRequestWithRetry(params, 20);
assertEquals("Task " + asyncId + " not found in completed tasks.",
"found " + asyncId + " in completed tasks", message);
}
@@ -92,7 +96,6 @@ public class AsyncMigrateRouteKeyTest ex
if (state.equals("completed") || state.equals("failed"))
return (String) status.get("msg");
-
try {
Thread.sleep(1000);
} catch (InterruptedException e) {