SOLR-6670: change BALANCESLICEUNIQUE to BALANCESHARDUNIQUE

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1636226 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Erick Erickson 2014-11-03 01:52:39 +00:00
parent 17ab4645d5
commit 7702efb849
7 changed files with 56 additions and 53 deletions

View File

@ -198,6 +198,9 @@ New Features
have similar content. It works in standalone/cloud mode and shares logic with the
Lucene MoreLikeThis class (Anshum Gupta).
* SOLR-6670: change BALANCESLICEUNIQUE to BALANCESHARDUNIQUE. Also, the parameter
for ADDREPLICAPROP that used to be sliceUnique is not shardUnique. (Erick Erickson)
Bug Fixes
----------------------

View File

@ -18,11 +18,11 @@ package org.apache.solr.cloud;
*/
import static java.util.Collections.singletonMap;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SLICE_UNIQUE;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.OverseerCollectionProcessor.COLL_PROP_PREFIX;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESLICEUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import java.io.Closeable;
import java.io.IOException;
@ -475,7 +475,7 @@ public class Overseer implements Closeable {
case DELETEREPLICAPROP:
clusterState = deleteReplicaProp(clusterState, message);
break;
case BALANCESLICEUNIQUE:
case BALANCESHARDUNIQUE:
ExclusiveSliceProperty dProp = new ExclusiveSliceProperty(this, clusterState, message);
if (dProp.balanceProperty()) {
String collName = message.getStr(ZkStateReader.COLLECTION_PROP);
@ -571,19 +571,19 @@ public class Overseer implements Closeable {
}
property = property.toLowerCase(Locale.ROOT);
String propVal = message.getStr(ZkStateReader.PROPERTY_VALUE_PROP);
String sliceUnique = message.getStr(OverseerCollectionProcessor.SLICE_UNIQUE);
String shardUnique = message.getStr(OverseerCollectionProcessor.SHARD_UNIQUE);
boolean isUnique = false;
if (sliceUniqueBooleanProperties.contains(property)) {
if (StringUtils.isNotBlank(sliceUnique) && Boolean.parseBoolean(sliceUnique) == false) {
if (StringUtils.isNotBlank(shardUnique) && Boolean.parseBoolean(shardUnique) == false) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Overseer SETREPLICAPROPERTY for " +
property + " cannot have " + OverseerCollectionProcessor.SLICE_UNIQUE + " set to anything other than" +
property + " cannot have " + OverseerCollectionProcessor.SHARD_UNIQUE + " set to anything other than" +
"'true'. No action taken");
}
isUnique = true;
} else {
isUnique = Boolean.parseBoolean(sliceUnique);
isUnique = Boolean.parseBoolean(shardUnique);
}
Replica replica = clusterState.getReplica(collectionName, replicaName);
@ -1456,12 +1456,12 @@ public class Overseer implements Closeable {
ZkStateReader.PROPERTY_PROP + "' parameters. No action taken ");
}
Boolean sliceUnique = Boolean.parseBoolean(message.getStr(SLICE_UNIQUE));
if (sliceUnique == false &&
Boolean shardUnique = Boolean.parseBoolean(message.getStr(SHARD_UNIQUE));
if (shardUnique == false &&
Overseer.sliceUniqueBooleanProperties.contains(this.property) == false) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Balancing properties amongst replicas in a slice requires that"
+ " the property be a pre-defined property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true' " +
" Property: " + this.property + " sliceUnique: " + Boolean.toString(sliceUnique));
+ " the property be a pre-defined property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true' " +
" Property: " + this.property + " shardUnique: " + Boolean.toString(shardUnique));
}
collection = clusterState.getCollection(collectionName);
@ -1508,7 +1508,7 @@ public class Overseer implements Closeable {
if (StringUtils.isNotBlank(replica.getStr(property))) {
if (sliceHasProp) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"'" + BALANCESLICEUNIQUE + "' should only be called for properties that have at most one member " +
"'" + BALANCESHARDUNIQUE + "' should only be called for properties that have at most one member " +
"in any slice with the property set. No action taken.");
}
if (nodesHostingProp.containsKey(nodeName) == false) {

View File

@ -28,7 +28,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESLICEUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERSTATUS;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
@ -155,7 +155,7 @@ public class OverseerCollectionProcessor implements Runnable, Closeable {
public static final String ONLY_IF_DOWN = "onlyIfDown";
public static final String SLICE_UNIQUE = "sliceUnique";
public static final String SHARD_UNIQUE = "shardUnique";
public static final String ONLY_ACTIVE_NODES = "onlyactivenodes";
@ -650,7 +650,7 @@ public class OverseerCollectionProcessor implements Runnable, Closeable {
case DELETEREPLICAPROP:
processReplicaDeletePropertyCommand(message);
break;
case BALANCESLICEUNIQUE:
case BALANCESHARDUNIQUE:
balanceProperty(message);
break;
case REBALANCELEADERS:
@ -749,12 +749,12 @@ public class OverseerCollectionProcessor implements Runnable, Closeable {
if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) {
throw new SolrException(ErrorCode.BAD_REQUEST,
"The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP +
"' parameters are required for the BALANCESLICEUNIQUE operation, no action taken");
"' parameters are required for the BALANCESHARDUNIQUE operation, no action taken");
}
SolrZkClient zkClient = zkStateReader.getZkClient();
DistributedQueue inQueue = Overseer.getInQueue(zkClient);
Map<String, Object> propMap = new HashMap<>();
propMap.put(Overseer.QUEUE_OPERATION, BALANCESLICEUNIQUE.toLower());
propMap.put(Overseer.QUEUE_OPERATION, BALANCESHARDUNIQUE.toLower());
propMap.putAll(message.getProperties());
inQueue.offer(ZkStateReader.toJSON(new ZkNodeProps(propMap)));
}

View File

@ -21,7 +21,7 @@ import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
import static org.apache.solr.cloud.OverseerCollectionProcessor.ASYNC;
import static org.apache.solr.cloud.OverseerCollectionProcessor.COLL_CONF;
import static org.apache.solr.cloud.OverseerCollectionProcessor.CREATE_NODE_SET;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SLICE_UNIQUE;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
import static org.apache.solr.cloud.OverseerCollectionProcessor.NUM_SLICES;
import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_ACTIVE_NODES;
import static org.apache.solr.cloud.OverseerCollectionProcessor.ONLY_IF_DOWN;
@ -46,7 +46,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.MAX_WAIT_SECONDS_PROP;
import static org.apache.solr.common.cloud.ZkStateReader.STATE_PROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDROLE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICAPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESLICEUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.BALANCESHARDUNIQUE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CLUSTERPROP;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATEALIAS;
@ -258,8 +258,8 @@ public class CollectionsHandler extends RequestHandlerBase {
this.handleDeleteReplicaProp(req, rsp);
break;
}
case BALANCESLICEUNIQUE: {
this.handleBalanceSliceUnique(req, rsp);
case BALANCESHARDUNIQUE: {
this.handleBalanceShardUnique(req, rsp);
break;
}
case REBALANCELEADERS: {
@ -430,23 +430,23 @@ public class CollectionsHandler extends RequestHandlerBase {
Map<String, Object> map = ZkNodeProps.makeMap(Overseer.QUEUE_OPERATION, ADDREPLICAPROP.toLower());
copyIfNotNull(req.getParams(), map, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP,
SLICE_UNIQUE, PROPERTY_VALUE_PROP);
SHARD_UNIQUE, PROPERTY_VALUE_PROP);
String property = (String) map.get(PROPERTY_PROP);
if (property.startsWith(OverseerCollectionProcessor.COLL_PROP_PREFIX) == false) {
property = OverseerCollectionProcessor.COLL_PROP_PREFIX + property;
}
boolean uniquePerSlice = Boolean.parseBoolean((String) map.get(SLICE_UNIQUE));
boolean uniquePerSlice = Boolean.parseBoolean((String) map.get(SHARD_UNIQUE));
// Check if we're trying to set a property with parameters that allow us to set the property on multiple replicas
// in a slice on properties that are known to only be one-per-slice and error out if so.
if (StringUtils.isNotBlank((String)map.get(SLICE_UNIQUE)) &&
if (StringUtils.isNotBlank((String)map.get(SHARD_UNIQUE)) &&
Overseer.sliceUniqueBooleanProperties.contains(property.toLowerCase(Locale.ROOT)) &&
uniquePerSlice == false) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
"Overseer replica property command received for property " + property +
" with the " + SLICE_UNIQUE +
" with the " + SHARD_UNIQUE +
" parameter set to something other than 'true'. No action taken.");
}
handleResponse(ADDREPLICAPROP.toLower(), new ZkNodeProps(map), rsp);
@ -463,25 +463,25 @@ public class CollectionsHandler extends RequestHandlerBase {
private void handleBalanceSliceUnique(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
private void handleBalanceShardUnique(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {
req.getParams().required().check(COLLECTION_PROP, PROPERTY_PROP);
Boolean sliceUnique = Boolean.parseBoolean(req.getParams().get(SLICE_UNIQUE));
Boolean shardUnique = Boolean.parseBoolean(req.getParams().get(SHARD_UNIQUE));
String prop = req.getParams().get(PROPERTY_PROP).toLowerCase(Locale.ROOT);
if (StringUtils.startsWith(prop, OverseerCollectionProcessor.COLL_PROP_PREFIX) == false) {
prop = OverseerCollectionProcessor.COLL_PROP_PREFIX + prop;
}
if (sliceUnique == false &&
if (shardUnique == false &&
Overseer.sliceUniqueBooleanProperties.contains(prop) == false) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Balancing properties amongst replicas in a slice requires that"
+ " the property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true'. " +
" Property: " + prop + " sliceUnique: " + Boolean.toString(sliceUnique));
+ " the property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'. " +
" Property: " + prop + " shardUnique: " + Boolean.toString(shardUnique));
}
Map<String, Object> map = ZkNodeProps.makeMap(Overseer.QUEUE_OPERATION, BALANCESLICEUNIQUE.toLower());
copyIfNotNull(req.getParams(), map, COLLECTION_PROP, PROPERTY_PROP, ONLY_ACTIVE_NODES, SLICE_UNIQUE);
Map<String, Object> map = ZkNodeProps.makeMap(Overseer.QUEUE_OPERATION, BALANCESHARDUNIQUE.toLower());
copyIfNotNull(req.getParams(), map, COLLECTION_PROP, PROPERTY_PROP, ONLY_ACTIVE_NODES, SHARD_UNIQUE);
handleResponse(BALANCESLICEUNIQUE.toLower(), new ZkNodeProps(map), rsp);
handleResponse(BALANCESHARDUNIQUE.toLower(), new ZkNodeProps(map), rsp);
}
private void handleOverseerStatus(SolrQueryRequest req, SolrQueryResponse rsp) throws KeeperException, InterruptedException {

View File

@ -35,7 +35,7 @@ import org.apache.solr.common.util.NamedList;
import org.apache.zookeeper.KeeperException;
import org.junit.Before;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SLICE_UNIQUE;
import static org.apache.solr.cloud.OverseerCollectionProcessor.SHARD_UNIQUE;
import java.io.IOException;
import java.util.ArrayList;
@ -429,7 +429,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "testprop",
"property.value", "nonsense",
SLICE_UNIQUE, "true");
SHARD_UNIQUE, "true");
verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
@ -446,7 +446,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "property.testprop",
"property.value", "true",
SLICE_UNIQUE, "false");
SHARD_UNIQUE, "false");
verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");
verifyPropertyVal(client, COLLECTION_NAME, c1_s2_r1, "property.preferredleader", "true");
@ -477,11 +477,11 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
"replica", c1_s1_r1,
"property", "preferredLeader",
"property.value", "true",
SLICE_UNIQUE, "false");
fail("Should have thrown an exception, setting sliceUnique=false is not allowed for 'preferredLeader'.");
SHARD_UNIQUE, "false");
fail("Should have thrown an exception, setting shardUnique=false is not allowed for 'preferredLeader'.");
} catch (SolrException se) {
assertTrue("Should have received a specific error message",
se.getMessage().contains("with the sliceUnique parameter set to something other than 'true'"));
se.getMessage().contains("with the shardUnique parameter set to something other than 'true'"));
}
verifyPropertyVal(client, COLLECTION_NAME, c1_s1_r2, "property.preferredleader", "true");

View File

@ -106,7 +106,7 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
client.connect();
try {
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
"property", "preferredLeader");
} catch (SolrException se) {
assertTrue("Should have seen missing required parameter 'collection' error",
@ -114,30 +114,30 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
}
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
"collection", COLLECTION_NAME,
"property", "preferredLeader");
verifyUniqueAcrossCollection(client, COLLECTION_NAME, "property.preferredleader");
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
"collection", COLLECTION_NAME,
"property", "property.newunique",
"sliceUnique", "true");
"shardUnique", "true");
verifyUniqueAcrossCollection(client, COLLECTION_NAME, "property.newunique");
try {
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
"collection", COLLECTION_NAME,
"property", "whatever",
"sliceUnique", "false");
"shardUnique", "false");
fail("Should have thrown an exception here.");
} catch (SolrException se) {
assertTrue("Should have gotten a specific error message here",
se.getMessage().contains("Balancing properties amongst replicas in a slice requires that the " +
"property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true'"));
"property be pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'"));
}
// Should be able to set non-unique-per-slice values in several places.
Map<String, Slice> slices = client.getZkStateReader().getClusterState().getCollection(COLLECTION_NAME).getSlicesMap();
@ -165,24 +165,24 @@ public class TestReplicaProperties extends ReplicaPropertiesBase {
try {
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
"collection", COLLECTION_NAME,
"property", "bogus1",
"sliceUnique", "false");
"shardUnique", "false");
fail("Should have thrown parameter error here");
} catch (SolrException se) {
assertTrue("Should have caught specific exception ",
se.getMessage().contains("Balancing properties amongst replicas in a slice requires that the property be " +
"pre-defined as a unique property (e.g. 'preferredLeader') or that 'sliceUnique' be set to 'true'"));
"pre-defined as a unique property (e.g. 'preferredLeader') or that 'shardUnique' be set to 'true'"));
}
// Should have no effect despite the "sliceUnique" param being set.
// Should have no effect despite the "shardUnique" param being set.
doPropertyAction(client,
"action", CollectionParams.CollectionAction.BALANCESLICEUNIQUE.toString(),
"action", CollectionParams.CollectionAction.BALANCESHARDUNIQUE.toString(),
"collection", COLLECTION_NAME,
"property", "property.bogus1",
"sliceUnique", "true");
"shardUnique", "true");
verifyPropertyVal(client, COLLECTION_NAME,
c1_s1_r1, "property.bogus1", "true");

View File

@ -49,7 +49,7 @@ public interface CollectionParams
CLUSTERSTATUS,
ADDREPLICAPROP,
DELETEREPLICAPROP,
BALANCESLICEUNIQUE,
BALANCESHARDUNIQUE,
REBALANCELEADERS;
public static CollectionAction get( String p )