You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2012/09/17 23:40:52 UTC
svn commit: r1386858 - in /lucene/dev/trunk/solr:
core/src/java/org/apache/solr/cloud/
solrj/src/java/org/apache/solr/common/cloud/
Author: yonik
Date: Mon Sep 17 21:40:51 2012
New Revision: 1386858
URL: http://svn.apache.org/viewvc?rev=1386858&view=rev
Log:
SOLR-3815: change clusterstate structure to add properties to slices
Modified:
lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1386858&r1=1386857&r2=1386858&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/Overseer.java Mon Sep 17 21:40:51 2012
@@ -20,12 +20,15 @@ package org.apache.solr.cloud;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
+import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import org.apache.noggit.JSONUtil;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ClosableThread;
+import org.apache.solr.common.cloud.HashPartitioner;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
@@ -216,48 +219,70 @@ public class Overseer {
}
// use the provided non null shardId
- String shardId = message.getStr(ZkStateReader.SHARD_ID_PROP);
- if (shardId == null) {
+ String sliceName = message.getStr(ZkStateReader.SHARD_ID_PROP);
+ if (sliceName == null) {
String nodeName = message.getStr(ZkStateReader.NODE_NAME_PROP);
//get shardId from ClusterState
- shardId = getAssignedId(state, nodeName, message);
+ sliceName = getAssignedId(state, nodeName, message);
}
- if(shardId == null) {
+ if(sliceName == null) {
//request new shardId
- shardId = AssignShard.assignShard(collection, state, numShards);
+ sliceName = AssignShard.assignShard(collection, state, numShards);
}
-
- Map<String,Object> props = new HashMap<String,Object>();
- Map<String,Object> coreProps = new HashMap<String,Object>(message.getProperties().size());
- coreProps.putAll(message.getProperties());
- // we don't put num_shards in the clusterstate
- coreProps.remove(ZkStateReader.NUM_SHARDS_PROP);
- coreProps.remove(QUEUE_OPERATION);
- for (Entry<String,Object> entry : coreProps.entrySet()) {
- props.put(entry.getKey(), entry.getValue());
+
+ Slice slice = state.getSlice(collection, sliceName);
+ Map<String,Object> replicaProps = new LinkedHashMap<String,Object>();
+
+ replicaProps.putAll(message.getProperties());
+ // System.out.println("########## UPDATE MESSAGE: " + JSONUtil.toJSON(message));
+ if (slice != null) {
+ Replica oldReplica = slice.getReplicasMap().get(zkCoreNodeName);
+ if (oldReplica != null && oldReplica.containsKey(ZkStateReader.LEADER_PROP)) {
+ replicaProps.put(ZkStateReader.LEADER_PROP, oldReplica.get(ZkStateReader.LEADER_PROP));
}
- Replica zkProps = new Replica(zkCoreNodeName, props);
- Slice slice = state.getSlice(collection, shardId);
- Map<String,Replica> shardProps;
- if (slice == null) {
- shardProps = new HashMap<String,Replica>();
+ }
+
+ // we don't put num_shards in the clusterstate
+ replicaProps.remove(ZkStateReader.NUM_SHARDS_PROP);
+ replicaProps.remove(QUEUE_OPERATION);
+
+
+ Replica replica = new Replica(zkCoreNodeName, replicaProps);
+
+ // TODO: where do we get slice properties in this message? or should there be a separate create-slice message if we want that?
+
+ Map<String,Object> sliceProps = null;
+ Map<String,Replica> replicas;
+
+ if (slice != null) {
+ sliceProps = slice.getProperties();
+ replicas = slice.getReplicasCopy();
} else {
- shardProps = state.getSlice(collection, shardId).getReplicasCopy();
+ replicas = new HashMap<String, Replica>(1);
}
- shardProps.put(zkCoreNodeName, zkProps);
- slice = new Slice(shardId, shardProps);
+ replicas.put(replica.getName(), replica);
+ slice = new Slice(sliceName, replicas, sliceProps);
+
ClusterState newClusterState = updateSlice(state, collection, slice);
return newClusterState;
}
private ClusterState createCollection(ClusterState state, String collectionName, int numShards) {
+ HashPartitioner hp = new HashPartitioner();
+ List<HashPartitioner.Range> ranges = hp.partitionRange(numShards, hp.fullRange());
+
+
Map<String, Map<String, Slice>> newStates = new LinkedHashMap<String,Map<String, Slice>>();
Map<String, Slice> newSlices = new LinkedHashMap<String,Slice>();
newStates.putAll(state.getCollectionStates());
for (int i = 0; i < numShards; i++) {
final String sliceName = "shard" + (i+1);
- newSlices.put(sliceName, new Slice(sliceName, Collections.EMPTY_MAP));
+
+ Map<String,Object> sliceProps = new LinkedHashMap<String,Object>(1);
+ sliceProps.put(Slice.RANGE, ranges.get(i));
+
+ newSlices.put(sliceName, new Slice(sliceName, null, sliceProps));
}
newStates.put(collectionName, newSlices);
ClusterState newClusterState = new ClusterState(state.getLiveNodes(), newStates);
@@ -282,36 +307,22 @@ public class Overseer {
}
private ClusterState updateSlice(ClusterState state, String collection, Slice slice) {
-
- final Map<String, Map<String, Slice>> newStates = new LinkedHashMap<String,Map<String,Slice>>();
- newStates.putAll(state.getCollectionStates());
-
- if (!newStates.containsKey(collection)) {
- newStates.put(collection, new LinkedHashMap<String,Slice>());
- }
-
- final Map<String, Slice> slices = newStates.get(collection);
- if (!slices.containsKey(slice.getName())) {
- slices.put(slice.getName(), slice);
+ // System.out.println("###!!!### OLD CLUSTERSTATE: " + JSONUtil.toJSON(state.getCollectionStates()));
+ // System.out.println("Updating slice:" + slice);
+
+ Map<String, Map<String, Slice>> newCollections = new LinkedHashMap<String,Map<String,Slice>>(state.getCollectionStates()); // make a shallow copy
+ Map<String, Slice> slices = newCollections.get(collection);
+ if (slices == null) {
+ slices = new HashMap<String, Slice>(1);
} else {
- final Map<String,Replica> shards = new LinkedHashMap<String,Replica>();
- final Slice existingSlice = slices.get(slice.getName());
- shards.putAll(existingSlice.getReplicasMap());
- //XXX preserve existing leader
- for(Entry<String, Replica> edit: slice.getReplicasMap().entrySet()) {
- if(existingSlice.getReplicasMap().get(edit.getKey())!=null && existingSlice.getReplicasMap().get(edit.getKey()).containsKey(ZkStateReader.LEADER_PROP)) {
- HashMap<String, Object> newProps = new HashMap<String,Object>();
- newProps.putAll(edit.getValue().getProperties());
- newProps.put(ZkStateReader.LEADER_PROP, existingSlice.getReplicasMap().get(edit.getKey()).getStr(ZkStateReader.LEADER_PROP));
- shards.put(edit.getKey(), new Replica(edit.getKey(), newProps));
- } else {
- shards.put(edit.getKey(), edit.getValue());
- }
- }
- final Slice updatedSlice = new Slice(slice.getName(), shards);
- slices.put(slice.getName(), updatedSlice);
+ slices = new LinkedHashMap<String, Slice>(slices); // make a shallow copy
}
- return new ClusterState(state.getLiveNodes(), newStates);
+ slices.put(slice.getName(), slice);
+ newCollections.put(collection, slices);
+
+ // System.out.println("###!!!### NEW CLUSTERSTATE: " + JSONUtil.toJSON(newCollections));
+
+ return new ClusterState(state.getLiveNodes(), newCollections);
}
private ClusterState setShardLeader(ClusterState state, String collection, String sliceName, String leaderUrl) {
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java?rev=1386858&r1=1386857&r2=1386858&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java Mon Sep 17 21:40:51 2012
@@ -286,21 +286,17 @@ public class ClusterState implements JSO
if (bytes == null || bytes.length == 0) {
return new ClusterState(version, liveNodes, Collections.<String, Map<String,Slice>>emptyMap());
}
-
+ // System.out.println("########## Loading ClusterState:" + new String(bytes));
LinkedHashMap<String, Object> stateMap = (LinkedHashMap<String, Object>) ZkStateReader.fromJSON(bytes);
HashMap<String,Map<String, Slice>> state = new HashMap<String,Map<String,Slice>>();
for(String collectionName: stateMap.keySet()){
Map<String, Object> collection = (Map<String, Object>)stateMap.get(collectionName);
Map<String, Slice> slices = new LinkedHashMap<String,Slice>();
- for(String sliceName: collection.keySet()) {
- Map<String, Map<String, Object>> sliceMap = (Map<String, Map<String, Object>>)collection.get(sliceName);
- Map<String, Replica> shards = new LinkedHashMap<String,Replica>();
- for(String shardName: sliceMap.keySet()) {
- shards.put(shardName, new Replica(shardName, sliceMap.get(shardName)));
- }
- Slice slice = new Slice(sliceName, shards);
- slices.put(sliceName, slice);
+
+ for (Entry<String,Object> sliceEntry : collection.entrySet()) {
+ Slice slice = new Slice(sliceEntry.getKey(), null, (Map<String,Object>)sliceEntry.getValue());
+ slices.put(slice.getName(), slice);
}
state.put(collectionName, slices);
}
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java?rev=1386858&r1=1386857&r2=1386858&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/HashPartitioner.java Mon Sep 17 21:40:51 2012
@@ -20,6 +20,7 @@ package org.apache.solr.common.cloud;
import org.apache.noggit.JSONWriter;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
/**
@@ -94,6 +95,7 @@ public class HashPartitioner {
*/
public List<Range> partitionRange(int partitions, int min, int max) {
assert max >= min;
+ if (partitions == 0) return Collections.EMPTY_LIST;
long range = (long)max - (long)min;
long srange = Math.max(1, range / partitions);
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java?rev=1386858&r1=1386857&r2=1386858&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java Mon Sep 17 21:40:51 2012
@@ -17,13 +17,15 @@ package org.apache.solr.common.cloud;
* limitations under the License.
*/
+import org.apache.noggit.JSONUtil;
+
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>
+ public Replica(String name, Map<String,Object> propMap) {
super(propMap);
this.name = name;
String nodeName = (String)propMap.get(ZkStateReader.NODE_NAME_PROP);
@@ -34,5 +36,9 @@ public class Replica extends ZkNodeProps
return name;
}
+ @Override
+ public String toString() {
+ return name + ':' + JSONUtil.toJSON(propMap, -1); // small enough, keep it on one line (i.e. no indent)
+ }
}
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java?rev=1386858&r1=1386857&r2=1386858&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java Mon Sep 17 21:40:51 2012
@@ -17,6 +17,7 @@ package org.apache.solr.common.cloud;
* limitations under the License.
*/
+import org.apache.noggit.JSONUtil;
import org.apache.noggit.JSONWriter;
import java.util.Collection;
@@ -30,11 +31,11 @@ import java.util.Map;
public class Slice extends ZkNodeProps {
public static String REPLICAS = "replicas";
public static String RANGE = "range";
- public static String LEADER = "leader";
+ public static String LEADER = "leader"; // FUTURE: do we want to record the leader as a slice property in the JSON (as opposed to isLeader as a replica property?)
private final String name;
private final HashPartitioner.Range range;
- // private final Integer replicationFactor;
+ private final Integer replicationFactor;
private final Map<String,Replica> replicas;
private final Replica leader;
@@ -45,18 +46,23 @@ public class Slice extends ZkNodeProps {
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.replicas = replicas != null ? replicas : makeReplicas((Map<String,Object>)propMap.get(REPLICAS));
- propMap.put(REPLICAS, replicas);
- String rangeStr = (String)propMap.get(RANGE);
+ Object rangeObj = propMap.get(RANGE);
HashPartitioner.Range tmpRange = null;
- if (rangeStr != null) {
+ if (rangeObj instanceof HashPartitioner.Range) {
+ tmpRange = (HashPartitioner.Range)rangeObj;
+ } else if (rangeObj != null) {
HashPartitioner hp = new HashPartitioner();
- tmpRange = hp.fromString(rangeStr);
+ tmpRange = hp.fromString(rangeObj.toString());
}
-
range = tmpRange;
- // replicationFactor = null; // future
+
+ replicationFactor = null; // future
+
+ // add the replicas *after* the other properties (for aesthetics, so it's easy to find slice properties in the JSON output)
+ this.replicas = replicas != null ? replicas : makeReplicas((Map<String,Object>)propMap.get(REPLICAS));
+ propMap.put(REPLICAS, this.replicas);
+
leader = findLeader();
}
@@ -101,10 +107,6 @@ public class Slice extends ZkNodeProps {
/**
* 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;
@@ -118,31 +120,13 @@ public class Slice extends ZkNodeProps {
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
public String toString() {
- return "Slice [replicas=" + replicas + ", name=" + name + "]";
+ return name + ':' + JSONUtil.toJSON(propMap);
}
@Override
public void write(JSONWriter jsonWriter) {
- jsonWriter.write(replicas);
+ jsonWriter.write(propMap);
}
}
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java?rev=1386858&r1=1386857&r2=1386858&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java Mon Sep 17 21:40:51 2012
@@ -17,6 +17,7 @@ package org.apache.solr.common.cloud;
* limitations under the License.
*/
+import org.apache.noggit.JSONUtil;
import org.apache.noggit.JSONWriter;
import java.util.Collections;
@@ -36,7 +37,7 @@ public class ZkNodeProps implements JSON
/**
* Construct ZKNodeProps from map.
*/
- public ZkNodeProps(Map<String,Object> propMap) { // TODO: back compat for handling Map<String,String>
+ public ZkNodeProps(Map<String,Object> propMap) {
this.propMap = propMap;
}
@@ -46,7 +47,7 @@ public class ZkNodeProps implements JSON
* key2, value2, ..., keyN, valueN
*/
public ZkNodeProps(String... keyVals) {
- this( makeMap(keyVals) );
+ this( makeMap((Object[])keyVals) );
}
public static ZkNodeProps fromKeyVals(Object... keyVals) {
@@ -105,18 +106,21 @@ public class ZkNodeProps implements JSON
return o == null ? null : o.toString();
}
- public Object get(String key,int foo) {
+ public Object get(String key) {
return propMap.get(key);
}
@Override
public String toString() {
+ return JSONUtil.toJSON(this);
+ /***
StringBuilder sb = new StringBuilder();
Set<Entry<String,Object>> entries = propMap.entrySet();
for(Entry<String,Object> entry : entries) {
sb.append(entry.getKey() + "=" + entry.getValue() + "\n");
}
return sb.toString();
+ ***/
}
/**