You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/08/04 20:32:34 UTC
svn commit: r1369396 [2/2] - in /lucene/dev/trunk/solr:
core/src/java/org/apache/solr/ core/src/java/org/apache/solr/cloud/
core/src/java/org/apache/solr/handler/admin/
core/src/java/org/apache/solr/handler/component/
core/src/java/org/apache/solr/serv...
Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=1369396&r1=1369395&r2=1369396&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java Sat Aug 4 18:32:33 2012
@@ -202,7 +202,7 @@ public class ZkControllerTest extends So
assertNotNull(reader.getLeaderUrl("collection1", "shard1", 15000));
- assertEquals("Shard(s) missing from cloudstate", 2, zkController.getZkStateReader().getCloudState().getSlice("collection1", "shard1").getShards().size());
+ assertEquals("Shard(s) missing from cloudstate", 2, zkController.getZkStateReader().getClusterState().getSlice("collection1", "shard1").getShards().size());
// unregister current leader
final ZkNodeProps shard1LeaderProps = reader.getLeaderProps(
@@ -224,10 +224,10 @@ public class ZkControllerTest extends So
reader.getLeaderUrl("collection1", "shard1", 15000));
for(int i=0;i<30;i++) {
- if(zkController.getZkStateReader().getCloudState().getSlice("collection1", "shard1").getShards().size()==1) break;
+ if(zkController.getZkStateReader().getClusterState().getSlice("collection1", "shard1").getShards().size()==1) break;
Thread.sleep(500);
}
- assertEquals("shard was not unregistered", 1, zkController.getZkStateReader().getCloudState().getSlice("collection1", "shard1").getShards().size());
+ assertEquals("shard was not unregistered", 1, zkController.getZkStateReader().getClusterState().getSlice("collection1", "shard1").getShards().size());
} finally {
System.clearProperty("solrcloud.skip.autorecovery");
System.clearProperty(ZkStateReader.NUM_SHARDS_PROP);
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java?rev=1369396&r1=1369395&r2=1369396&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrServer.java Sat Aug 4 18:32:33 2012
@@ -34,7 +34,7 @@ import org.apache.solr.client.solrj.Solr
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.util.ClientUtils;
import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.CloudState;
+import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
import org.apache.solr.common.cloud.ZkNodeProps;
@@ -139,7 +139,7 @@ public class CloudSolrServer extends Sol
// TODO: if you can hash here, you could favor the shard leader
- CloudState cloudState = zkStateReader.getCloudState();
+ ClusterState clusterState = zkStateReader.getClusterState();
SolrParams reqParams = request.getParams();
if (reqParams == null) {
@@ -159,12 +159,12 @@ public class CloudSolrServer extends Sol
Map<String,Slice> slices = new HashMap<String,Slice>();
for (int i = 0; i < collectionList.size(); i++) {
String coll= collectionList.get(i);
- ClientUtils.appendMap(coll, slices, cloudState.getSlices(coll));
+ ClientUtils.appendMap(coll, slices, clusterState.getSlices(coll));
}
- Set<String> liveNodes = cloudState.getLiveNodes();
+ Set<String> liveNodes = clusterState.getLiveNodes();
- // IDEA: have versions on various things... like a global cloudState version
+ // IDEA: have versions on various things... like a global clusterState version
// or shardAddressVersion (which only changes when the shards change)
// to allow caching.
Copied: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java (from r1369377, lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java?p2=lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java&p1=lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java&r1=1369377&r2=1369396&rev=1369396&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterState.java Sat Aug 4 18:32:33 2012
@@ -37,10 +37,10 @@ import org.slf4j.LoggerFactory;
/**
* Immutable state of the cloud. Normally you can get the state by using
- * {@link ZkStateReader#getCloudState()}.
+ * {@link ZkStateReader#getClusterState()}.
*/
-public class CloudState implements JSONWriter.Writable {
- private static Logger log = LoggerFactory.getLogger(CloudState.class);
+public class ClusterState implements JSONWriter.Writable {
+ private static Logger log = LoggerFactory.getLogger(ClusterState.class);
private final Map<String, Map<String,Slice>> collectionStates; // Map<collectionName, Map<sliceName,Slice>>
private final Set<String> liveNodes;
@@ -50,7 +50,7 @@ public class CloudState implements JSONW
private final Map<String,RangeInfo> rangeInfos = new HashMap<String,RangeInfo>();
private final Map<String,Map<String,ZkNodeProps>> leaders = new HashMap<String,Map<String,ZkNodeProps>>();
- public CloudState(Set<String> liveNodes,
+ public ClusterState(Set<String> liveNodes,
Map<String, Map<String,Slice>> collectionStates) {
this.liveNodes = new HashSet<String>(liveNodes.size());
this.liveNodes.addAll(liveNodes);
@@ -236,20 +236,20 @@ public class CloudState implements JSONW
}
/**
- * Create CloudState by reading the current state from zookeeper.
+ * Create ClusterState by reading the current state from zookeeper.
*/
- public static CloudState load(SolrZkClient zkClient, Set<String> liveNodes) throws KeeperException, InterruptedException {
+ public static ClusterState load(SolrZkClient zkClient, Set<String> liveNodes) throws KeeperException, InterruptedException {
byte[] state = zkClient.getData(ZkStateReader.CLUSTER_STATE,
null, null, true);
return load(state, liveNodes);
}
/**
- * Create CloudState from json string that is typically stored in zookeeper.
+ * Create ClusterState from json string that is typically stored in zookeeper.
*/
- public static CloudState load(byte[] bytes, Set<String> liveNodes) {
+ public static ClusterState load(byte[] bytes, Set<String> liveNodes) {
if (bytes == null || bytes.length == 0) {
- return new CloudState(liveNodes, Collections.<String, Map<String,Slice>>emptyMap());
+ return new ClusterState(liveNodes, Collections.<String, Map<String,Slice>>emptyMap());
}
LinkedHashMap<String, Object> stateMap = (LinkedHashMap<String, Object>) ZkStateReader.fromJSON(bytes);
@@ -269,7 +269,7 @@ public class CloudState implements JSONW
}
state.put(collectionName, slices);
}
- return new CloudState(liveNodes, state);
+ return new ClusterState(liveNodes, state);
}
@Override
Modified: lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1369396&r1=1369395&r2=1369396&view=diff
==============================================================================
--- lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/trunk/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Sat Aug 4 18:32:33 2012
@@ -68,7 +68,7 @@ public class ZkStateReader {
public static final String DOWN = "down";
public static final String SYNC = "sync";
- private volatile CloudState cloudState;
+ private volatile ClusterState clusterState;
private static final long SOLRCLOUD_UPDATE_DELAY = Long.parseLong(System.getProperty("solrcloud.update.delay", "5000"));
@@ -120,7 +120,7 @@ public class ZkStateReader {
}
private ScheduledExecutorService updateCloudExecutor = Executors.newScheduledThreadPool(1, new ZKTF());
- private boolean cloudStateUpdateScheduled;
+ private boolean clusterStateUpdateScheduled;
private SolrZkClient zkClient;
@@ -158,13 +158,13 @@ public class ZkStateReader {
}
// load and publish a new CollectionInfo
- public void updateCloudState(boolean immediate) throws KeeperException, InterruptedException {
- updateCloudState(immediate, false);
+ public void updateClusterState(boolean immediate) throws KeeperException, InterruptedException {
+ updateClusterState(immediate, false);
}
// load and publish a new CollectionInfo
public void updateLiveNodes() throws KeeperException, InterruptedException {
- updateCloudState(true, true);
+ updateClusterState(true, true);
}
public synchronized void createClusterStateWatchersAndUpdate() throws KeeperException,
@@ -189,17 +189,17 @@ public class ZkStateReader {
try {
// delayed approach
- // ZkStateReader.this.updateCloudState(false, false);
+ // ZkStateReader.this.updateClusterState(false, false);
synchronized (ZkStateReader.this.getUpdateLock()) {
// remake watch
final Watcher thisWatch = this;
byte[] data = zkClient.getData(CLUSTER_STATE, thisWatch, null,
true);
- CloudState clusterState = CloudState.load(data,
- ZkStateReader.this.cloudState.getLiveNodes());
+ ClusterState clusterState = ClusterState.load(data,
+ ZkStateReader.this.clusterState.getLiveNodes());
// update volatile
- cloudState = clusterState;
+ ZkStateReader.this.clusterState = clusterState;
}
} catch (KeeperException e) {
if (e.code() == KeeperException.Code.SESSIONEXPIRED
@@ -236,15 +236,15 @@ public class ZkStateReader {
log.info("Updating live nodes");
try {
// delayed approach
- // ZkStateReader.this.updateCloudState(false, true);
+ // ZkStateReader.this.updateClusterState(false, true);
synchronized (ZkStateReader.this.getUpdateLock()) {
List<String> liveNodes = zkClient.getChildren(
LIVE_NODES_ZKNODE, this, true);
Set<String> liveNodesSet = new HashSet<String>();
liveNodesSet.addAll(liveNodes);
- CloudState clusterState = new CloudState(liveNodesSet,
- ZkStateReader.this.cloudState.getCollectionStates());
- ZkStateReader.this.cloudState = clusterState;
+ ClusterState clusterState = new ClusterState(liveNodesSet,
+ ZkStateReader.this.clusterState.getCollectionStates());
+ ZkStateReader.this.clusterState = clusterState;
}
} catch (KeeperException e) {
if (e.code() == KeeperException.Code.SESSIONEXPIRED
@@ -267,51 +267,52 @@ public class ZkStateReader {
Set<String> liveNodeSet = new HashSet<String>();
liveNodeSet.addAll(liveNodes);
- CloudState clusterState = CloudState.load(zkClient, liveNodeSet);
- this.cloudState = clusterState;
+ ClusterState clusterState = ClusterState.load(zkClient, liveNodeSet);
+ this.clusterState = clusterState;
}
}
// load and publish a new CollectionInfo
- private synchronized void updateCloudState(boolean immediate,
+ private synchronized void updateClusterState(boolean immediate,
final boolean onlyLiveNodes) throws KeeperException,
InterruptedException {
// build immutable CloudInfo
if (immediate) {
- CloudState clusterState;
+ ClusterState clusterState;
synchronized (getUpdateLock()) {
- List<String> liveNodes = zkClient.getChildren(LIVE_NODES_ZKNODE, null, true);
- Set<String> liveNodesSet = new HashSet<String>();
- liveNodesSet.addAll(liveNodes);
-
+ List<String> liveNodes = zkClient.getChildren(LIVE_NODES_ZKNODE, null,
+ true);
+ Set<String> liveNodesSet = new HashSet<String>();
+ liveNodesSet.addAll(liveNodes);
+
if (!onlyLiveNodes) {
log.info("Updating cloud state from ZooKeeper... ");
- clusterState = CloudState.load(zkClient, liveNodesSet);
+ clusterState = ClusterState.load(zkClient, liveNodesSet);
} else {
log.info("Updating live nodes from ZooKeeper... ");
- clusterState = new CloudState(liveNodesSet,
- ZkStateReader.this.cloudState.getCollectionStates());
+ clusterState = new ClusterState(liveNodesSet,
+ ZkStateReader.this.clusterState.getCollectionStates());
}
}
- this.cloudState = clusterState;
+ this.clusterState = clusterState;
} else {
- if (cloudStateUpdateScheduled) {
+ if (clusterStateUpdateScheduled) {
log.info("Cloud state update for ZooKeeper already scheduled");
return;
}
log.info("Scheduling cloud state update from ZooKeeper...");
- cloudStateUpdateScheduled = true;
+ clusterStateUpdateScheduled = true;
updateCloudExecutor.schedule(new Runnable() {
public void run() {
log.info("Updating cluster state from ZooKeeper...");
synchronized (getUpdateLock()) {
- cloudStateUpdateScheduled = false;
- CloudState clusterState;
+ clusterStateUpdateScheduled = false;
+ ClusterState clusterState;
try {
List<String> liveNodes = zkClient.getChildren(LIVE_NODES_ZKNODE,
null, true);
@@ -321,13 +322,13 @@ public class ZkStateReader {
if (!onlyLiveNodes) {
log.info("Updating cloud state from ZooKeeper... ");
- clusterState = CloudState.load(zkClient, liveNodesSet);
+ clusterState = ClusterState.load(zkClient, liveNodesSet);
} else {
log.info("Updating live nodes from ZooKeeper... ");
- clusterState = new CloudState(liveNodesSet, ZkStateReader.this.cloudState.getCollectionStates());
+ clusterState = new ClusterState(liveNodesSet, ZkStateReader.this.clusterState.getCollectionStates());
}
- ZkStateReader.this.cloudState = clusterState;
+ ZkStateReader.this.clusterState = clusterState;
} catch (KeeperException e) {
if (e.code() == KeeperException.Code.SESSIONEXPIRED
@@ -346,7 +347,7 @@ public class ZkStateReader {
SolrException.ErrorCode.SERVER_ERROR, "", e);
}
// update volatile
- ZkStateReader.this.cloudState = cloudState;
+ ZkStateReader.this.clusterState = clusterState;
}
}
}, SOLRCLOUD_UPDATE_DELAY, TimeUnit.MILLISECONDS);
@@ -357,8 +358,8 @@ public class ZkStateReader {
/**
* @return information about the cluster from ZooKeeper
*/
- public CloudState getCloudState() {
- return cloudState;
+ public ClusterState getClusterState() {
+ return clusterState;
}
public Object getUpdateLock() {
@@ -411,8 +412,8 @@ public class ZkStateReader {
public ZkNodeProps getLeaderProps(String collection, String shard, int timeout) throws InterruptedException {
long timeoutAt = System.currentTimeMillis() + timeout;
while (System.currentTimeMillis() < timeoutAt) {
- if (cloudState != null) {
- final ZkNodeProps nodeProps = cloudState.getLeader(collection, shard);
+ if (clusterState != null) {
+ final ZkNodeProps nodeProps = clusterState.getLeader(collection, shard);
if (nodeProps != null) {
return nodeProps;
}
@@ -451,15 +452,15 @@ public class ZkStateReader {
public List<ZkCoreNodeProps> getReplicaProps(String collection,
String shardId, String thisNodeName, String coreName, String mustMatchStateFilter, String mustNotMatchStateFilter) {
- CloudState cloudState = this.cloudState;
- if (cloudState == null) {
+ ClusterState clusterState = this.clusterState;
+ if (clusterState == null) {
return null;
}
- Map<String,Slice> slices = cloudState.getSlices(collection);
+ Map<String,Slice> slices = clusterState.getSlices(collection);
if (slices == null) {
throw new ZooKeeperException(ErrorCode.BAD_REQUEST,
"Could not find collection in zk: " + collection + " "
- + cloudState.getCollections());
+ + clusterState.getCollections());
}
Slice replicas = slices.get(shardId);
@@ -473,7 +474,7 @@ public class ZkStateReader {
for (Entry<String,ZkNodeProps> entry : shardMap.entrySet()) {
ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(entry.getValue());
String coreNodeName = nodeProps.getNodeName() + "_" + nodeProps.getCoreName();
- if (cloudState.liveNodesContain(nodeProps.getNodeName()) && !coreNodeName.equals(filterNodeName)) {
+ if (clusterState.liveNodesContain(nodeProps.getNodeName()) && !coreNodeName.equals(filterNodeName)) {
if (mustMatchStateFilter == null || mustMatchStateFilter.equals(nodeProps.getState())) {
if (mustNotMatchStateFilter == null || !mustNotMatchStateFilter.equals(nodeProps.getState())) {
nodes.add(nodeProps);