lucene-solr-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From markrmil...@apache.org
Subject svn commit: r900915 - in /lucene/solr/branches/cloud/src: java/org/apache/solr/cloud/CloudState.java java/org/apache/solr/cloud/ZkController.java java/org/apache/solr/handler/component/QueryComponent.java test/org/apache/solr/cloud/ZkControllerTest.java
Date Tue, 19 Jan 2010 19:11:27 GMT
Author: markrmiller
Date: Tue Jan 19 19:11:23 2010
New Revision: 900915

URL: http://svn.apache.org/viewvc?rev=900915&view=rev
Log:
change slices from list to map key'd by slice name

Modified:
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/CloudState.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java
    lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/CloudState.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/CloudState.java?rev=900915&r1=900914&r2=900915&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/CloudState.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/CloudState.java Tue Jan 19 19:11:23
2010
@@ -19,13 +19,12 @@
 
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 // effectively immutable
 public class CloudState {
-  private Map<String,List<Slice>> collectionStates = new HashMap<String,List<Slice>>();
+  private Map<String,Map<String,Slice>> collectionStates = new HashMap<String,Map<String,Slice>>();
   private Set<String> liveNodes = null;
   
   public CloudState(Set<String> liveNodes) {
@@ -33,13 +32,13 @@
   }
   
   // nocommit : only call before publishing
-  void addSlices(String collection, List<Slice> slices) {
+  void addSlices(String collection, Map<String,Slice> slices) {
     collectionStates.put(collection, slices);
   }
   
   // nocommit
-  public List<Slice> getSlices(String collection) {
-    return Collections.unmodifiableList(collectionStates.get(collection));
+  public Map<String,Slice> getSlices(String collection) {
+    return Collections.unmodifiableMap(collectionStates.get(collection));
   }
   
   public Set<String> getLiveNodes() {

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java?rev=900915&r1=900914&r2=900915&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/cloud/ZkController.java Tue Jan 19
19:11:23 2010
@@ -383,11 +383,11 @@
 
   private void createEphemeralNode() throws KeeperException,
       InterruptedException {
-    String nodeName = getNodeUrl();
+    String nodeName = getNodeName();
     zkClient.makePath(NODES_ZKNODE + "/" + nodeName, CreateMode.EPHEMERAL);
   }
   
-  private String getNodeUrl() {
+  private String getNodeName() {
     return hostName + ":" + localHostPort + "_"+ localHostContext;
   }
 
@@ -407,11 +407,11 @@
     for (String collection : collections) {
       String shardIdPaths = COLLECTIONS_ZKNODE + "/" + collection + SHARDS_ZKNODE;
       List<String> shardIdNames = zkClient.getChildren(shardIdPaths, null);
-      List<Slice> slices = new ArrayList<Slice>();
+      Map<String,Slice> slices = new HashMap<String,Slice>();
       for(String shardIdZkPath : shardIdNames) {
         Map<String,ZkNodeProps> shardsMap = readShards(shardIdPaths + "/" + shardIdZkPath);
         Slice slice = new Slice(shardIdZkPath, shardsMap);
-        slices.add(slice);
+        slices.put(shardIdZkPath, slice);
       }
       cloudInfo.addSlices(collection, slices);
       
@@ -575,7 +575,7 @@
 
     props.put(ROLE_PROP, cloudDesc.getRole());
     
-    props.put(NODE_NAME, getNodeUrl());
+    props.put(NODE_NAME, getNodeName());
 
     props.store(new DataOutputStream(baos));
 

Modified: lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=900915&r1=900914&r2=900915&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryComponent.java
(original)
+++ lucene/solr/branches/cloud/src/java/org/apache/solr/handler/component/QueryComponent.java
Tue Jan 19 19:11:23 2010
@@ -150,7 +150,7 @@
               cloudState =  req.getCore().getCoreDescriptor().getCoreContainer().getZooKeeperController().getCloudState();
           }
           String sliceStr = rb.slices[i];
-          List<Slice> slices = cloudState.getSlices(sliceStr);
+          Map<String,Slice> slices = cloudState.getSlices(sliceStr);
 
           if (slices==null || slices.size() == 0) {
             // TODO: we could treat this as "all servers down" for a slice if partial results
are enabled.

Modified: lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=900915&r1=900914&r2=900915&view=diff
==============================================================================
--- lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/solr/branches/cloud/src/test/org/apache/solr/cloud/ZkControllerTest.java Tue Jan
19 19:11:23 2010
@@ -22,6 +22,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import junit.framework.TestCase;
 
@@ -77,12 +78,12 @@
           "localhost", "8983", "/solr");
       zkController.updateCloudState();
       CloudState cloudInfo = zkController.getCloudState();
-      List<Slice> slices = cloudInfo.getSlices("collection1");
+      Map<String,Slice> slices = cloudInfo.getSlices("collection1");
       assertNotNull(slices);
 
 
       if (DEBUG) {
-        for(Slice slice : slices) {
+        for(Slice slice : slices.values()) {
           for (String shard : slice.getShards().keySet()) {
             System.out.println("shard:" + shard);
           }



Mime
View raw message