You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/07/13 02:35:54 UTC
svn commit: r1361017 - in /lucene/dev/trunk/solr:
core/src/java/org/apache/solr/cloud/
core/src/java/org/apache/solr/handler/admin/
core/src/test/org/apache/solr/cloud/
solrj/src/java/org/apache/solr/common/params/
Author: markrmiller
Date: Fri Jul 13 00:35:54 2012
New Revision: 1361017
URL: http://svn.apache.org/viewvc?rev=1361017&view=rev
Log:
Add reload command to collections api - also improve tests.
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/BasicDistributedZkTest.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.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=1361017&r1=1361016&r2=1361017&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 Fri Jul 13 00:35:54 2012
@@ -45,8 +45,12 @@ public class OverseerCollectionProcessor
public static final String CREATECOLLECTION = "createcollection";
+ public static final String RELOADCOLLECTION = "reloadcollection";
+
// TODO: use from Overseer?
private static final String QUEUE_OPERATION = "operation";
+
+
private static Logger log = LoggerFactory
.getLogger(OverseerCollectionProcessor.class);
@@ -124,77 +128,19 @@ public class OverseerCollectionProcessor
if (CREATECOLLECTION.equals(operation)) {
return createCollection(zkStateReader.getCloudState(), message);
} else if (DELETECOLLECTION.equals(operation)) {
- return deleteCollection(zkStateReader.getCloudState(), message);
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
+ params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true);
+ return collectionCmd(zkStateReader.getCloudState(), message, params);
+ } else if (RELOADCOLLECTION.equals(operation)) {
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set(CoreAdminParams.ACTION, CoreAdminAction.RELOAD.toString());
+ return collectionCmd(zkStateReader.getCloudState(), message, params);
}
// unknown command, toss it from our queue
return true;
}
-
- private boolean deleteCollection(CloudState cloudState, ZkNodeProps message) {
-
- String name = message.get("name");
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set(CoreAdminParams.ACTION, CoreAdminAction.UNLOAD.toString());
-
- Map<String,Slice> slices = cloudState.getCollectionStates().get(name);
-
- if (slices == null) {
- throw new SolrException(ErrorCode.BAD_REQUEST, "Could not find collection:" + name);
- }
-
- for (Map.Entry<String,Slice> entry : slices.entrySet()) {
- Slice slice = entry.getValue();
- Map<String,ZkNodeProps> shards = slice.getShards();
- Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet();
- for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) {
- final ZkNodeProps node = shardEntry.getValue();
- if (cloudState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
- params.set(CoreAdminParams.CORE, name);
- params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true);
- String replica = node.get(ZkStateReader.BASE_URL_PROP);
- ShardRequest sreq = new ShardRequest();
- // yes, they must use same admin handler path everywhere...
- params.set("qt", adminPath);
-
- sreq.purpose = 1;
- // TODO: this sucks
- if (replica.startsWith("http://")) replica = replica.substring(7);
- sreq.shards = new String[] {replica};
- sreq.actualShards = sreq.shards;
- sreq.params = params;
-
- shardHandler.submit(sreq, replica, sreq.params);
- }
- }
- }
-
- int failed = 0;
- ShardResponse srsp;
- do {
- srsp = shardHandler.takeCompletedOrError();
- if (srsp != null) {
- Throwable e = srsp.getException();
- if (e != null) {
- // should we retry?
- // TODO: we should return errors to the client
- // TODO: what if one fails and others succeed?
- failed++;
- log.error("Error talking to shard: " + srsp.getShard(), e);
- }
- }
- } while (srsp != null);
-
-
- // if all calls succeeded, return true
- if (failed > 0) {
- return false;
- }
- return true;
- }
-
- // TODO: bad name conflict with another method
private boolean createCollection(CloudState cloudState, ZkNodeProps message) {
// look at the replication factor and see if it matches reality
@@ -236,10 +182,13 @@ public class OverseerCollectionProcessor
Collections.shuffle(nodeList);
int numNodes = numShards * (numReplicas + 1);
- List<String> createOnNodes = nodeList.subList(0, Math.min(nodeList.size() -1, numNodes - 1));
+ List<String> createOnNodes = nodeList.subList(0, Math.min(nodeList.size(), numNodes));
+
+ log.info("Create collection " + name + " on " + createOnNodes);
for (String replica : createOnNodes) {
// TODO: this does not work if original url had _ in it
+ // We should have a master list
replica = replica.replaceAll("_", "/");
params.set(CoreAdminParams.NAME, name);
params.set("collection.configName", configName);
@@ -279,4 +228,64 @@ public class OverseerCollectionProcessor
}
return true;
}
+
+ private boolean collectionCmd(CloudState cloudState, ZkNodeProps message, ModifiableSolrParams params) {
+ log.info("Executing Collection Cmd : " + params);
+ String name = message.get("name");
+
+ Map<String,Slice> slices = cloudState.getCollectionStates().get(name);
+
+ if (slices == null) {
+ throw new SolrException(ErrorCode.BAD_REQUEST, "Could not find collection:" + name);
+ }
+
+ for (Map.Entry<String,Slice> entry : slices.entrySet()) {
+ Slice slice = entry.getValue();
+ Map<String,ZkNodeProps> shards = slice.getShards();
+ Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet();
+ for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) {
+ final ZkNodeProps node = shardEntry.getValue();
+ if (cloudState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
+ params.set(CoreAdminParams.CORE, node.get(ZkStateReader.CORE_NAME_PROP));
+
+ String replica = node.get(ZkStateReader.BASE_URL_PROP);
+ ShardRequest sreq = new ShardRequest();
+ // yes, they must use same admin handler path everywhere...
+ params.set("qt", adminPath);
+
+ sreq.purpose = 1;
+ // TODO: this sucks
+ if (replica.startsWith("http://")) replica = replica.substring(7);
+ sreq.shards = new String[] {replica};
+ sreq.actualShards = sreq.shards;
+ sreq.params = params;
+
+ shardHandler.submit(sreq, replica, sreq.params);
+ }
+ }
+ }
+
+ int failed = 0;
+ ShardResponse srsp;
+ do {
+ srsp = shardHandler.takeCompletedOrError();
+ if (srsp != null) {
+ Throwable e = srsp.getException();
+ if (e != null) {
+ // should we retry?
+ // TODO: we should return errors to the client
+ // TODO: what if one fails and others succeed?
+ failed++;
+ log.error("Error talking to shard: " + srsp.getShard(), e);
+ }
+ }
+ } while (srsp != null);
+
+
+ // if all calls succeeded, return true
+ if (failed > 0) {
+ return false;
+ }
+ return true;
+ }
}
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=1361017&r1=1361016&r2=1361017&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 Fri Jul 13 00:35:54 2012
@@ -99,6 +99,10 @@ public class CollectionsHandler extends
this.handleDeleteAction(req, rsp);
break;
}
+ case RELOAD: {
+ this.handleReloadAction(req, rsp);
+ break;
+ }
default: {
throw new RuntimeException("Unknown action: " + action);
@@ -109,7 +113,21 @@ public class CollectionsHandler extends
rsp.setHttpCaching(false);
}
+ private void handleReloadAction(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
+ log.info("Reloading Collection : " + req.getParamString());
+ String name = req.getParams().required().get("name");
+
+ ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
+ OverseerCollectionProcessor.RELOADCOLLECTION, "name", name);
+
+ // TODO: what if you want to block until the collection is available?
+ coreContainer.getZkController().getOverseerCollectionQueue().offer(ZkStateReader.toJSON(m));
+ }
+
+
private void handleDeleteAction(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
+ log.info("Deleting Collection : " + req.getParamString());
+
String name = req.getParams().required().get("name");
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
@@ -127,7 +145,7 @@ public class CollectionsHandler extends
// as well as specific replicas= options
private void handleCreateAction(SolrQueryRequest req,
SolrQueryResponse rsp) throws InterruptedException, KeeperException {
-
+ log.info("Creating Collection : " + req.getParamString());
Integer numReplicas = req.getParams().getInt("numReplicas", 0);
String name = req.getParams().required().get("name");
String configName = req.getParams().get("collection.configName");
Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java?rev=1361017&r1=1361016&r2=1361017&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/BasicDistributedZkTest.java Fri Jul 13 00:35:54 2012
@@ -37,6 +37,7 @@ import java.util.concurrent.ThreadPoolEx
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util._TestUtil;
import org.apache.solr.JSONTestUtil;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrRequest;
@@ -46,21 +47,25 @@ import org.apache.solr.client.solrj.impl
import org.apache.solr.client.solrj.impl.HttpSolrServer;
import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
import org.apache.solr.client.solrj.request.ContentStreamUpdateRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.UpdateResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.CloudState;
import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.update.SolrCmdDistributor.Request;
import org.apache.solr.util.DefaultSolrThreadFactory;
@@ -316,70 +321,216 @@ public class BasicDistributedZkTest exte
// if there is only one conf set. That and the fact that other tests run first in this
// env make this pretty fragile
- // create 2 new collections rapid fire
-
- ModifiableSolrParams params = new ModifiableSolrParams();
- params.set("action", CollectionAction.CREATE.toString());
- params.set("numReplicas", 1);
- params.set("numShards", 3);
- String collectionName = "awholynewcollection";
- params.set("name", collectionName);
- SolrRequest request = new QueryRequest(params);
- request.setPath("/admin/collections");
-
- clients.get(0).request(request);
+ // create new collections rapid fire
+ Map<String,List<Integer>> collectionInfos = new HashMap<String,List<Integer>>();
+ int cnt = atLeast(3);
+ for (int i = 0; i < cnt; i++) {
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set("action", CollectionAction.CREATE.toString());
+ int numShards = _TestUtil.nextInt(random(), 0, shardCount) + 1;
+ int numReplicas = _TestUtil.nextInt(random(), 0, 5) + 1;
+ params.set("numShards", numShards);
+ params.set("numReplicas", numReplicas);
+ String collectionName = "awholynewcollection_" + i;
+ int clientIndex = random().nextInt(2);
+ List<Integer> list = new ArrayList<Integer>();
+ list.add(numShards);
+ list.add(numReplicas);
+ collectionInfos.put(collectionName, list);
+ params.set("name", collectionName);
+ SolrRequest request = new QueryRequest(params);
+ request.setPath("/admin/collections");
+
+ clients.get(clientIndex).request(request);
+ }
+
+ Set<Entry<String,List<Integer>>> collectionInfosEntrySet = collectionInfos.entrySet();
+ for (Entry<String,List<Integer>> entry : collectionInfosEntrySet) {
+ String collection = entry.getKey();
+ List<Integer> list = entry.getValue();
+ checkForCollection(collection, list.get(0));
+
+ String url = getUrlFromZk(collection);
+
+ HttpSolrServer collectionClient = new HttpSolrServer(url);
+
+ // poll for a second - it can take a moment before we are ready to serve
+ waitForNon404(collectionClient);
+ }
- String collectionName2 = "awholynewcollection2";
- params.set("name", collectionName2);
- params.set("numShards", 2);
- clients.get(1).request(request);
+ List<String> collectionNameList = new ArrayList<String>();
+ collectionNameList.addAll(collectionInfos.keySet());
+ String collectionName = collectionNameList.get(random().nextInt(collectionNameList.size()));
- checkForCollection(collectionName, 3);
- checkForCollection(collectionName2, 2);
+ String url = getUrlFromZk(collectionName);
+ HttpSolrServer collectionClient = new HttpSolrServer(url);
+
+
// lets try and use the solrj client to index and retrieve a couple documents
- SolrInputDocument doc = getDoc(id, 6, i1, -600, tlong, 600, t1,
+ SolrInputDocument doc1 = getDoc(id, 6, i1, -600, tlong, 600, t1,
"humpty dumpy sat on a wall");
- int which = (doc.getField(id).toString().hashCode() & 0x7fffffff)
- % clients.size();
- SolrServer client = clients.get(which);
- client.add(doc);
+ SolrInputDocument doc2 = getDoc(id, 7, i1, -600, tlong, 600, t1,
+ "humpty dumpy3 sat on a walls");
+ SolrInputDocument doc3 = getDoc(id, 8, i1, -600, tlong, 600, t1,
+ "humpty dumpy2 sat on a walled");
+
+ collectionClient.add(doc1);
- doc = getDoc(id, 7, i1, 123, tlong, 123, t1,
- "humpty dumpy had a great fall");
- which = (doc.getField(id).toString().hashCode() & 0x7fffffff)
- % clients.size();
- client = clients.get(which);
- client.add(doc);
- doc = getDoc(id, 8, i1, 876, tlong, 876, t1,
- "all the kings horses and all the kings men");
- which = (doc.getField(id).toString().hashCode() & 0x7fffffff)
- % clients.size();
- client = clients.get(which);
- client.add(doc);
+ collectionClient.add(doc2);
+
+ collectionClient.add(doc3);
- commit();
+ collectionClient.commit();
+
+ assertEquals(3, collectionClient.query(new SolrQuery("*:*")).getResults().getNumFound());
+
+ // lets try a collection reload
+
+ // get core open times
+ Map<String,Long> urlToTimeBefore = new HashMap<String,Long>();
+ collectStartTimes(collectionName, urlToTimeBefore);
+ assertTrue(urlToTimeBefore.size() > 0);
+ ModifiableSolrParams params = new ModifiableSolrParams();
+ params.set("action", CollectionAction.RELOAD.toString());
+ params.set("name", collectionName);
+ QueryRequest request = new QueryRequest(params);
+ request.setPath("/admin/collections");
+
+ // we can use this client because we just want base url
+ clients.get(0).request(request);
+
+ // reloads make take a short while
+ boolean allTimesAreCorrect = waitForReloads(collectionName, urlToTimeBefore);
+ assertTrue("some core start times did not change on reload", allTimesAreCorrect);
// remove a collection
params = new ModifiableSolrParams();
params.set("action", CollectionAction.DELETE.toString());
- params.set("name", collectionName2);
+ params.set("name", collectionName);
request = new QueryRequest(params);
request.setPath("/admin/collections");
clients.get(0).request(request);
// ensure its out of the state
- checkForMissingCollection(collectionName2);
+ checkForMissingCollection(collectionName);
+
+ }
+
+ private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
+
+ long timeoutAt = System.currentTimeMillis() + 30000;
+
+ boolean allTimesAreCorrect = false;
+ while (System.currentTimeMillis() < timeoutAt) {
+ Map<String,Long> urlToTimeAfter = new HashMap<String,Long>();
+ collectStartTimes(collectionName, urlToTimeAfter);
+
+ boolean retry = false;
+ Set<Entry<String,Long>> entries = urlToTimeBefore.entrySet();
+ for (Entry<String,Long> entry : entries) {
+ Long beforeTime = entry.getValue();
+ Long afterTime = urlToTimeAfter.get(entry.getKey());
+ assertNotNull(afterTime);
+ if (afterTime <= beforeTime) {
+ retry = true;
+ break;
+ }
+
+ }
+ if (!retry) {
+ allTimesAreCorrect = true;
+ break;
+ }
+ }
+ return allTimesAreCorrect;
+ }
+
+ private void collectStartTimes(String collectionName,
+ Map<String,Long> urlToTime) throws SolrServerException, IOException {
+ Map<String,Map<String,Slice>> collections = solrj.getZkStateReader()
+ .getCloudState().getCollectionStates();
+ if (collections.containsKey(collectionName)) {
+ Map<String,Slice> slices = collections.get(collectionName);
+
+ Iterator<Entry<String,Slice>> it = slices.entrySet().iterator();
+ while (it.hasNext()) {
+ Entry<String,Slice> sliceEntry = it.next();
+ Map<String,ZkNodeProps> sliceShards = sliceEntry.getValue().getShards();
+ Iterator<Entry<String,ZkNodeProps>> shardIt = sliceShards.entrySet()
+ .iterator();
+ while (shardIt.hasNext()) {
+ Entry<String,ZkNodeProps> shardEntry = shardIt.next();
+ ZkCoreNodeProps coreProps = new ZkCoreNodeProps(shardEntry.getValue());
+ CoreAdminResponse mcr = CoreAdminRequest.getStatus(
+ coreProps.getCoreName(),
+ new HttpSolrServer(coreProps.getBaseUrl()));
+ long before = mcr.getStartTime(coreProps.getCoreName()).getTime();
+ urlToTime.put(coreProps.getCoreUrl(), before);
+ }
+ }
+ } else {
+ throw new IllegalArgumentException("Could not find collection in :"
+ + collections.keySet());
+ }
+ }
+
+ private String getUrlFromZk(String collection) {
+ CloudState cloudState = solrj.getZkStateReader().getCloudState();
+ Map<String,Slice> slices = cloudState.getCollectionStates().get(collection);
+
+ if (slices == null) {
+ throw new SolrException(ErrorCode.BAD_REQUEST, "Could not find collection:" + collection);
+ }
+
+ for (Map.Entry<String,Slice> entry : slices.entrySet()) {
+ Slice slice = entry.getValue();
+ Map<String,ZkNodeProps> shards = slice.getShards();
+ Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet();
+ for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) {
+ final ZkNodeProps node = shardEntry.getValue();
+ if (cloudState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) {
+ return new ZkCoreNodeProps(node).getCoreUrl();
+ }
+ }
+ }
+
+ throw new RuntimeException("Could not find a live node for collection:" + collection);
+ }
+
+ private void waitForNon404(HttpSolrServer collectionClient)
+ throws Exception {
+
+ long timeoutAt = System.currentTimeMillis() + 30000;
+
+ while (System.currentTimeMillis() < timeoutAt) {
+ boolean missing = false;
+ try {
+ collectionClient.query(new SolrQuery("*:*"));
+ } catch (SolrException e) {
+ // How do I get the response code!?
+ if (!e.getMessage().contains("(404)")) {
+ throw e;
+ }
+ missing = true;
+ }
+ if (!missing) {
+ return;
+ }
+ Thread.sleep(50);
+ }
printLayout();
+ fail("Could not find the new collection - 404 : " + collectionClient.getBaseURL());
}
private void checkForCollection(String collectionName, int expectedSlices)
throws Exception {
// check for an expectedSlices new collection - we poll the state
- long timeoutAt = System.currentTimeMillis() + 30000;
+ long timeoutAt = System.currentTimeMillis() + 60000;
boolean found = false;
+ boolean sliceMatch = false;
while (System.currentTimeMillis() < timeoutAt) {
solrj.getZkStateReader().updateCloudState(true);
CloudState cloudState = solrj.getZkStateReader().getCloudState();
@@ -389,6 +540,7 @@ public class BasicDistributedZkTest exte
Map<String,Slice> slices = collections.get(collectionName);
// did we find expectedSlices slices/shards?
if (slices.size() == expectedSlices) {
+ sliceMatch = true;
found = true;
// also make sure each are active
Iterator<Entry<String,Slice>> it = slices.entrySet().iterator();
@@ -406,15 +558,19 @@ public class BasicDistributedZkTest exte
break;
}
}
-
}
if (found) break;
}
}
- Thread.sleep(50);
+ Thread.sleep(100);
}
if (!found) {
- fail("Could not find new " + expectedSlices + " slice collection");
+ printLayout();
+ if (!sliceMatch) {
+ fail("Could not find new " + expectedSlices + " slice collection called " + collectionName);
+ } else {
+ fail("Found expected # of slices, but some nodes are not active for collection called " + collectionName);
+ }
}
}
@@ -432,7 +588,7 @@ public class BasicDistributedZkTest exte
found = false;
break;
}
- Thread.sleep(50);
+ Thread.sleep(100);
}
if (found) {
fail("Found collection that should be gone " + collectionName);
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java?rev=1361017&r1=1361016&r2=1361017&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java Fri Jul 13 00:35:54 2012
@@ -28,7 +28,7 @@ public interface CollectionParams
public enum CollectionAction {
- CREATE, DELETE;
+ CREATE, DELETE, RELOAD;
public static CollectionAction get( String p )
{