SOLR-3815: separate ZkNodeProps from Replica, change properties from String values to Object values

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1382621 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Yonik Seeley 2012-09-10 02:14:50 +00:00
parent 926277b304
commit b34c9a9683
46 changed files with 467 additions and 307 deletions

View File

@ -19,7 +19,7 @@ package org.apache.solr;
import org.apache.solr.cloud.ZkController; import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
import org.apache.solr.request.SolrQueryRequest; import org.apache.solr.request.SolrQueryRequest;
@ -99,7 +99,7 @@ public class SolrLogFormatter extends Formatter {
static int maxCoreNum; static int maxCoreNum;
String shortId; String shortId;
String url; String url;
Map<String, String> coreProps; Map<String, Object> coreProps;
} }
Map<SolrCore, CoreInfo> coreInfoMap = new WeakHashMap<SolrCore, CoreInfo>(); // TODO: use something that survives across a core reload? Map<SolrCore, CoreInfo> coreInfoMap = new WeakHashMap<SolrCore, CoreInfo>(); // TODO: use something that survives across a core reload?
@ -200,7 +200,7 @@ sb.append("(group_name=").append(tg.getName()).append(")");
info.coreProps = getCoreProps(zkController, core); info.coreProps = getCoreProps(zkController, core);
} }
Map<String, String> coreProps = getCoreProps(zkController, core); Map<String, Object> coreProps = getCoreProps(zkController, core);
if(!coreProps.equals(info.coreProps)) { if(!coreProps.equals(info.coreProps)) {
info.coreProps = coreProps; info.coreProps = coreProps;
final String corePropsString = "coll:" + core.getCoreDescriptor().getCloudDescriptor().getCollectionName() + " core:" + core.getName() + " props:" + coreProps; final String corePropsString = "coll:" + core.getCoreDescriptor().getCloudDescriptor().getCollectionName() + " core:" + core.getName() + " props:" + coreProps;
@ -261,9 +261,9 @@ sb.append("(group_name=").append(tg.getName()).append(")");
return sb.toString(); return sb.toString();
} }
private Map<String,String> getCoreProps(ZkController zkController, SolrCore core) { private Map<String,Object> getCoreProps(ZkController zkController, SolrCore core) {
final String collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName(); final String collection = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
ZkNodeProps props = zkController.getClusterState().getShardProps(collection, ZkStateReader.getCoreNodeName(zkController.getNodeName(), core.getName())); Replica props = zkController.getClusterState().getShardProps(collection, ZkStateReader.getCoreNodeName(zkController.getNodeName(), core.getName()));
if(props!=null) { if(props!=null) {
return props.getProperties(); return props.getProperties();
} }

View File

@ -56,7 +56,7 @@ public class AssignShard {
// else figure out which shard needs more replicas // else figure out which shard needs more replicas
final Map<String, Integer> map = new HashMap<String, Integer>(); final Map<String, Integer> map = new HashMap<String, Integer>();
for (String shardId : shardIdNames) { for (String shardId : shardIdNames) {
int cnt = sliceMap.get(shardId).getShards().size(); int cnt = sliceMap.get(shardId).getReplicasMap().size();
map.put(shardId, cnt); map.put(shardId, cnt);
} }

View File

@ -8,6 +8,7 @@ import java.util.Set;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
@ -89,10 +90,10 @@ class ShardLeaderElectionContextBase extends ElectionContext {
zkClient.makePath(leaderPath, ZkStateReader.toJSON(leaderProps), zkClient.makePath(leaderPath, ZkStateReader.toJSON(leaderProps),
CreateMode.EPHEMERAL, true); CreateMode.EPHEMERAL, true);
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "leader", ZkNodeProps m = ZkNodeProps.fromKeyVals(Overseer.QUEUE_OPERATION, "leader",
ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP,
collection, ZkStateReader.BASE_URL_PROP, leaderProps.getProperties() collection, ZkStateReader.BASE_URL_PROP, leaderProps.getProperties()
.get(ZkStateReader.BASE_URL_PROP), ZkStateReader.CORE_NAME_PROP, .get(ZkStateReader.BASE_URL_PROP), ZkStateReader.CORE_NAME_PROP,
leaderProps.getProperties().get(ZkStateReader.CORE_NAME_PROP), leaderProps.getProperties().get(ZkStateReader.CORE_NAME_PROP),
ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE); ZkStateReader.STATE_PROP, ZkStateReader.ACTIVE);
Overseer.getInQueue(zkClient).offer(ZkStateReader.toJSON(m)); Overseer.getInQueue(zkClient).offer(ZkStateReader.toJSON(m));
@ -133,7 +134,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
InterruptedException, IOException { InterruptedException, IOException {
log.info("Running the leader process. afterExpiration=" + afterExpiration); log.info("Running the leader process. afterExpiration=" + afterExpiration);
String coreName = leaderProps.get(ZkStateReader.CORE_NAME_PROP); String coreName = leaderProps.getStr(ZkStateReader.CORE_NAME_PROP);
// clear the leader in clusterstate // clear the leader in clusterstate
ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "leader", ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "leader",
@ -244,11 +245,11 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
while (true && !isClosed) { while (true && !isClosed) {
// wait for everyone to be up // wait for everyone to be up
if (slices != null) { if (slices != null) {
Map<String,ZkNodeProps> shards = slices.getShards(); Map<String,Replica> shards = slices.getReplicasMap();
Set<Entry<String,ZkNodeProps>> entrySet = shards.entrySet(); Set<Entry<String,Replica>> entrySet = shards.entrySet();
int found = 0; int found = 0;
tryAgain = false; tryAgain = false;
for (Entry<String,ZkNodeProps> entry : entrySet) { for (Entry<String,Replica> entry : entrySet) {
ZkCoreNodeProps props = new ZkCoreNodeProps(entry.getValue()); ZkCoreNodeProps props = new ZkCoreNodeProps(entry.getValue());
if (props.getState().equals(ZkStateReader.ACTIVE) if (props.getState().equals(ZkStateReader.ACTIVE)
&& zkController.getClusterState().liveNodesContain( && zkController.getClusterState().liveNodesContain(
@ -259,16 +260,16 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
// on startup and after connection timeout, wait for all known shards // on startup and after connection timeout, wait for all known shards
if ((afterExpiration || !weAreReplacement) if ((afterExpiration || !weAreReplacement)
&& found >= slices.getShards().size()) { && found >= slices.getReplicasMap().size()) {
log.info("Enough replicas found to continue."); log.info("Enough replicas found to continue.");
break; break;
} else if (!afterExpiration && found >= slices.getShards().size() - 1) { } else if (!afterExpiration && found >= slices.getReplicasMap().size() - 1) {
// a previous leader went down - wait for one less than the total // a previous leader went down - wait for one less than the total
// known shards // known shards
log.info("Enough replicas found to continue."); log.info("Enough replicas found to continue.");
break; break;
} else { } else {
log.info("Waiting until we see more replicas up: total=" + slices.getShards().size() + " found=" + found + " timeoutin=" + (timeoutAt - System.currentTimeMillis())); log.info("Waiting until we see more replicas up: total=" + slices.getReplicasMap().size() + " found=" + found + " timeoutin=" + (timeoutAt - System.currentTimeMillis()));
} }
if (System.currentTimeMillis() > timeoutAt) { if (System.currentTimeMillis() > timeoutAt) {
@ -310,16 +311,16 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
ClusterState clusterState = zkController.getZkStateReader().getClusterState(); ClusterState clusterState = zkController.getZkStateReader().getClusterState();
Map<String,Slice> slices = clusterState.getSlices(this.collection); Map<String,Slice> slices = clusterState.getSlices(this.collection);
Slice slice = slices.get(shardId); Slice slice = slices.get(shardId);
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
boolean foundSomeoneElseActive = false; boolean foundSomeoneElseActive = false;
for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) { for (Map.Entry<String,Replica> shard : shards.entrySet()) {
String state = shard.getValue().get(ZkStateReader.STATE_PROP); String state = shard.getValue().getStr(ZkStateReader.STATE_PROP);
if (new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals( if (new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
new ZkCoreNodeProps(leaderProps).getCoreUrl())) { new ZkCoreNodeProps(leaderProps).getCoreUrl())) {
if (state.equals(ZkStateReader.ACTIVE) if (state.equals(ZkStateReader.ACTIVE)
&& clusterState.liveNodesContain(shard.getValue().get( && clusterState.liveNodesContain(shard.getValue().getStr(
ZkStateReader.NODE_NAME_PROP))) { ZkStateReader.NODE_NAME_PROP))) {
// we are alive // we are alive
log.info("I am Active and live, it's okay to be the leader."); log.info("I am Active and live, it's okay to be the leader.");
return true; return true;
@ -327,8 +328,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
} }
if ((state.equals(ZkStateReader.ACTIVE)) if ((state.equals(ZkStateReader.ACTIVE))
&& clusterState.liveNodesContain(shard.getValue().get( && clusterState.liveNodesContain(shard.getValue().getStr(
ZkStateReader.NODE_NAME_PROP)) ZkStateReader.NODE_NAME_PROP))
&& !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals( && !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
new ZkCoreNodeProps(leaderProps).getCoreUrl())) { new ZkCoreNodeProps(leaderProps).getCoreUrl())) {
foundSomeoneElseActive = true; foundSomeoneElseActive = true;
@ -354,15 +355,15 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
ClusterState clusterState = zkController.getZkStateReader().getClusterState(); ClusterState clusterState = zkController.getZkStateReader().getClusterState();
Map<String,Slice> slices = clusterState.getSlices(this.collection); Map<String,Slice> slices = clusterState.getSlices(this.collection);
Slice slice = slices.get(shardId); Slice slice = slices.get(shardId);
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) { for (Map.Entry<String,Replica> shard : shards.entrySet()) {
String state = shard.getValue().get(ZkStateReader.STATE_PROP); String state = shard.getValue().getStr(ZkStateReader.STATE_PROP);
if ((state.equals(ZkStateReader.ACTIVE)) if ((state.equals(ZkStateReader.ACTIVE))
&& clusterState.liveNodesContain(shard.getValue().get( && clusterState.liveNodesContain(shard.getValue().getStr(
ZkStateReader.NODE_NAME_PROP))) { ZkStateReader.NODE_NAME_PROP))) {
return true; return true;
} }
} }

View File

@ -26,6 +26,7 @@ import java.util.Map.Entry;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ClosableThread; import org.apache.solr.common.cloud.ClosableThread;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
@ -87,7 +88,7 @@ public class Overseer {
while (head != null && amILeader()) { while (head != null && amILeader()) {
final ZkNodeProps message = ZkNodeProps.load(head); final ZkNodeProps message = ZkNodeProps.load(head);
final String operation = message final String operation = message
.get(QUEUE_OPERATION); .getStr(QUEUE_OPERATION);
clusterState = processMessage(clusterState, message, operation); clusterState = processMessage(clusterState, message, operation);
zkClient.setData(ZkStateReader.CLUSTER_STATE, zkClient.setData(ZkStateReader.CLUSTER_STATE,
ZkStateReader.toJSON(clusterState), true); ZkStateReader.toJSON(clusterState), true);
@ -123,7 +124,7 @@ public class Overseer {
while (head != null) { while (head != null) {
final ZkNodeProps message = ZkNodeProps.load(head); final ZkNodeProps message = ZkNodeProps.load(head);
final String operation = message.get(QUEUE_OPERATION); final String operation = message.getStr(QUEUE_OPERATION);
clusterState = processMessage(clusterState, message, operation); clusterState = processMessage(clusterState, message, operation);
workQueue.offer(head); workQueue.offer(head);
@ -168,15 +169,15 @@ public class Overseer {
} else if (ZkStateReader.LEADER_PROP.equals(operation)) { } else if (ZkStateReader.LEADER_PROP.equals(operation)) {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
String baseUrl = message.get(ZkStateReader.BASE_URL_PROP); String baseUrl = message.getStr(ZkStateReader.BASE_URL_PROP);
String coreName = message.get(ZkStateReader.CORE_NAME_PROP); String coreName = message.getStr(ZkStateReader.CORE_NAME_PROP);
sb.append(baseUrl); sb.append(baseUrl);
if (baseUrl != null && !baseUrl.endsWith("/")) sb.append("/"); if (baseUrl != null && !baseUrl.endsWith("/")) sb.append("/");
sb.append(coreName == null ? "" : coreName); sb.append(coreName == null ? "" : coreName);
if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/"); if (!(sb.substring(sb.length() - 1).equals("/"))) sb.append("/");
clusterState = setShardLeader(clusterState, clusterState = setShardLeader(clusterState,
message.get(ZkStateReader.COLLECTION_PROP), message.getStr(ZkStateReader.COLLECTION_PROP),
message.get(ZkStateReader.SHARD_ID_PROP), message.getStr(ZkStateReader.SHARD_ID_PROP),
sb.length() > 0 ? sb.toString() : null); sb.length() > 0 ? sb.toString() : null);
} else { } else {
@ -189,7 +190,7 @@ public class Overseer {
private boolean amILeader() { private boolean amILeader() {
try { try {
ZkNodeProps props = ZkNodeProps.load(zkClient.getData("/overseer_elect/leader", null, null, true)); ZkNodeProps props = ZkNodeProps.load(zkClient.getData("/overseer_elect/leader", null, null, true));
if(myId.equals(props.get("id"))) { if(myId.equals(props.getStr("id"))) {
return true; return true;
} }
} catch (KeeperException e) { } catch (KeeperException e) {
@ -204,9 +205,9 @@ public class Overseer {
* Try to assign core to the cluster. * Try to assign core to the cluster.
*/ */
private ClusterState updateState(ClusterState state, final ZkNodeProps message) { private ClusterState updateState(ClusterState state, final ZkNodeProps message) {
final String collection = message.get(ZkStateReader.COLLECTION_PROP); final String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
final String zkCoreNodeName = message.get(ZkStateReader.NODE_NAME_PROP) + "_" + message.get(ZkStateReader.CORE_NAME_PROP); final String zkCoreNodeName = message.getStr(ZkStateReader.NODE_NAME_PROP) + "_" + message.getStr(ZkStateReader.CORE_NAME_PROP);
final Integer numShards = message.get(ZkStateReader.NUM_SHARDS_PROP)!=null?Integer.parseInt(message.get(ZkStateReader.NUM_SHARDS_PROP)):null; final Integer numShards = message.getStr(ZkStateReader.NUM_SHARDS_PROP)!=null?Integer.parseInt(message.getStr(ZkStateReader.NUM_SHARDS_PROP)):null;
//collection does not yet exist, create placeholders if num shards is specified //collection does not yet exist, create placeholders if num shards is specified
if (!state.getCollections().contains(collection) if (!state.getCollections().contains(collection)
@ -215,9 +216,9 @@ public class Overseer {
} }
// use the provided non null shardId // use the provided non null shardId
String shardId = message.get(ZkStateReader.SHARD_ID_PROP); String shardId = message.getStr(ZkStateReader.SHARD_ID_PROP);
if (shardId == null) { if (shardId == null) {
String nodeName = message.get(ZkStateReader.NODE_NAME_PROP); String nodeName = message.getStr(ZkStateReader.NODE_NAME_PROP);
//get shardId from ClusterState //get shardId from ClusterState
shardId = getAssignedId(state, nodeName, message); shardId = getAssignedId(state, nodeName, message);
} }
@ -226,22 +227,22 @@ public class Overseer {
shardId = AssignShard.assignShard(collection, state, numShards); shardId = AssignShard.assignShard(collection, state, numShards);
} }
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
Map<String,String> coreProps = new HashMap<String,String>(message.getProperties().size()); Map<String,Object> coreProps = new HashMap<String,Object>(message.getProperties().size());
coreProps.putAll(message.getProperties()); coreProps.putAll(message.getProperties());
// we don't put num_shards in the clusterstate // we don't put num_shards in the clusterstate
coreProps.remove(ZkStateReader.NUM_SHARDS_PROP); coreProps.remove(ZkStateReader.NUM_SHARDS_PROP);
coreProps.remove(QUEUE_OPERATION); coreProps.remove(QUEUE_OPERATION);
for (Entry<String,String> entry : coreProps.entrySet()) { for (Entry<String,Object> entry : coreProps.entrySet()) {
props.put(entry.getKey(), entry.getValue()); props.put(entry.getKey(), entry.getValue());
} }
ZkNodeProps zkProps = new ZkNodeProps(props); Replica zkProps = new Replica(zkCoreNodeName, props);
Slice slice = state.getSlice(collection, shardId); Slice slice = state.getSlice(collection, shardId);
Map<String,ZkNodeProps> shardProps; Map<String,Replica> shardProps;
if (slice == null) { if (slice == null) {
shardProps = new HashMap<String,ZkNodeProps>(); shardProps = new HashMap<String,Replica>();
} else { } else {
shardProps = state.getSlice(collection, shardId).getShardsCopy(); shardProps = state.getSlice(collection, shardId).getReplicasCopy();
} }
shardProps.put(zkCoreNodeName, zkProps); shardProps.put(zkCoreNodeName, zkProps);
@ -268,11 +269,11 @@ public class Overseer {
*/ */
private String getAssignedId(final ClusterState state, final String nodeName, private String getAssignedId(final ClusterState state, final String nodeName,
final ZkNodeProps coreState) { final ZkNodeProps coreState) {
final String key = coreState.get(ZkStateReader.NODE_NAME_PROP) + "_" + coreState.get(ZkStateReader.CORE_NAME_PROP); final String key = coreState.getStr(ZkStateReader.NODE_NAME_PROP) + "_" + coreState.getStr(ZkStateReader.CORE_NAME_PROP);
Map<String, Slice> slices = state.getSlices(coreState.get(ZkStateReader.COLLECTION_PROP)); Map<String, Slice> slices = state.getSlices(coreState.getStr(ZkStateReader.COLLECTION_PROP));
if (slices != null) { if (slices != null) {
for (Slice slice : slices.values()) { for (Slice slice : slices.values()) {
if (slice.getShards().get(key) != null) { if (slice.getReplicasMap().get(key) != null) {
return slice.getName(); return slice.getName();
} }
} }
@ -293,16 +294,16 @@ public class Overseer {
if (!slices.containsKey(slice.getName())) { if (!slices.containsKey(slice.getName())) {
slices.put(slice.getName(), slice); slices.put(slice.getName(), slice);
} else { } else {
final Map<String,ZkNodeProps> shards = new LinkedHashMap<String,ZkNodeProps>(); final Map<String,Replica> shards = new LinkedHashMap<String,Replica>();
final Slice existingSlice = slices.get(slice.getName()); final Slice existingSlice = slices.get(slice.getName());
shards.putAll(existingSlice.getShards()); shards.putAll(existingSlice.getReplicasMap());
//XXX preserve existing leader //XXX preserve existing leader
for(Entry<String, ZkNodeProps> edit: slice.getShards().entrySet()) { for(Entry<String, Replica> edit: slice.getReplicasMap().entrySet()) {
if(existingSlice.getShards().get(edit.getKey())!=null && existingSlice.getShards().get(edit.getKey()).containsKey(ZkStateReader.LEADER_PROP)) { if(existingSlice.getReplicasMap().get(edit.getKey())!=null && existingSlice.getReplicasMap().get(edit.getKey()).containsKey(ZkStateReader.LEADER_PROP)) {
HashMap<String, String> newProps = new HashMap<String,String>(); HashMap<String, Object> newProps = new HashMap<String,Object>();
newProps.putAll(edit.getValue().getProperties()); newProps.putAll(edit.getValue().getProperties());
newProps.put(ZkStateReader.LEADER_PROP, existingSlice.getShards().get(edit.getKey()).get(ZkStateReader.LEADER_PROP)); newProps.put(ZkStateReader.LEADER_PROP, existingSlice.getReplicasMap().get(edit.getKey()).getStr(ZkStateReader.LEADER_PROP));
shards.put(edit.getKey(), new ZkNodeProps(newProps)); shards.put(edit.getKey(), new Replica(edit.getKey(), newProps));
} else { } else {
shards.put(edit.getKey(), edit.getValue()); shards.put(edit.getKey(), edit.getValue());
} }
@ -329,9 +330,9 @@ public class Overseer {
log.error("Could not mark leader for non existing slice:" + sliceName); log.error("Could not mark leader for non existing slice:" + sliceName);
return state; return state;
} else { } else {
final Map<String,ZkNodeProps> newShards = new LinkedHashMap<String,ZkNodeProps>(); final Map<String,Replica> newShards = new LinkedHashMap<String,Replica>();
for(Entry<String, ZkNodeProps> shard: slices.get(sliceName).getShards().entrySet()) { for(Entry<String, Replica> shard: slices.get(sliceName).getReplicasMap().entrySet()) {
Map<String, String> newShardProps = new LinkedHashMap<String,String>(); Map<String, Object> newShardProps = new LinkedHashMap<String,Object>();
newShardProps.putAll(shard.getValue().getProperties()); newShardProps.putAll(shard.getValue().getProperties());
newShardProps.remove(ZkStateReader.LEADER_PROP); //clean any previously existed flag newShardProps.remove(ZkStateReader.LEADER_PROP); //clean any previously existed flag
@ -340,7 +341,7 @@ public class Overseer {
if(leaderUrl!=null && leaderUrl.equals(zkCoreNodeProps.getCoreUrl())) { if(leaderUrl!=null && leaderUrl.equals(zkCoreNodeProps.getCoreUrl())) {
newShardProps.put(ZkStateReader.LEADER_PROP,"true"); newShardProps.put(ZkStateReader.LEADER_PROP,"true");
} }
newShards.put(shard.getKey(), new ZkNodeProps(newShardProps)); newShards.put(shard.getKey(), new Replica(shard.getKey(), newShardProps));
} }
Slice slice = new Slice(sliceName, newShards); Slice slice = new Slice(sliceName, newShards);
slices.put(sliceName, slice); slices.put(sliceName, slice);
@ -353,8 +354,8 @@ public class Overseer {
*/ */
private ClusterState removeCore(final ClusterState clusterState, ZkNodeProps message) { private ClusterState removeCore(final ClusterState clusterState, ZkNodeProps message) {
final String coreNodeName = message.get(ZkStateReader.NODE_NAME_PROP) + "_" + message.get(ZkStateReader.CORE_NAME_PROP); final String coreNodeName = message.getStr(ZkStateReader.NODE_NAME_PROP) + "_" + message.getStr(ZkStateReader.CORE_NAME_PROP);
final String collection = message.get(ZkStateReader.COLLECTION_PROP); final String collection = message.getStr(ZkStateReader.COLLECTION_PROP);
final LinkedHashMap<String, Map<String, Slice>> newStates = new LinkedHashMap<String,Map<String,Slice>>(); final LinkedHashMap<String, Map<String, Slice>> newStates = new LinkedHashMap<String,Map<String,Slice>>();
for(String collectionName: clusterState.getCollections()) { for(String collectionName: clusterState.getCollections()) {
@ -362,9 +363,9 @@ public class Overseer {
Map<String, Slice> slices = clusterState.getSlices(collection); Map<String, Slice> slices = clusterState.getSlices(collection);
LinkedHashMap<String, Slice> newSlices = new LinkedHashMap<String, Slice>(); LinkedHashMap<String, Slice> newSlices = new LinkedHashMap<String, Slice>();
for(Slice slice: slices.values()) { for(Slice slice: slices.values()) {
if(slice.getShards().containsKey(coreNodeName)) { if(slice.getReplicasMap().containsKey(coreNodeName)) {
LinkedHashMap<String, ZkNodeProps> newShards = new LinkedHashMap<String, ZkNodeProps>(); LinkedHashMap<String, Replica> newShards = new LinkedHashMap<String, Replica>();
newShards.putAll(slice.getShards()); newShards.putAll(slice.getReplicasMap());
newShards.remove(coreNodeName); newShards.remove(coreNodeName);
Slice newSlice = new Slice(slice.getName(), newShards); Slice newSlice = new Slice(slice.getName(), newShards);
@ -376,7 +377,7 @@ public class Overseer {
} }
int cnt = 0; int cnt = 0;
for (Slice slice : newSlices.values()) { for (Slice slice : newSlices.values()) {
cnt+=slice.getShards().size(); cnt+=slice.getReplicasMap().size();
} }
// TODO: if no nodes are left after this unload // TODO: if no nodes are left after this unload
// remove from zk - do we have a race where Overseer // remove from zk - do we have a race where Overseer

View File

@ -26,6 +26,7 @@ import java.util.Set;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
@ -84,13 +85,13 @@ public class OverseerCollectionProcessor implements Runnable {
//if (head != null) { // should not happen since we block above //if (head != null) { // should not happen since we block above
final ZkNodeProps message = ZkNodeProps.load(head); final ZkNodeProps message = ZkNodeProps.load(head);
final String operation = message.get(QUEUE_OPERATION); final String operation = message.getStr(QUEUE_OPERATION);
boolean success = processMessage(message, operation); boolean success = processMessage(message, operation);
if (!success) { if (!success) {
// TODO: what to do on failure / partial failure // TODO: what to do on failure / partial failure
// if we fail, do we clean up then ? // if we fail, do we clean up then ?
SolrException.log(log, "Collection creation of " + message.get("name") + " failed"); SolrException.log(log, "Collection creation of " + message.getStr("name") + " failed");
} }
//} //}
workQueue.remove(); workQueue.remove();
@ -118,7 +119,7 @@ public class OverseerCollectionProcessor implements Runnable {
try { try {
ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData( ZkNodeProps props = ZkNodeProps.load(zkStateReader.getZkClient().getData(
"/overseer_elect/leader", null, null, true)); "/overseer_elect/leader", null, null, true));
if (myId.equals(props.get("id"))) { if (myId.equals(props.getStr("id"))) {
return true; return true;
} }
} catch (KeeperException e) { } catch (KeeperException e) {
@ -152,7 +153,7 @@ public class OverseerCollectionProcessor implements Runnable {
// look at the replication factor and see if it matches reality // look at the replication factor and see if it matches reality
// if it does not, find best nodes to create more cores // if it does not, find best nodes to create more cores
String numReplicasString = message.get("numReplicas"); String numReplicasString = message.getStr("numReplicas");
int numReplicas; int numReplicas;
try { try {
numReplicas = numReplicasString == null ? 0 : Integer.parseInt(numReplicasString); numReplicas = numReplicasString == null ? 0 : Integer.parseInt(numReplicasString);
@ -160,7 +161,7 @@ public class OverseerCollectionProcessor implements Runnable {
SolrException.log(log, "Could not parse numReplicas", ex); SolrException.log(log, "Could not parse numReplicas", ex);
return false; return false;
} }
String numShardsString = message.get("numShards"); String numShardsString = message.getStr("numShards");
int numShards; int numShards;
try { try {
numShards = numShardsString == null ? 0 : Integer.parseInt(numShardsString); numShards = numShardsString == null ? 0 : Integer.parseInt(numShardsString);
@ -169,8 +170,8 @@ public class OverseerCollectionProcessor implements Runnable {
return false; return false;
} }
String name = message.get("name"); String name = message.getStr("name");
String configName = message.get("collection.configName"); String configName = message.getStr("collection.configName");
// we need to look at every node and see how many cores it serves // we need to look at every node and see how many cores it serves
// add our new cores to existing nodes serving the least number of cores // add our new cores to existing nodes serving the least number of cores
@ -237,7 +238,7 @@ public class OverseerCollectionProcessor implements Runnable {
private boolean collectionCmd(ClusterState clusterState, ZkNodeProps message, ModifiableSolrParams params) { private boolean collectionCmd(ClusterState clusterState, ZkNodeProps message, ModifiableSolrParams params) {
log.info("Executing Collection Cmd : " + params); log.info("Executing Collection Cmd : " + params);
String name = message.get("name"); String name = message.getStr("name");
Map<String,Slice> slices = clusterState.getCollectionStates().get(name); Map<String,Slice> slices = clusterState.getCollectionStates().get(name);
@ -247,14 +248,14 @@ public class OverseerCollectionProcessor implements Runnable {
for (Map.Entry<String,Slice> entry : slices.entrySet()) { for (Map.Entry<String,Slice> entry : slices.entrySet()) {
Slice slice = entry.getValue(); Slice slice = entry.getValue();
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet(); Set<Map.Entry<String,Replica>> shardEntries = shards.entrySet();
for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) { for (Map.Entry<String,Replica> shardEntry : shardEntries) {
final ZkNodeProps node = shardEntry.getValue(); final ZkNodeProps node = shardEntry.getValue();
if (clusterState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) { if (clusterState.liveNodesContain(node.getStr(ZkStateReader.NODE_NAME_PROP))) {
params.set(CoreAdminParams.CORE, node.get(ZkStateReader.CORE_NAME_PROP)); params.set(CoreAdminParams.CORE, node.getStr(ZkStateReader.CORE_NAME_PROP));
String replica = node.get(ZkStateReader.BASE_URL_PROP); String replica = node.getStr(ZkStateReader.BASE_URL_PROP);
ShardRequest sreq = new ShardRequest(); ShardRequest sreq = new ShardRequest();
// yes, they must use same admin handler path everywhere... // yes, they must use same admin handler path everywhere...
params.set("qt", adminPath); params.set("qt", adminPath);

View File

@ -119,7 +119,7 @@ public class RecoveryStrategy extends Thread implements ClosableThread {
private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops, String baseUrl) private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops, String baseUrl)
throws SolrServerException, IOException { throws SolrServerException, IOException {
String leaderBaseUrl = leaderprops.get(ZkStateReader.BASE_URL_PROP); String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
ZkCoreNodeProps leaderCNodeProps = new ZkCoreNodeProps(leaderprops); ZkCoreNodeProps leaderCNodeProps = new ZkCoreNodeProps(leaderprops);
String leaderUrl = leaderCNodeProps.getCoreUrl(); String leaderUrl = leaderCNodeProps.getCoreUrl();
@ -318,8 +318,8 @@ public class RecoveryStrategy extends Thread implements ClosableThread {
ZkNodeProps leaderprops = zkStateReader.getLeaderProps( ZkNodeProps leaderprops = zkStateReader.getLeaderProps(
cloudDesc.getCollectionName(), cloudDesc.getShardId()); cloudDesc.getCollectionName(), cloudDesc.getShardId());
String leaderBaseUrl = leaderprops.get(ZkStateReader.BASE_URL_PROP); String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
String leaderCoreName = leaderprops.get(ZkStateReader.CORE_NAME_PROP); String leaderCoreName = leaderprops.getStr(ZkStateReader.CORE_NAME_PROP);
String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderBaseUrl, leaderCoreName); String leaderUrl = ZkCoreNodeProps.getCoreUrl(leaderBaseUrl, leaderCoreName);

View File

@ -29,6 +29,7 @@ import org.apache.solr.client.solrj.impl.HttpSolrServer;
import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery; import org.apache.solr.client.solrj.request.CoreAdminRequest.RequestRecovery;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -136,9 +137,9 @@ public class SyncStrategy {
ClusterState clusterState = zkController.getZkStateReader().getClusterState(); ClusterState clusterState = zkController.getZkStateReader().getClusterState();
Map<String,Slice> slices = clusterState.getSlices(collection); Map<String,Slice> slices = clusterState.getSlices(collection);
Slice slice = slices.get(shardId); Slice slice = slices.get(shardId);
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) { for (Map.Entry<String,Replica> shard : shards.entrySet()) {
String state = shard.getValue().get(ZkStateReader.STATE_PROP); String state = shard.getValue().getStr(ZkStateReader.STATE_PROP);
// System.out.println("state:" // System.out.println("state:"
// + state // + state
// + shard.getValue().get(ZkStateReader.NODE_NAME_PROP) // + shard.getValue().get(ZkStateReader.NODE_NAME_PROP)
@ -146,8 +147,8 @@ public class SyncStrategy {
// + clusterState.liveNodesContain(shard.getValue().get( // + clusterState.liveNodesContain(shard.getValue().get(
// ZkStateReader.NODE_NAME_PROP))); // ZkStateReader.NODE_NAME_PROP)));
if ((state.equals(ZkStateReader.ACTIVE)) if ((state.equals(ZkStateReader.ACTIVE))
&& clusterState.liveNodesContain(shard.getValue().get( && clusterState.liveNodesContain(shard.getValue().getStr(
ZkStateReader.NODE_NAME_PROP)) ZkStateReader.NODE_NAME_PROP))
&& !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals( && !new ZkCoreNodeProps(shard.getValue()).getCoreUrl().equals(
new ZkCoreNodeProps(leaderProps).getCoreUrl())) { new ZkCoreNodeProps(leaderProps).getCoreUrl())) {
return true; return true;
@ -161,8 +162,8 @@ public class SyncStrategy {
ZkNodeProps props, String collection, String shardId) { ZkNodeProps props, String collection, String shardId) {
List<ZkCoreNodeProps> nodes = zkController.getZkStateReader() List<ZkCoreNodeProps> nodes = zkController.getZkStateReader()
.getReplicaProps(collection, shardId, .getReplicaProps(collection, shardId,
props.get(ZkStateReader.NODE_NAME_PROP), props.getStr(ZkStateReader.NODE_NAME_PROP),
props.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.ACTIVE); // TODO: props.getStr(ZkStateReader.CORE_NAME_PROP), ZkStateReader.ACTIVE); // TODO:
// TODO should there be a state filter? // TODO should there be a state filter?
if (nodes == null) { if (nodes == null) {
@ -191,8 +192,8 @@ public class SyncStrategy {
List<ZkCoreNodeProps> nodes = zkController List<ZkCoreNodeProps> nodes = zkController
.getZkStateReader() .getZkStateReader()
.getReplicaProps(collection, shardId, .getReplicaProps(collection, shardId,
leaderProps.get(ZkStateReader.NODE_NAME_PROP), leaderProps.getStr(ZkStateReader.NODE_NAME_PROP),
leaderProps.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.ACTIVE); leaderProps.getStr(ZkStateReader.CORE_NAME_PROP), ZkStateReader.ACTIVE);
if (nodes == null) { if (nodes == null) {
log.info(ZkCoreNodeProps.getCoreUrl(leaderProps) + " has no replicas"); log.info(ZkCoreNodeProps.getCoreUrl(leaderProps) + " has no replicas");
return; return;

View File

@ -41,7 +41,6 @@ import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.OnReconnect; import org.apache.solr.common.cloud.OnReconnect;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkClientConnectionStrategy;
import org.apache.solr.common.cloud.ZkCmdExecutor; import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -492,7 +491,7 @@ public final class ZkController {
if(data != null) { if(data != null) {
ZkNodeProps props = ZkNodeProps.load(data); ZkNodeProps props = ZkNodeProps.load(data);
configName = props.get(CONFIGNAME_PROP); configName = props.getStr(CONFIGNAME_PROP);
} }
if (configName != null && !zkClient.exists(CONFIGS_ZKNODE + "/" + configName, true)) { if (configName != null && !zkClient.exists(CONFIGS_ZKNODE + "/" + configName, true)) {
@ -539,7 +538,7 @@ public final class ZkController {
String shardId = cloudDesc.getShardId(); String shardId = cloudDesc.getShardId();
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
// we only put a subset of props into the leader node // we only put a subset of props into the leader node
props.put(ZkStateReader.BASE_URL_PROP, baseUrl); props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
props.put(ZkStateReader.CORE_NAME_PROP, coreName); props.put(ZkStateReader.CORE_NAME_PROP, coreName);
@ -695,7 +694,7 @@ public final class ZkController {
String shardId = cd.getCloudDescriptor().getShardId(); String shardId = cd.getCloudDescriptor().getShardId();
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
// we only put a subset of props into the leader node // we only put a subset of props into the leader node
props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl()); props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
props.put(ZkStateReader.CORE_NAME_PROP, cd.getName()); props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
@ -872,7 +871,7 @@ public final class ZkController {
SolrParams params = cd.getParams(); SolrParams params = cd.getParams();
try { try {
Map<String,String> collectionProps = new HashMap<String,String>(); Map<String,Object> collectionProps = new HashMap<String,Object>();
// TODO: if collection.configName isn't set, and there isn't already a conf in zk, just use that? // TODO: if collection.configName isn't set, and there isn't already a conf in zk, just use that?
String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX+CONFIGNAME_PROP, collection); String defaultConfigName = System.getProperty(COLLECTION_PARAM_PREFIX+CONFIGNAME_PROP, collection);
@ -937,7 +936,7 @@ public final class ZkController {
private void getConfName(String collection, String collectionPath, private void getConfName(String collection, String collectionPath,
Map<String,String> collectionProps) throws KeeperException, Map<String,Object> collectionProps) throws KeeperException,
InterruptedException { InterruptedException {
// check for configName // check for configName
log.info("Looking for collection configName"); log.info("Looking for collection configName");
@ -1168,7 +1167,7 @@ public final class ZkController {
ZkNodeProps props = null; ZkNodeProps props = null;
if(data != null) { if(data != null) {
props = ZkNodeProps.load(data); props = ZkNodeProps.load(data);
Map<String,String> newProps = new HashMap<String,String>(); Map<String,Object> newProps = new HashMap<String,Object>();
newProps.putAll(props.getProperties()); newProps.putAll(props.getProperties());
newProps.put(CONFIGNAME_PROP, confSetName); newProps.put(CONFIGNAME_PROP, confSetName);
props = new ZkNodeProps(newProps); props = new ZkNodeProps(newProps);

View File

@ -851,7 +851,7 @@ public class CoreAdminHandler extends RequestHandlerBase {
if (core != null) { if (core != null) {
SyncStrategy syncStrategy = new SyncStrategy(); SyncStrategy syncStrategy = new SyncStrategy();
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl()); props.put(ZkStateReader.BASE_URL_PROP, zkController.getBaseUrl());
props.put(ZkStateReader.CORE_NAME_PROP, cname); props.put(ZkStateReader.CORE_NAME_PROP, cname);
props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName()); props.put(ZkStateReader.NODE_NAME_PROP, zkController.getNodeName());
@ -934,9 +934,9 @@ public class CoreAdminHandler extends RequestHandlerBase {
Slice slice = clusterState.getSlice(collection, Slice slice = clusterState.getSlice(collection,
cloudDescriptor.getShardId()); cloudDescriptor.getShardId());
if (slice != null) { if (slice != null) {
ZkNodeProps nodeProps = slice.getShards().get(coreNodeName); ZkNodeProps nodeProps = slice.getReplicasMap().get(coreNodeName);
if (nodeProps != null) { if (nodeProps != null) {
state = nodeProps.get(ZkStateReader.STATE_PROP); state = nodeProps.getStr(ZkStateReader.STATE_PROP);
live = clusterState.liveNodesContain(nodeName); live = clusterState.liveNodesContain(nodeName);
if (nodeProps != null && state.equals(waitForState)) { if (nodeProps != null && state.equals(waitForState)) {
if (checkLive == null) { if (checkLive == null) {

View File

@ -42,6 +42,7 @@ import org.apache.solr.cloud.ZkController;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -350,7 +351,7 @@ public class HttpShardHandler extends ShardHandler {
// throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such shard: " + sliceName); // throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "no such shard: " + sliceName);
} }
Map<String, ZkNodeProps> sliceShards = slice.getShards(); Map<String, Replica> sliceShards = slice.getReplicasMap();
// For now, recreate the | delimited list of equivalent servers // For now, recreate the | delimited list of equivalent servers
Set<String> liveNodes = clusterState.getLiveNodes(); Set<String> liveNodes = clusterState.getLiveNodes();

View File

@ -40,6 +40,7 @@ import javax.servlet.http.HttpServletResponse;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
@ -335,10 +336,10 @@ public class SolrDispatchFilter implements Filter
} }
// check everyone then // check everyone then
Map<String,ZkNodeProps> shards = entry.getValue().getShards(); Map<String,Replica> shards = entry.getValue().getReplicasMap();
Set<Entry<String,ZkNodeProps>> shardEntries = shards.entrySet(); Set<Entry<String,Replica>> shardEntries = shards.entrySet();
for (Entry<String,ZkNodeProps> shardEntry : shardEntries) { for (Entry<String,Replica> shardEntry : shardEntries) {
ZkNodeProps zkProps = shardEntry.getValue(); Replica zkProps = shardEntry.getValue();
core = checkProps(cores, path, zkProps); core = checkProps(cores, path, zkProps);
if (core != null) { if (core != null) {
break done; break done;
@ -352,8 +353,8 @@ public class SolrDispatchFilter implements Filter
ZkNodeProps zkProps) { ZkNodeProps zkProps) {
String corename; String corename;
SolrCore core = null; SolrCore core = null;
if (cores.getZkController().getNodeName().equals(zkProps.get(ZkStateReader.NODE_NAME_PROP))) { if (cores.getZkController().getNodeName().equals(zkProps.getStr(ZkStateReader.NODE_NAME_PROP))) {
corename = zkProps.get(ZkStateReader.CORE_NAME_PROP); corename = zkProps.getStr(ZkStateReader.CORE_NAME_PROP);
core = cores.getCore(corename); core = cores.getCore(corename);
} }
return core; return core;

View File

@ -37,6 +37,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.SolrInputField; import org.apache.solr.common.SolrInputField;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -1062,9 +1063,9 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
for (Map.Entry<String,Slice> sliceEntry : slices.entrySet()) { for (Map.Entry<String,Slice> sliceEntry : slices.entrySet()) {
Slice replicas = slices.get(sliceEntry.getKey()); Slice replicas = slices.get(sliceEntry.getKey());
Map<String,ZkNodeProps> shardMap = replicas.getShards(); Map<String,Replica> shardMap = replicas.getReplicasMap();
for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) { for (Entry<String,Replica> entry : shardMap.entrySet()) {
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue()); ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) { if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !entry.getKey().equals(shardZkNodeName)) {
urls.add(new StdNode(nodeProps)); urls.add(new StdNode(nodeProps));

View File

@ -34,9 +34,6 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.servlet.SolrDispatchFilter;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
/** /**
* This test simply does a bunch of basic things in solrcloud mode and asserts things * This test simply does a bunch of basic things in solrcloud mode and asserts things
* work as expected. * work as expected.
@ -106,7 +103,7 @@ public class BasicDistributedZk2Test extends AbstractFullDistribZkTestBase {
ZkNodeProps leaderProps = zkStateReader.getLeaderProps( ZkNodeProps leaderProps = zkStateReader.getLeaderProps(
DEFAULT_COLLECTION, SHARD2); DEFAULT_COLLECTION, SHARD2);
String nodeName = leaderProps.get(ZkStateReader.NODE_NAME_PROP); String nodeName = leaderProps.getStr(ZkStateReader.NODE_NAME_PROP);
chaosMonkey.stopShardExcept(SHARD2, nodeName); chaosMonkey.stopShardExcept(SHARD2, nodeName);
SolrServer client = getClient(nodeName); SolrServer client = getClient(nodeName);

View File

@ -58,6 +58,7 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode; import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -70,9 +71,6 @@ import org.apache.solr.common.util.NamedList;
import org.apache.solr.update.SolrCmdDistributor.Request; import org.apache.solr.update.SolrCmdDistributor.Request;
import org.apache.solr.util.DefaultSolrThreadFactory; import org.apache.solr.util.DefaultSolrThreadFactory;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakAction.Action;
/** /**
* This test simply does a bunch of basic things in solrcloud mode and asserts things * This test simply does a bunch of basic things in solrcloud mode and asserts things
* work as expected. * work as expected.
@ -462,11 +460,11 @@ public class BasicDistributedZkTest extends AbstractDistribZkTestBase {
Iterator<Entry<String,Slice>> it = slices.entrySet().iterator(); Iterator<Entry<String,Slice>> it = slices.entrySet().iterator();
while (it.hasNext()) { while (it.hasNext()) {
Entry<String,Slice> sliceEntry = it.next(); Entry<String,Slice> sliceEntry = it.next();
Map<String,ZkNodeProps> sliceShards = sliceEntry.getValue().getShards(); Map<String,Replica> sliceShards = sliceEntry.getValue().getReplicasMap();
Iterator<Entry<String,ZkNodeProps>> shardIt = sliceShards.entrySet() Iterator<Entry<String,Replica>> shardIt = sliceShards.entrySet()
.iterator(); .iterator();
while (shardIt.hasNext()) { while (shardIt.hasNext()) {
Entry<String,ZkNodeProps> shardEntry = shardIt.next(); Entry<String,Replica> shardEntry = shardIt.next();
ZkCoreNodeProps coreProps = new ZkCoreNodeProps(shardEntry.getValue()); ZkCoreNodeProps coreProps = new ZkCoreNodeProps(shardEntry.getValue());
CoreAdminResponse mcr = CoreAdminRequest.getStatus( CoreAdminResponse mcr = CoreAdminRequest.getStatus(
coreProps.getCoreName(), coreProps.getCoreName(),
@ -491,11 +489,11 @@ public class BasicDistributedZkTest extends AbstractDistribZkTestBase {
for (Map.Entry<String,Slice> entry : slices.entrySet()) { for (Map.Entry<String,Slice> entry : slices.entrySet()) {
Slice slice = entry.getValue(); Slice slice = entry.getValue();
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
Set<Map.Entry<String,ZkNodeProps>> shardEntries = shards.entrySet(); Set<Map.Entry<String,Replica>> shardEntries = shards.entrySet();
for (Map.Entry<String,ZkNodeProps> shardEntry : shardEntries) { for (Map.Entry<String,Replica> shardEntry : shardEntries) {
final ZkNodeProps node = shardEntry.getValue(); final ZkNodeProps node = shardEntry.getValue();
if (clusterState.liveNodesContain(node.get(ZkStateReader.NODE_NAME_PROP))) { if (clusterState.liveNodesContain(node.getStr(ZkStateReader.NODE_NAME_PROP))) {
return new ZkCoreNodeProps(node).getCoreUrl(); return new ZkCoreNodeProps(node).getCoreUrl();
} }
} }
@ -551,13 +549,13 @@ public class BasicDistributedZkTest extends AbstractDistribZkTestBase {
Iterator<Entry<String,Slice>> it = slices.entrySet().iterator(); Iterator<Entry<String,Slice>> it = slices.entrySet().iterator();
while (it.hasNext()) { while (it.hasNext()) {
Entry<String,Slice> sliceEntry = it.next(); Entry<String,Slice> sliceEntry = it.next();
Map<String,ZkNodeProps> sliceShards = sliceEntry.getValue() Map<String,Replica> sliceShards = sliceEntry.getValue()
.getShards(); .getReplicasMap();
Iterator<Entry<String,ZkNodeProps>> shardIt = sliceShards Iterator<Entry<String,Replica>> shardIt = sliceShards
.entrySet().iterator(); .entrySet().iterator();
while (shardIt.hasNext()) { while (shardIt.hasNext()) {
Entry<String,ZkNodeProps> shardEntry = shardIt.next(); Entry<String,Replica> shardEntry = shardIt.next();
if (!shardEntry.getValue().get(ZkStateReader.STATE_PROP) if (!shardEntry.getValue().getStr(ZkStateReader.STATE_PROP)
.equals(ZkStateReader.ACTIVE)) { .equals(ZkStateReader.ACTIVE)) {
found = false; found = false;
break; break;
@ -780,7 +778,7 @@ public class BasicDistributedZkTest extends AbstractDistribZkTestBase {
zkStateReader.updateClusterState(true); zkStateReader.updateClusterState(true);
Map<String,Slice> slices = zkStateReader.getClusterState().getSlices(oneInstanceCollection2); Map<String,Slice> slices = zkStateReader.getClusterState().getSlices(oneInstanceCollection2);
assertNotNull(slices); assertNotNull(slices);
String roles = slices.get("slice1").getShards().values().iterator().next().get(ZkStateReader.ROLES_PROP); String roles = slices.get("slice1").getReplicasMap().values().iterator().next().getStr(ZkStateReader.ROLES_PROP);
assertEquals("none", roles); assertEquals("none", roles);
} }

View File

@ -25,7 +25,7 @@ import java.util.Set;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.junit.Test; import org.junit.Test;
@ -38,13 +38,13 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
liveNodes.add("node2"); liveNodes.add("node2");
Map<String,Slice> slices = new HashMap<String,Slice>(); Map<String,Slice> slices = new HashMap<String,Slice>();
Map<String,ZkNodeProps> sliceToProps = new HashMap<String,ZkNodeProps>(); Map<String,Replica> sliceToProps = new HashMap<String,Replica>();
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
props.put("prop1", "value"); props.put("prop1", "value");
props.put("prop2", "value2"); props.put("prop2", "value2");
ZkNodeProps zkNodeProps = new ZkNodeProps(props); Replica replica = new Replica("node1", props);
sliceToProps.put("node1", zkNodeProps); sliceToProps.put("node1", replica);
Slice slice = new Slice("shard1", sliceToProps); Slice slice = new Slice("shard1", sliceToProps);
slices.put("shard1", slice); slices.put("shard1", slice);
Slice slice2 = new Slice("shard2", sliceToProps); Slice slice2 = new Slice("shard2", sliceToProps);
@ -60,8 +60,8 @@ public class ClusterStateTest extends SolrTestCaseJ4 {
assertEquals("Provided liveNodes not used properly", 2, loadedClusterState assertEquals("Provided liveNodes not used properly", 2, loadedClusterState
.getLiveNodes().size()); .getLiveNodes().size());
assertEquals("No collections found", 2, loadedClusterState.getCollections().size()); assertEquals("No collections found", 2, loadedClusterState.getCollections().size());
assertEquals("Poperties not copied properly", zkNodeProps.get("prop1"), loadedClusterState.getSlice("collection1", "shard1").getShards().get("node1").get("prop1")); assertEquals("Poperties not copied properly", replica.getStr("prop1"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop1"));
assertEquals("Poperties not copied properly", zkNodeProps.get("prop2"), loadedClusterState.getSlice("collection1", "shard1").getShards().get("node1").get("prop2")); assertEquals("Poperties not copied properly", replica.getStr("prop2"), loadedClusterState.getSlice("collection1", "shard1").getReplicasMap().get("node1").getStr("prop2"));
loadedClusterState = ClusterState.load(null, new byte[0], liveNodes); loadedClusterState = ClusterState.load(null, new byte[0], liveNodes);

View File

@ -25,6 +25,7 @@ import java.util.Set;
import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -139,7 +140,7 @@ public class ClusterStateUpdateTest extends SolrTestCaseJ4 {
System.setProperty("solrcloud.update.delay", "1"); System.setProperty("solrcloud.update.delay", "1");
Map<String,String> props2 = new HashMap<String,String>(); Map<String,Object> props2 = new HashMap<String,Object>();
props2.put("configName", "conf1"); props2.put("configName", "conf1");
ZkNodeProps zkProps2 = new ZkNodeProps(props2); ZkNodeProps zkProps2 = new ZkNodeProps(props2);
@ -173,7 +174,7 @@ public class ClusterStateUpdateTest extends SolrTestCaseJ4 {
slices = clusterState2.getSlices("testcore"); slices = clusterState2.getSlices("testcore");
if (slices != null && slices.containsKey("shard1") if (slices != null && slices.containsKey("shard1")
&& slices.get("shard1").getShards().size() > 0) { && slices.get("shard1").getReplicasMap().size() > 0) {
break; break;
} }
Thread.sleep(500); Thread.sleep(500);
@ -185,17 +186,17 @@ public class ClusterStateUpdateTest extends SolrTestCaseJ4 {
Slice slice = slices.get("shard1"); Slice slice = slices.get("shard1");
assertEquals("shard1", slice.getName()); assertEquals("shard1", slice.getName());
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
assertEquals(1, shards.size()); assertEquals(1, shards.size());
ZkNodeProps zkProps = shards.get(host + ":1661_solr_testcore"); Replica zkProps = shards.get(host + ":1661_solr_testcore");
assertNotNull(zkProps); assertNotNull(zkProps);
assertEquals(host + ":1661_solr", zkProps.get(ZkStateReader.NODE_NAME_PROP)); assertEquals(host + ":1661_solr", zkProps.getStr(ZkStateReader.NODE_NAME_PROP));
assertEquals("http://" + host + ":1661/solr", zkProps.get(ZkStateReader.BASE_URL_PROP)); assertEquals("http://" + host + ":1661/solr", zkProps.getStr(ZkStateReader.BASE_URL_PROP));
Set<String> liveNodes = clusterState2.getLiveNodes(); Set<String> liveNodes = clusterState2.getLiveNodes();
assertNotNull(liveNodes); assertNotNull(liveNodes);

View File

@ -249,7 +249,7 @@ public class LeaderElectionIntegrationTest extends SolrTestCaseJ4 {
private String getLeader() throws InterruptedException { private String getLeader() throws InterruptedException {
ZkNodeProps props = reader.getLeaderProps("collection1", "shard1", 30000); ZkNodeProps props = reader.getLeaderProps("collection1", "shard1", 30000);
String leader = props.get(ZkStateReader.NODE_NAME_PROP); String leader = props.getStr(ZkStateReader.NODE_NAME_PROP);
return leader; return leader;
} }

View File

@ -152,7 +152,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
collection); collection);
if (slices != null) { if (slices != null) {
for (Slice slice : slices.values()) { for (Slice slice : slices.values()) {
if (slice.getShards().containsKey(nodeName + "_" + coreName)) { if (slice.getReplicasMap().containsKey(nodeName + "_" + coreName)) {
return slice.getName(); return slice.getName();
} }
} }
@ -204,9 +204,9 @@ public class OverseerTest extends SolrTestCaseJ4 {
assertNotNull("shard got no id?", zkController.publishState("core" + (i+1), ZkStateReader.ACTIVE, 3)); assertNotNull("shard got no id?", zkController.publishState("core" + (i+1), ZkStateReader.ACTIVE, 3));
} }
assertEquals(2, reader.getClusterState().getSlice("collection1", "shard1").getShards().size()); assertEquals(2, reader.getClusterState().getSlice("collection1", "shard1").getReplicasMap().size());
assertEquals(2, reader.getClusterState().getSlice("collection1", "shard2").getShards().size()); assertEquals(2, reader.getClusterState().getSlice("collection1", "shard2").getReplicasMap().size());
assertEquals(2, reader.getClusterState().getSlice("collection1", "shard3").getShards().size()); assertEquals(2, reader.getClusterState().getSlice("collection1", "shard3").getReplicasMap().size());
//make sure leaders are in cloud state //make sure leaders are in cloud state
assertNotNull(reader.getLeaderUrl("collection1", "shard1", 15000)); assertNotNull(reader.getLeaderUrl("collection1", "shard1", 15000));
@ -303,7 +303,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
ClusterState state = reader.getClusterState(); ClusterState state = reader.getClusterState();
Map<String,Slice> slices = state.getSlices("collection1"); Map<String,Slice> slices = state.getSlices("collection1");
for (String name : slices.keySet()) { for (String name : slices.keySet()) {
cloudStateSliceCount += slices.get(name).getShards().size(); cloudStateSliceCount += slices.get(name).getReplicasMap().size();
} }
if (coreCount == cloudStateSliceCount) break; if (coreCount == cloudStateSliceCount) break;
Thread.sleep(200); Thread.sleep(200);
@ -435,8 +435,8 @@ public class OverseerTest extends SolrTestCaseJ4 {
waitForCollections(reader, "collection1"); waitForCollections(reader, "collection1");
assertEquals(reader.getClusterState().toString(), ZkStateReader.RECOVERING, assertEquals(reader.getClusterState().toString(), ZkStateReader.RECOVERING,
reader.getClusterState().getSlice("collection1", "shard1").getShards() reader.getClusterState().getSlice("collection1", "shard1").getReplicasMap()
.get("node1_core1").get(ZkStateReader.STATE_PROP)); .get("node1_core1").getStr(ZkStateReader.STATE_PROP));
//publish node state (active) //publish node state (active)
m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state", m = new ZkNodeProps(Overseer.QUEUE_OPERATION, "state",
@ -467,7 +467,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
while(maxIterations-->0) { while(maxIterations-->0) {
Slice slice = reader.getClusterState().getSlice("collection1", "shard1"); Slice slice = reader.getClusterState().getSlice("collection1", "shard1");
if(slice!=null) { if(slice!=null) {
coreState = slice.getShards().get("node1_core1").get(ZkStateReader.STATE_PROP); coreState = slice.getReplicasMap().get("node1_core1").getStr(ZkStateReader.STATE_PROP);
if(coreState.equals(expectedState)) { if(coreState.equals(expectedState)) {
return; return;
} }
@ -483,14 +483,14 @@ public class OverseerTest extends SolrTestCaseJ4 {
reader.updateClusterState(true); // poll state reader.updateClusterState(true); // poll state
ZkNodeProps props = reader.getClusterState().getLeader(collection, shard); ZkNodeProps props = reader.getClusterState().getLeader(collection, shard);
if(props!=null) { if(props!=null) {
if(expectedCore.equals(props.get(ZkStateReader.CORE_NAME_PROP))) { if(expectedCore.equals(props.getStr(ZkStateReader.CORE_NAME_PROP))) {
return; return;
} }
} }
Thread.sleep(100); Thread.sleep(100);
} }
assertEquals("Unexpected shard leader coll:" + collection + " shard:" + shard, expectedCore, (reader.getClusterState().getLeader(collection, shard)!=null)?reader.getClusterState().getLeader(collection, shard).get(ZkStateReader.CORE_NAME_PROP):null); assertEquals("Unexpected shard leader coll:" + collection + " shard:" + shard, expectedCore, (reader.getClusterState().getLeader(collection, shard)!=null)?reader.getClusterState().getLeader(collection, shard).getStr(ZkStateReader.CORE_NAME_PROP):null);
} }
@Test @Test
@ -547,7 +547,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
assertEquals("Live nodes count does not match", 1, reader.getClusterState() assertEquals("Live nodes count does not match", 1, reader.getClusterState()
.getLiveNodes().size()); .getLiveNodes().size());
assertEquals("Shard count does not match", 1, reader.getClusterState() assertEquals("Shard count does not match", 1, reader.getClusterState()
.getSlice("collection1", "shard1").getShards().size()); .getSlice("collection1", "shard1").getReplicasMap().size());
version = getClusterStateVersion(controllerClient); version = getClusterStateVersion(controllerClient);
mockController.publishState("core1", null,1); mockController.publishState("core1", null,1);
while(version == getClusterStateVersion(controllerClient)); while(version == getClusterStateVersion(controllerClient));
@ -714,7 +714,7 @@ public class OverseerTest extends SolrTestCaseJ4 {
int numFound = 0; int numFound = 0;
for (Map<String,Slice> collection : state.getCollectionStates().values()) { for (Map<String,Slice> collection : state.getCollectionStates().values()) {
for (Slice slice : collection.values()) { for (Slice slice : collection.values()) {
if (slice.getShards().get("node1_core1") != null) { if (slice.getReplicasMap().get("node1_core1") != null) {
numFound++; numFound++;
} }
} }
@ -834,11 +834,11 @@ public class OverseerTest extends SolrTestCaseJ4 {
for(int i=0;i<100;i++) { for(int i=0;i<100;i++) {
Slice s = reader.getClusterState().getSlice("collection1", "s1"); Slice s = reader.getClusterState().getSlice("collection1", "s1");
if(s!=null && s.getShards().size()==3) break; if(s!=null && s.getReplicasMap().size()==3) break;
Thread.sleep(100); Thread.sleep(100);
} }
assertNotNull(reader.getClusterState().getSlice("collection1", "s1")); assertNotNull(reader.getClusterState().getSlice("collection1", "s1"));
assertEquals(3, reader.getClusterState().getSlice("collection1", "s1").getShards().size()); assertEquals(3, reader.getClusterState().getSlice("collection1", "s1").getReplicasMap().size());
} finally { } finally {
close(overseerClient); close(overseerClient);
close(zkClient); close(zkClient);

View File

@ -30,7 +30,7 @@ public class TestHashPartitioner extends SolrTestCaseJ4 {
List<Range> ranges; List<Range> ranges;
// make sure the partitioner uses the "natural" boundaries and doesn't suffer from an off-by-one // make sure the partitioner uses the "natural" boundaries and doesn't suffer from an off-by-one
ranges = hp.partitionRange(2, Integer.MIN_VALUE, Integer.MAX_VALUE); ranges = hp.partitionRange(2, hp.fullRange());
assertEquals(Integer.MIN_VALUE, ranges.get(0).min); assertEquals(Integer.MIN_VALUE, ranges.get(0).min);
assertEquals(0x80000000, ranges.get(0).min); assertEquals(0x80000000, ranges.get(0).min);
assertEquals(0xffffffff, ranges.get(0).max); assertEquals(0xffffffff, ranges.get(0).max);
@ -44,7 +44,7 @@ public class TestHashPartitioner extends SolrTestCaseJ4 {
assertEquals(0x7fffffff, ranges.get(1).max); assertEquals(0x7fffffff, ranges.get(1).max);
for (int i = 1; i <= 30000; i += 13) { for (int i = 1; i <= 30000; i += 13) {
ranges = hp.partitionRange(i, Integer.MIN_VALUE, Integer.MAX_VALUE); ranges = hp.partitionRange(i, hp.fullRange());
assertEquals(i, ranges.size()); assertEquals(i, ranges.size());
assertTrue("First range does not start before " + Integer.MIN_VALUE assertTrue("First range does not start before " + Integer.MIN_VALUE
+ " it is:" + ranges.get(0).min, + " it is:" + ranges.get(0).min,
@ -52,6 +52,14 @@ public class TestHashPartitioner extends SolrTestCaseJ4 {
assertTrue("Last range does not end after " + Integer.MAX_VALUE assertTrue("Last range does not end after " + Integer.MAX_VALUE
+ " it is:" + ranges.get(ranges.size() - 1).max, + " it is:" + ranges.get(ranges.size() - 1).max,
ranges.get(ranges.size() - 1).max >= Integer.MAX_VALUE); ranges.get(ranges.size() - 1).max >= Integer.MAX_VALUE);
for (Range range : ranges) {
String s = range.toString();
Range newRange = hp.fromString(s);
assertEquals(range, newRange);
}
} }
} }

View File

@ -20,7 +20,6 @@ package org.apache.solr.cloud;
import java.io.File; import java.io.File;
import java.util.List; import java.util.List;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4; import org.apache.solr.SolrTestCaseJ4;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -136,7 +135,7 @@ public class ZkCLITest extends SolrTestCaseJ4 {
ZkNodeProps collectionProps = ZkNodeProps.load(zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/collection1", null, null, true)); ZkNodeProps collectionProps = ZkNodeProps.load(zkClient.getData(ZkStateReader.COLLECTIONS_ZKNODE + "/collection1", null, null, true));
assertTrue(collectionProps.containsKey("configName")); assertTrue(collectionProps.containsKey("configName"));
assertEquals(confsetname, collectionProps.get("configName")); assertEquals(confsetname, collectionProps.getStr("configName"));
// test down config // test down config
File confDir = new File(TEMP_DIR, File confDir = new File(TEMP_DIR,

View File

@ -67,7 +67,7 @@ public class ZkControllerTest extends SolrTestCaseJ4 {
zkClient.makePath(ZkController.CONFIGS_ZKNODE + "/" + actualConfigName, true); zkClient.makePath(ZkController.CONFIGS_ZKNODE + "/" + actualConfigName, true);
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
props.put("configName", actualConfigName); props.put("configName", actualConfigName);
ZkNodeProps zkProps = new ZkNodeProps(props); ZkNodeProps zkProps = new ZkNodeProps(props);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/"

View File

@ -30,7 +30,7 @@ public class ZkNodePropsTest extends SolrTestCaseJ4 {
@Test @Test
public void testBasic() throws IOException { public void testBasic() throws IOException {
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
props.put("prop1", "value1"); props.put("prop1", "value1");
props.put("prop2", "value2"); props.put("prop2", "value2");
props.put("prop3", "value3"); props.put("prop3", "value3");
@ -42,11 +42,11 @@ public class ZkNodePropsTest extends SolrTestCaseJ4 {
byte[] bytes = ZkStateReader.toJSON(zkProps); byte[] bytes = ZkStateReader.toJSON(zkProps);
ZkNodeProps props2 = ZkNodeProps.load(bytes); ZkNodeProps props2 = ZkNodeProps.load(bytes);
assertEquals("value1", props2.get("prop1")); assertEquals("value1", props2.getStr("prop1"));
assertEquals("value2", props2.get("prop2")); assertEquals("value2", props2.getStr("prop2"));
assertEquals("value3", props2.get("prop3")); assertEquals("value3", props2.getStr("prop3"));
assertEquals("value4", props2.get("prop4")); assertEquals("value4", props2.getStr("prop4"));
assertEquals("value5", props2.get("prop5")); assertEquals("value5", props2.getStr("prop5"));
assertEquals("value6", props2.get("prop6")); assertEquals("value6", props2.getStr("prop6"));
} }
} }

View File

@ -208,7 +208,7 @@ public class CloudSolrServer extends SolrServer {
Map<String,ZkNodeProps> nodes = new HashMap<String,ZkNodeProps>(); Map<String,ZkNodeProps> nodes = new HashMap<String,ZkNodeProps>();
List<String> urlList = new ArrayList<String>(); List<String> urlList = new ArrayList<String>();
for (Slice slice : slices.values()) { for (Slice slice : slices.values()) {
for (ZkNodeProps nodeProps : slice.getShards().values()) { for (ZkNodeProps nodeProps : slice.getReplicasMap().values()) {
ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(nodeProps); ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(nodeProps);
String node = coreNodeProps.getNodeName(); String node = coreNodeProps.getNodeName();
if (!liveNodes.contains(coreNodeProps.getNodeName()) if (!liveNodes.contains(coreNodeProps.getNodeName())

View File

@ -92,9 +92,9 @@ public class ClusterState implements JSONWriter.Writable {
Set<Entry<String,Slice>> slices = state.entrySet(); Set<Entry<String,Slice>> slices = state.entrySet();
for (Entry<String,Slice> sliceEntry : slices) { for (Entry<String,Slice> sliceEntry : slices) {
Slice slice = sliceEntry.getValue(); Slice slice = sliceEntry.getValue();
Map<String,ZkNodeProps> shards = slice.getShards(); Map<String,Replica> shards = slice.getReplicasMap();
Set<Entry<String,ZkNodeProps>> shardsEntries = shards.entrySet(); Set<Entry<String,Replica>> shardsEntries = shards.entrySet();
for (Entry<String,ZkNodeProps> shardEntry : shardsEntries) { for (Entry<String,Replica> shardEntry : shardsEntries) {
ZkNodeProps props = shardEntry.getValue(); ZkNodeProps props = shardEntry.getValue();
if (props.containsKey(ZkStateReader.LEADER_PROP)) { if (props.containsKey(ZkStateReader.LEADER_PROP)) {
Map<String,ZkNodeProps> leadersForCollection = leaders.get(collection.getKey()); Map<String,ZkNodeProps> leadersForCollection = leaders.get(collection.getKey());
@ -122,11 +122,11 @@ public class ClusterState implements JSONWriter.Writable {
/** /**
* Get shard properties or null if shard is not found. * Get shard properties or null if shard is not found.
*/ */
public ZkNodeProps getShardProps(final String collection, final String coreNodeName) { public Replica getShardProps(final String collection, final String coreNodeName) {
Map<String, Slice> slices = getSlices(collection); Map<String, Slice> slices = getSlices(collection);
for(Slice slice: slices.values()) { for(Slice slice: slices.values()) {
if(slice.getShards().get(coreNodeName)!=null) { if(slice.getReplicasMap().get(coreNodeName)!=null) {
return slice.getShards().get(coreNodeName); return slice.getReplicasMap().get(coreNodeName);
} }
} }
return null; return null;
@ -185,7 +185,7 @@ public class ClusterState implements JSONWriter.Writable {
public String getShardId(String coreNodeName) { public String getShardId(String coreNodeName) {
for (Entry<String, Map<String, Slice>> states: collectionStates.entrySet()){ for (Entry<String, Map<String, Slice>> states: collectionStates.entrySet()){
for(Entry<String, Slice> slices: states.getValue().entrySet()) { for(Entry<String, Slice> slices: states.getValue().entrySet()) {
for(Entry<String, ZkNodeProps> shards: slices.getValue().getShards().entrySet()){ for(Entry<String, Replica> shards: slices.getValue().getReplicasMap().entrySet()){
if(coreNodeName.equals(shards.getKey())) { if(coreNodeName.equals(shards.getKey())) {
return slices.getKey(); return slices.getKey();
} }
@ -294,10 +294,10 @@ public class ClusterState implements JSONWriter.Writable {
Map<String, Object> collection = (Map<String, Object>)stateMap.get(collectionName); Map<String, Object> collection = (Map<String, Object>)stateMap.get(collectionName);
Map<String, Slice> slices = new LinkedHashMap<String,Slice>(); Map<String, Slice> slices = new LinkedHashMap<String,Slice>();
for(String sliceName: collection.keySet()) { for(String sliceName: collection.keySet()) {
Map<String, Map<String, String>> sliceMap = (Map<String, Map<String, String>>)collection.get(sliceName); Map<String, Map<String, Object>> sliceMap = (Map<String, Map<String, Object>>)collection.get(sliceName);
Map<String, ZkNodeProps> shards = new LinkedHashMap<String,ZkNodeProps>(); Map<String, Replica> shards = new LinkedHashMap<String,Replica>();
for(String shardName: sliceMap.keySet()) { for(String shardName: sliceMap.keySet()) {
shards.put(shardName, new ZkNodeProps(sliceMap.get(shardName))); shards.put(shardName, new Replica(shardName, sliceMap.get(shardName)));
} }
Slice slice = new Slice(sliceName, shards); Slice slice = new Slice(sliceName, shards);
slices.put(sliceName, slice); slices.put(sliceName, slice);

View File

@ -17,6 +17,8 @@ package org.apache.solr.common.cloud;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.noggit.JSONWriter;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
@ -29,11 +31,12 @@ public class HashPartitioner {
// Hash ranges can't currently "wrap" - i.e. max must be greater or equal to min. // Hash ranges can't currently "wrap" - i.e. max must be greater or equal to min.
// TODO: ranges may not be all contiguous in the future (either that or we will // TODO: ranges may not be all contiguous in the future (either that or we will
// need an extra class to model a collection of ranges) // need an extra class to model a collection of ranges)
public static class Range { public static class Range implements JSONWriter.Writable {
public int min; // inclusive public int min; // inclusive
public int max; // inclusive public int max; // inclusive
public Range(int min, int max) { public Range(int min, int max) {
assert min <= max;
this.min = min; this.min = min;
this.max = max; this.max = max;
} }
@ -46,12 +49,39 @@ public class HashPartitioner {
return Integer.toHexString(min) + '-' + Integer.toHexString(max); return Integer.toHexString(min) + '-' + Integer.toHexString(max);
} }
public static Range fromString(String range) {
return null; // TODO @Override
public int hashCode() {
// difficult numbers to hash... only the highest bits will tend to differ.
// ranges will only overlap during a split, so we can just hash the lower range.
return (min>>28) + (min>>25) + (min>>21) + min;
}
@Override
public boolean equals(Object obj) {
if (obj.getClass() != getClass()) return false;
Range other = (Range)obj;
return this.min == other.min && this.max == other.max;
}
@Override
public void write(JSONWriter writer) {
writer.write(toString());
} }
} }
public Range fromString(String range) {
int middle = range.indexOf('-');
String minS = range.substring(0, middle);
String maxS = range.substring(middle+1);
long min = Long.parseLong(minS, 16); // use long to prevent the parsing routines from potentially worrying about overflow
long max = Long.parseLong(maxS, 16);
return new Range((int)min, (int)max);
}
public Range fullRange() {
return new Range(Integer.MIN_VALUE, Integer.MAX_VALUE);
}
public List<Range> partitionRange(int partitions, Range range) { public List<Range> partitionRange(int partitions, Range range) {
return partitionRange(partitions, range.min, range.max); return partitionRange(partitions, range.min, range.max);

View File

@ -0,0 +1,38 @@
package org.apache.solr.common.cloud;
/*
* 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.
*/
import java.util.Map;
public class Replica extends ZkNodeProps {
private final String name;
public Replica(String name, Map<String,Object> propMap) { // TODO: back compat for handling Map<String,String>
super(propMap);
this.name = name;
String nodeName = (String)propMap.get(ZkStateReader.NODE_NAME_PROP);
assert nodeName == null || name.startsWith(nodeName);
}
public String getName() {
return name;
}
}

View File

@ -19,44 +19,70 @@ package org.apache.solr.common.cloud;
import org.apache.noggit.JSONWriter; import org.apache.noggit.JSONWriter;
import java.util.Collections; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map; import java.util.Map;
/** /**
* A Slice contains immutable information about all shards that share the same * A Slice contains immutable information about a logical shard (all replicas that share the same shard id).
* shard id (shard leader and replicas).
*/ */
public class Slice implements JSONWriter.Writable { public class Slice extends ZkNodeProps {
private final Map<String,ZkNodeProps> shards; public static String REPLICAS = "replicas";
public static String RANGE = "range";
public static String LEADER = "leader";
private final String name; private final String name;
private final HashPartitioner.Range range;
// private final Integer replicationFactor;
private final Map<String,Replica> replicas;
private final Replica leader;
public Slice(String name, Map<String,ZkNodeProps> shards) { public Slice(String name, Map<String,Replica> replicas) {
this.shards = shards; this(name, replicas, null);
}
public Slice(String name, Map<String,Replica> replicas, Map<String,Object> props) {
super( props==null ? new LinkedHashMap<String,Object>(2) : new LinkedHashMap<String,Object>(props));
this.name = name; this.name = name;
} this.replicas = replicas != null ? replicas : makeReplicas((Map<String,Object>)propMap.get(REPLICAS));
propMap.put(REPLICAS, replicas);
/** String rangeStr = (String)propMap.get(RANGE);
* Get properties for all shards in this slice. HashPartitioner.Range tmpRange = null;
* if (rangeStr != null) {
* @return map containing coreNodeName as the key, see HashPartitioner hp = new HashPartitioner();
* {@link ZkStateReader#getCoreNodeName(String, String)}, ZKNodeProps tmpRange = hp.fromString(rangeStr);
* as the value.
*/
public Map<String,ZkNodeProps> getShards() {
return Collections.unmodifiableMap(shards);
}
/**
* Get a copy of the shards data this object holds.
*/
public Map<String,ZkNodeProps> getShardsCopy() {
Map<String,ZkNodeProps> shards = new HashMap<String,ZkNodeProps>();
for (Map.Entry<String,ZkNodeProps> entry : this.shards.entrySet()) {
ZkNodeProps zkProps = new ZkNodeProps(entry.getValue());
shards.put(entry.getKey(), zkProps);
} }
return shards;
range = tmpRange;
// replicationFactor = null; // future
leader = findLeader();
}
private Map<String,Replica> makeReplicas(Map<String,Object> genericReplicas) {
if (genericReplicas == null) return new HashMap<String,Replica>(1);
Map<String,Replica> result = new LinkedHashMap<String, Replica>(genericReplicas.size());
for (Map.Entry<String,Object> entry : genericReplicas.entrySet()) {
String name = entry.getKey();
Object val = entry.getValue();
Replica r;
if (val instanceof Replica) {
r = (Replica)val;
} else {
r = new Replica(name, (Map<String,Object>)val);
}
result.put(name, r);
}
return result;
}
private Replica findLeader() {
for (Replica replica : replicas.values()) {
if (replica.getStr(LEADER) != null) return replica;
}
return null;
} }
/** /**
@ -66,13 +92,57 @@ public class Slice implements JSONWriter.Writable {
return name; return name;
} }
/**
* Gets the list of replicas for this slice.
*/
public Collection<Replica> getReplicas() {
return replicas.values();
}
/**
* Get the map of coreNodeName to replicas for this slice.
*
* @return map containing coreNodeName as the key, see
* {@link ZkStateReader#getCoreNodeName(String, String)}, Replica
* as the value.
*/
public Map<String, Replica> getReplicasMap() {
return replicas;
}
public Map<String,Replica> getReplicasCopy() {
return new LinkedHashMap<String,Replica>(replicas);
}
public Replica getLeader() {
return leader;
}
/***
// returns a copy of this slice containing the new replica
public Slice addReplica(Replica replica) {
Map<String, Object> newProps = new LinkedHashMap<String,Object>(props);
Map<String, Replica> replicas = getReplicasMap();
Map<String, Replica> newReplicas = replicas == null ? new HashMap<String, Replica>(1) : new LinkedHashMap<String, Replica>(replicas);
// newReplicas.put(replica.getName(), replica);
newProps.put(REPLICAS, replicas);
return new Slice(name, newProps); // TODO: new constructor that takes replicas as-is w/o rebuilding
}
public static Slice newSlice(String name) {
Map<String, Object> props = new HashMap<String,Object>(1);
props.put("replicas", new HashMap<String,Object>(1));
return new Slice(name, props);
}
***/
@Override @Override
public String toString() { public String toString() {
return "Slice [shards=" + shards + ", name=" + name + "]"; return "Slice [replicas=" + replicas + ", name=" + name + "]";
} }
@Override @Override
public void write(JSONWriter jsonWriter) { public void write(JSONWriter jsonWriter) {
jsonWriter.write(shards); jsonWriter.write(replicas);
} }
} }

View File

@ -25,27 +25,27 @@ public class ZkCoreNodeProps {
} }
public String getCoreUrl() { public String getCoreUrl() {
return getCoreUrl(nodeProps.get(ZkStateReader.BASE_URL_PROP), nodeProps.get(ZkStateReader.CORE_NAME_PROP)); return getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), nodeProps.getStr(ZkStateReader.CORE_NAME_PROP));
} }
public String getNodeName() { public String getNodeName() {
return nodeProps.get(ZkStateReader.NODE_NAME_PROP); return nodeProps.getStr(ZkStateReader.NODE_NAME_PROP);
} }
public String getState() { public String getState() {
return nodeProps.get(ZkStateReader.STATE_PROP); return nodeProps.getStr(ZkStateReader.STATE_PROP);
} }
public String getBaseUrl() { public String getBaseUrl() {
return nodeProps.get(ZkStateReader.BASE_URL_PROP); return nodeProps.getStr(ZkStateReader.BASE_URL_PROP);
} }
public String getCoreName() { public String getCoreName() {
return nodeProps.get(ZkStateReader.CORE_NAME_PROP); return nodeProps.getStr(ZkStateReader.CORE_NAME_PROP);
} }
public static String getCoreUrl(ZkNodeProps nodeProps) { public static String getCoreUrl(ZkNodeProps nodeProps) {
return getCoreUrl(nodeProps.get(ZkStateReader.BASE_URL_PROP), nodeProps.get(ZkStateReader.CORE_NAME_PROP)); return getCoreUrl(nodeProps.getStr(ZkStateReader.BASE_URL_PROP), nodeProps.getStr(ZkStateReader.CORE_NAME_PROP));
} }
public static String getCoreUrl(String baseUrl, String coreName) { public static String getCoreUrl(String baseUrl, String coreName) {

View File

@ -17,51 +17,54 @@ package org.apache.solr.common.cloud;
* limitations under the License. * limitations under the License.
*/ */
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.Map.Entry;
import org.apache.noggit.JSONWriter; import org.apache.noggit.JSONWriter;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
/** /**
* ZkNodeProps contains immutable properties for a shard/solr core. * ZkNodeProps contains generic immutable properties.
*/ */
public class ZkNodeProps implements JSONWriter.Writable { public class ZkNodeProps implements JSONWriter.Writable {
private final Map<String,String> propMap; protected final Map<String,Object> propMap;
/** /**
* Construct ZKNodeProps from map. * Construct ZKNodeProps from map.
*/ */
public ZkNodeProps(Map<String,String> propMap) { public ZkNodeProps(Map<String,Object> propMap) { // TODO: back compat for handling Map<String,String>
this.propMap = new HashMap<String,String>(); this.propMap = propMap;
this.propMap.putAll(propMap);
} }
/**
* Construct ZKNodeProps from information of an existingZKNodeProps.
*/
public ZkNodeProps(ZkNodeProps zkNodeProps) {
this.propMap = new HashMap<String,String>();
this.propMap.putAll(zkNodeProps.propMap);
}
/** /**
* Constructor that populates the from array of Strings in form key1, value1, * Constructor that populates the from array of Strings in form key1, value1,
* key2, value2, ..., keyN, valueN * key2, value2, ..., keyN, valueN
*/ */
public ZkNodeProps(String... keyVals) { public ZkNodeProps(String... keyVals) {
if (keyVals.length % 2 != 0) { this( makeMap(keyVals) );
}
public static ZkNodeProps fromKeyVals(Object... keyVals) {
return new ZkNodeProps( makeMap(keyVals) );
}
public static Map<String,Object> makeMap(Object... keyVals) {
if ((keyVals.length & 0x01) != 0) {
throw new IllegalArgumentException("arguments should be key,value"); throw new IllegalArgumentException("arguments should be key,value");
} }
propMap = new HashMap<String,String>(); Map<String,Object> propMap = new HashMap<String,Object>(keyVals.length>>1);
for (int i = 0; i < keyVals.length; i+=2) { for (int i = 0; i < keyVals.length; i+=2) {
propMap.put(keyVals[i], keyVals[i+1]); propMap.put(keyVals[i].toString(), keyVals[i+1]);
} }
return propMap;
} }
/** /**
* Get property keys. * Get property keys.
*/ */
@ -72,15 +75,20 @@ public class ZkNodeProps implements JSONWriter.Writable {
/** /**
* Get all properties as map. * Get all properties as map.
*/ */
public Map<String,String> getProperties() { public Map<String, Object> getProperties() {
return Collections.unmodifiableMap(propMap); return Collections.unmodifiableMap(propMap);
} }
/** Returns a shallow writable copy of the properties */
public Map<String,Object> shallowCopy() {
return new LinkedHashMap<String, Object>(propMap);
}
/** /**
* Create ZkNodeProps from json string that is typically stored in zookeeper. * Create Replica from json string that is typically stored in zookeeper.
*/ */
public static ZkNodeProps load(byte[] bytes) { public static ZkNodeProps load(byte[] bytes) {
Map<String, String> props = (Map<String, String>) ZkStateReader.fromJSON(bytes); Map<String, Object> props = (Map<String, Object>) ZkStateReader.fromJSON(bytes);
return new ZkNodeProps(props); return new ZkNodeProps(props);
} }
@ -90,17 +98,22 @@ public class ZkNodeProps implements JSONWriter.Writable {
} }
/** /**
* Get property value. * Get a string property value.
*/ */
public String get(String key) { public String getStr(String key) {
Object o = propMap.get(key);
return o == null ? null : o.toString();
}
public Object get(String key,int foo) {
return propMap.get(key); return propMap.get(key);
} }
@Override @Override
public String toString() { public String toString() {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
Set<Entry<String,String>> entries = propMap.entrySet(); Set<Entry<String,Object>> entries = propMap.entrySet();
for(Entry<String,String> entry : entries) { for(Entry<String,Object> entry : entries) {
sb.append(entry.getKey() + "=" + entry.getValue() + "\n"); sb.append(entry.getKey() + "=" + entry.getValue() + "\n");
} }
return sb.toString(); return sb.toString();

View File

@ -458,10 +458,10 @@ public class ZkStateReader {
throw new ZooKeeperException(ErrorCode.BAD_REQUEST, "Could not find shardId in zk: " + shardId); throw new ZooKeeperException(ErrorCode.BAD_REQUEST, "Could not find shardId in zk: " + shardId);
} }
Map<String,ZkNodeProps> shardMap = replicas.getShards(); Map<String,Replica> shardMap = replicas.getReplicasMap();
List<ZkCoreNodeProps> nodes = new ArrayList<ZkCoreNodeProps>(shardMap.size()); List<ZkCoreNodeProps> nodes = new ArrayList<ZkCoreNodeProps>(shardMap.size());
String filterNodeName = thisNodeName + "_" + coreName; String filterNodeName = thisNodeName + "_" + coreName;
for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) { for (Entry<String,Replica> entry : shardMap.entrySet()) {
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue()); ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
String coreNodeName = nodeProps.getNodeName() + "_" + nodeProps.getCoreName(); String coreNodeName = nodeProps.getNodeName() + "_" + nodeProps.getCoreName();
if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !coreNodeName.equals(filterNodeName)) { if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !coreNodeName.equals(filterNodeName)) {

View File

@ -24,11 +24,10 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.solr.BaseDistributedSearchTestCase; import org.apache.solr.BaseDistributedSearchTestCase;
import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.cloud.ZkTestServer;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient; import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.servlet.SolrDispatchFilter;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
@ -130,18 +129,18 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
ClusterState clusterState = zkStateReader.getClusterState(); ClusterState clusterState = zkStateReader.getClusterState();
Map<String,Slice> slices = clusterState.getSlices(collection); Map<String,Slice> slices = clusterState.getSlices(collection);
for (Map.Entry<String,Slice> entry : slices.entrySet()) { for (Map.Entry<String,Slice> entry : slices.entrySet()) {
Map<String,ZkNodeProps> shards = entry.getValue().getShards(); Map<String,Replica> shards = entry.getValue().getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) { for (Map.Entry<String,Replica> shard : shards.entrySet()) {
if (verbose) System.out.println("rstate:" if (verbose) System.out.println("rstate:"
+ shard.getValue().get(ZkStateReader.STATE_PROP) + shard.getValue().getStr(ZkStateReader.STATE_PROP)
+ " live:" + " live:"
+ clusterState.liveNodesContain(shard.getValue().get( + clusterState.liveNodesContain(shard.getValue().getStr(
ZkStateReader.NODE_NAME_PROP))); ZkStateReader.NODE_NAME_PROP)));
String state = shard.getValue().get(ZkStateReader.STATE_PROP); String state = shard.getValue().getStr(ZkStateReader.STATE_PROP);
if ((state.equals(ZkStateReader.RECOVERING) || state if ((state.equals(ZkStateReader.RECOVERING) || state
.equals(ZkStateReader.SYNC) || state.equals(ZkStateReader.DOWN)) .equals(ZkStateReader.SYNC) || state.equals(ZkStateReader.DOWN))
&& clusterState.liveNodesContain(shard.getValue().get( && clusterState.liveNodesContain(shard.getValue().getStr(
ZkStateReader.NODE_NAME_PROP))) { ZkStateReader.NODE_NAME_PROP))) {
sawLiveRecovering = true; sawLiveRecovering = true;
} }
} }
@ -176,10 +175,10 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
throw new IllegalArgumentException("Cannot find collection:" + collection); throw new IllegalArgumentException("Cannot find collection:" + collection);
} }
for (Map.Entry<String,Slice> entry : slices.entrySet()) { for (Map.Entry<String,Slice> entry : slices.entrySet()) {
Map<String,ZkNodeProps> shards = entry.getValue().getShards(); Map<String,Replica> shards = entry.getValue().getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : shards.entrySet()) { for (Map.Entry<String,Replica> shard : shards.entrySet()) {
String state = shard.getValue().get(ZkStateReader.STATE_PROP); String state = shard.getValue().getStr(ZkStateReader.STATE_PROP);
if (!state.equals(ZkStateReader.ACTIVE)) { if (!state.equals(ZkStateReader.ACTIVE)) {
fail("Not all shards are ACTIVE - found a shard that is: " + state); fail("Not all shards are ACTIVE - found a shard that is: " + state);
} }

View File

@ -45,6 +45,7 @@ import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument; import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps; import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
@ -340,7 +341,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
Map<String,Slice> slices = this.zkStateReader.getClusterState().getSlices(defaultCollection); Map<String,Slice> slices = this.zkStateReader.getClusterState().getSlices(defaultCollection);
int cnt = 0; int cnt = 0;
for (Map.Entry<String,Slice> entry : slices.entrySet()) { for (Map.Entry<String,Slice> entry : slices.entrySet()) {
cnt += entry.getValue().getShards().size(); cnt += entry.getValue().getReplicasMap().size();
} }
return cnt; return cnt;
@ -378,8 +379,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
nextClient: nextClient:
// we find ou state by simply matching ports... // we find ou state by simply matching ports...
for (Map.Entry<String,Slice> slice : slices.entrySet()) { for (Map.Entry<String,Slice> slice : slices.entrySet()) {
Map<String,ZkNodeProps> theShards = slice.getValue().getShards(); Map<String,Replica> theShards = slice.getValue().getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : theShards.entrySet()) { for (Map.Entry<String,Replica> shard : theShards.entrySet()) {
int port = new URI(((HttpSolrServer) client).getBaseURL()) int port = new URI(((HttpSolrServer) client).getBaseURL())
.getPort(); .getPort();
@ -387,7 +388,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
CloudSolrServerClient csc = new CloudSolrServerClient(); CloudSolrServerClient csc = new CloudSolrServerClient();
csc.solrClient = client; csc.solrClient = client;
csc.port = port; csc.port = port;
csc.shardName = shard.getValue().get(ZkStateReader.NODE_NAME_PROP); csc.shardName = shard.getValue().getStr(ZkStateReader.NODE_NAME_PROP);
csc.info = shard.getValue(); csc.info = shard.getValue();
theClients .add(csc); theClients .add(csc);
@ -406,8 +407,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
nextJetty: nextJetty:
for (Map.Entry<String,Slice> slice : slices.entrySet()) { for (Map.Entry<String,Slice> slice : slices.entrySet()) {
Map<String,ZkNodeProps> theShards = slice.getValue().getShards(); Map<String,Replica> theShards = slice.getValue().getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : theShards.entrySet()) { for (Map.Entry<String,Replica> shard : theShards.entrySet()) {
if (shard.getKey().contains(":" + port + "_")) { if (shard.getKey().contains(":" + port + "_")) {
List<CloudJettyRunner> list = shardToJetty.get(slice.getKey()); List<CloudJettyRunner> list = shardToJetty.get(slice.getKey());
if (list == null) { if (list == null) {
@ -419,9 +420,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
CloudJettyRunner cjr = new CloudJettyRunner(); CloudJettyRunner cjr = new CloudJettyRunner();
cjr.jetty = jetty; cjr.jetty = jetty;
cjr.info = shard.getValue(); cjr.info = shard.getValue();
cjr.nodeName = shard.getValue().get(ZkStateReader.NODE_NAME_PROP); cjr.nodeName = shard.getValue().getStr(ZkStateReader.NODE_NAME_PROP);
cjr.coreNodeName = shard.getKey(); cjr.coreNodeName = shard.getKey();
cjr.url = shard.getValue().get(ZkStateReader.BASE_URL_PROP) + "/" + shard.getValue().get(ZkStateReader.CORE_NAME_PROP); cjr.url = shard.getValue().getStr(ZkStateReader.BASE_URL_PROP) + "/" + shard.getValue().getStr(ZkStateReader.CORE_NAME_PROP);
cjr.client = findClientByPort(port, theClients); cjr.client = findClientByPort(port, theClients);
list.add(cjr); list.add(cjr);
if (isLeader) { if (isLeader) {
@ -442,7 +443,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
List<CloudJettyRunner> jetties = shardToJetty.get(slice.getKey()); List<CloudJettyRunner> jetties = shardToJetty.get(slice.getKey());
assertNotNull("Test setup problem: We found no jetties for shard: " + slice.getKey() assertNotNull("Test setup problem: We found no jetties for shard: " + slice.getKey()
+ " just:" + shardToJetty.keySet(), jetties); + " just:" + shardToJetty.keySet(), jetties);
assertEquals(slice.getValue().getShards().size(), jetties.size()); assertEquals(slice.getValue().getReplicasMap().size(), jetties.size());
} }
} }
@ -752,7 +753,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
"The client count does not match up with the shard count for slice:" "The client count does not match up with the shard count for slice:"
+ shard, + shard,
zkStateReader.getClusterState().getSlice(DEFAULT_COLLECTION, shard) zkStateReader.getClusterState().getSlice(DEFAULT_COLLECTION, shard)
.getShards().size(), solrJetties.size()); .getReplicasMap().size(), solrJetties.size());
CloudJettyRunner lastJetty = null; CloudJettyRunner lastJetty = null;
for (CloudJettyRunner cjetty : solrJetties) { for (CloudJettyRunner cjetty : solrJetties) {
@ -775,7 +776,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
} }
boolean live = false; boolean live = false;
String nodeName = props.get(ZkStateReader.NODE_NAME_PROP); String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
if (zkStateReader.getClusterState().liveNodesContain(nodeName)) { if (zkStateReader.getClusterState().liveNodesContain(nodeName)) {
live = true; live = true;
} }
@ -783,7 +784,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
if (verbose) System.err.println(" num:" + num + "\n"); if (verbose) System.err.println(" num:" + num + "\n");
boolean active = props.get(ZkStateReader.STATE_PROP).equals( boolean active = props.getStr(ZkStateReader.STATE_PROP).equals(
ZkStateReader.ACTIVE); ZkStateReader.ACTIVE);
if (active && live) { if (active && live) {
if (lastNum > -1 && lastNum != num && failMessage == null) { if (lastNum > -1 && lastNum != num && failMessage == null) {
@ -877,7 +878,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
CloudJettyRunner cjetty = shardToJetty.get(s).get(i); CloudJettyRunner cjetty = shardToJetty.get(s).get(i);
ZkNodeProps props = cjetty.info; ZkNodeProps props = cjetty.info;
SolrServer client = cjetty.client.solrClient; SolrServer client = cjetty.client.solrClient;
boolean active = props.get(ZkStateReader.STATE_PROP).equals( boolean active = props.getStr(ZkStateReader.STATE_PROP).equals(
ZkStateReader.ACTIVE); ZkStateReader.ACTIVE);
if (active) { if (active) {
SolrQuery query = new SolrQuery("*:*"); SolrQuery query = new SolrQuery("*:*");
@ -886,7 +887,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
if (verbose) System.err.println(new ZkCoreNodeProps(props) if (verbose) System.err.println(new ZkCoreNodeProps(props)
.getCoreUrl() + " : " + results); .getCoreUrl() + " : " + results);
if (verbose) System.err.println("shard:" if (verbose) System.err.println("shard:"
+ props.get(ZkStateReader.SHARD_ID_PROP)); + props.getStr(ZkStateReader.SHARD_ID_PROP));
cnt += results; cnt += results;
break; break;
} }
@ -948,8 +949,8 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
for (CloudJettyRunner cjetty : cloudJettys) { for (CloudJettyRunner cjetty : cloudJettys) {
CloudSolrServerClient client = cjetty.client; CloudSolrServerClient client = cjetty.client;
for (Map.Entry<String,Slice> slice : slices.entrySet()) { for (Map.Entry<String,Slice> slice : slices.entrySet()) {
Map<String,ZkNodeProps> theShards = slice.getValue().getShards(); Map<String,Replica> theShards = slice.getValue().getReplicasMap();
for (Map.Entry<String,ZkNodeProps> shard : theShards.entrySet()) { for (Map.Entry<String,Replica> shard : theShards.entrySet()) {
String shardName = new URI( String shardName = new URI(
((HttpSolrServer) client.solrClient).getBaseURL()).getPort() ((HttpSolrServer) client.solrClient).getBaseURL()).getPort()
+ "_solr_"; + "_solr_";
@ -961,11 +962,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
} }
long count = 0; long count = 0;
String currentState = cjetty.info.get(ZkStateReader.STATE_PROP); String currentState = cjetty.info.getStr(ZkStateReader.STATE_PROP);
if (currentState != null if (currentState != null
&& currentState.equals(ZkStateReader.ACTIVE) && currentState.equals(ZkStateReader.ACTIVE)
&& zkStateReader.getClusterState().liveNodesContain( && zkStateReader.getClusterState().liveNodesContain(
cjetty.info.get(ZkStateReader.NODE_NAME_PROP))) { cjetty.info.getStr(ZkStateReader.NODE_NAME_PROP))) {
SolrQuery query = new SolrQuery("*:*"); SolrQuery query = new SolrQuery("*:*");
query.set("distrib", false); query.set("distrib", false);
count = client.solrClient.query(query).getResults().getNumFound(); count = client.solrClient.query(query).getResults().getNumFound();
@ -1209,7 +1210,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
CloudJettyRunner cjetty) throws InterruptedException { CloudJettyRunner cjetty) throws InterruptedException {
int tries = 0; int tries = 0;
while (zkStateReader.getClusterState() while (zkStateReader.getClusterState()
.liveNodesContain(cjetty.info.get(ZkStateReader.NODE_NAME_PROP))) { .liveNodesContain(cjetty.info.getStr(ZkStateReader.NODE_NAME_PROP))) {
if (tries++ == 120) { if (tries++ == 120) {
fail("Shard still reported as live in zk"); fail("Shard still reported as live in zk");
} }

View File

@ -93,7 +93,7 @@ public abstract class AbstractZkTestCase extends SolrTestCaseJ4 {
zkClient = new SolrZkClient(zkAddress, AbstractZkTestCase.TIMEOUT); zkClient = new SolrZkClient(zkAddress, AbstractZkTestCase.TIMEOUT);
Map<String,String> props = new HashMap<String,String>(); Map<String,Object> props = new HashMap<String,Object>();
props.put("configName", "conf1"); props.put("configName", "conf1");
final ZkNodeProps zkProps = new ZkNodeProps(props); final ZkNodeProps zkProps = new ZkNodeProps(props);

View File

@ -259,13 +259,13 @@ public class ChaosMonkey {
Slice theShards = zkStateReader.getClusterState().getSlices(collection) Slice theShards = zkStateReader.getClusterState().getSlices(collection)
.get(slice); .get(slice);
ZkNodeProps props = theShards.getShards().get(cloudJetty.coreNodeName); ZkNodeProps props = theShards.getReplicasMap().get(cloudJetty.coreNodeName);
if (props == null) { if (props == null) {
throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getShards().keySet()); throw new RuntimeException("shard name " + cloudJetty.coreNodeName + " not found in " + theShards.getReplicasMap().keySet());
} }
String state = props.get(ZkStateReader.STATE_PROP); String state = props.getStr(ZkStateReader.STATE_PROP);
String nodeName = props.get(ZkStateReader.NODE_NAME_PROP); String nodeName = props.getStr(ZkStateReader.NODE_NAME_PROP);
if (!cloudJetty.jetty.isRunning() if (!cloudJetty.jetty.isRunning()
@ -309,7 +309,7 @@ public class ChaosMonkey {
cjetty = jetties.get(index); cjetty = jetties.get(index);
ZkNodeProps leader = zkStateReader.getLeaderProps(collection, slice); ZkNodeProps leader = zkStateReader.getLeaderProps(collection, slice);
boolean isLeader = leader.get(ZkStateReader.NODE_NAME_PROP).equals(jetties.get(index).nodeName); boolean isLeader = leader.getStr(ZkStateReader.NODE_NAME_PROP).equals(jetties.get(index).nodeName);
if (!aggressivelyKillLeaders && isLeader) { if (!aggressivelyKillLeaders && isLeader) {
// we don't kill leaders... // we don't kill leaders...
monkeyLog("abort! I don't kill leaders"); monkeyLog("abort! I don't kill leaders");