You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/06/06 10:56:56 UTC

svn commit: r1346791 - in /lucene/dev/branches/branch_4x/solr: ./ solrj/ solrj/src/java/org/apache/solr/common/cloud/

Author: siren
Date: Wed Jun  6 08:56:55 2012
New Revision: 1346791

URL: http://svn.apache.org/viewvc?rev=1346791&view=rev
Log:
improve javadocs, remove unused constructor. merged from trunk

Modified:
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java?rev=1346791&r1=1346790&r2=1346791&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java Wed Jun  6 08:56:55 2012
@@ -33,6 +33,10 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.common.cloud.HashPartitioner.Range;
 import org.apache.zookeeper.KeeperException;
 
+/**
+ * Immutable state of the cloud. Normally you can get the state by using
+ * {@link ZkStateReader#getCloudState()}.
+ */
 public class CloudState implements JSONWriter.Writable {
 	private final Map<String, Map<String,Slice>> collectionStates;  // Map<collectionName, Map<sliceName,Slice>>
 	private final Set<String> liveNodes;
@@ -40,15 +44,7 @@ public class CloudState implements JSONW
   private final HashPartitioner hp = new HashPartitioner();
   
   private final Map<String,RangeInfo> rangeInfos = new HashMap<String,RangeInfo>();
-  public Map<String,Map<String,ZkNodeProps>> leaders = new HashMap<String,Map<String,ZkNodeProps>>();
-
-
-	public CloudState() {
-		this.liveNodes = new HashSet<String>();
-		this.collectionStates = new HashMap<String,Map<String,Slice>>(0);
-		addRangeInfos(collectionStates.keySet());
-		getShardLeaders();
-	}
+  private final Map<String,Map<String,ZkNodeProps>> leaders = new HashMap<String,Map<String,ZkNodeProps>>();
 
 	public CloudState(Set<String> liveNodes,
 			Map<String, Map<String,Slice>> collectionStates) {
@@ -84,7 +80,10 @@ public class CloudState implements JSONW
       }
     }
   }
-	
+
+	/**
+	 * Get properties of a shard leader for specific collection.
+	 */
 	public ZkNodeProps getLeader(String collection, String shard) {
 	  Map<String,ZkNodeProps> collectionLeaders = leaders.get(collection);
 	  if (collectionLeaders == null) return null;
@@ -97,6 +96,9 @@ public class CloudState implements JSONW
     }
   }
 
+  /**
+   * Get the index Slice for collection.
+   */
   public Slice getSlice(String collection, String slice) {
 		if (collectionStates.containsKey(collection)
 				&& collectionStates.get(collection).containsKey(slice))
@@ -104,12 +106,18 @@ public class CloudState implements JSONW
 		return null;
 	}
 
+  /**
+   * Get all slices for collection.
+   */
 	public Map<String, Slice> getSlices(String collection) {
 		if(!collectionStates.containsKey(collection))
 			return null;
 		return Collections.unmodifiableMap(collectionStates.get(collection));
 	}
 
+	/**
+	 * Get collection names.
+	 */
 	public Set<String> getCollections() {
 		return Collections.unmodifiableSet(collectionStates.keySet());
 	}
@@ -118,10 +126,17 @@ public class CloudState implements JSONW
 		return Collections.unmodifiableMap(collectionStates);
 	}
 
+	/**
+	 * Get names of the currently live nodes.
+	 */
 	public Set<String> getLiveNodes() {
 		return Collections.unmodifiableSet(liveNodes);
 	}
 
+	/**
+	 * Get shardId for core.
+	 * @param coreNodeName in the form of nodeName_coreName
+	 */
 	public String getShardId(String coreNodeName) {
 	  for (Entry<String, Map<String, Slice>> states: collectionStates.entrySet()){
 	    for(Entry<String, Slice> slices: states.getValue().entrySet()) {
@@ -135,6 +150,9 @@ public class CloudState implements JSONW
 	  return null;
 	}
 
+	/**
+	 * Check if node is alive. 
+	 */
 	public boolean liveNodesContain(String name) {
 		return liveNodes.contains(name);
 	}
@@ -170,7 +188,11 @@ public class CloudState implements JSONW
     rangeInfos.put(collection, rangeInfo);
     return rangeInfo;
   }
-	
+
+  /**
+   * Get shard id for hash. This is used when determining which Slice the
+   * document is to be submitted to.
+   */
   public String getShard(int hash, String collection) {
     RangeInfo rangInfo = getRanges(collection);
     
@@ -193,12 +215,18 @@ public class CloudState implements JSONW
 		return sb.toString();
 	}
 
+	/**
+	 * Create CloudState by reading the current state from zookeeper. 
+	 */
 	public static CloudState 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.
+	 */
 	public static CloudState load(byte[] bytes, Set<String> liveNodes) throws KeeperException, InterruptedException {
     if (bytes == null || bytes.length == 0) {
       return new CloudState(liveNodes, Collections.<String, Map<String,Slice>>emptyMap());
@@ -229,7 +257,7 @@ public class CloudState implements JSONW
     jsonWriter.write(collectionStates);
   }
   
-  class RangeInfo {
+  private class RangeInfo {
     private List<Range> ranges;
     private ArrayList<String> shardList;
   }

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java?rev=1346791&r1=1346790&r2=1346791&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java Wed Jun  6 08:56:55 2012
@@ -23,7 +23,10 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-// immutable
+/**
+ * A Slice contains immutable information about all shards that share the same
+ * shard id (shard leader and replicas).
+ */
 public class Slice implements JSONWriter.Writable {
   private final Map<String,ZkNodeProps> shards;
   private final String name;
@@ -33,10 +36,20 @@ public class Slice implements JSONWriter
     this.name = name;
   }
   
+  /**
+   * Get properties for all shards in this slice.
+   * 
+   * @return map containing coreNodeName as the key, see
+   *         {@link ZkStateReader#getCoreNodeName(String, String)}, ZKNodeProps
+   *         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()) {
@@ -46,6 +59,9 @@ public class Slice implements JSONWriter
     return shards;
   }
   
+  /**
+   * Return slice name (shard id).
+   */
   public String getName() {
     return name;
   }

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java?rev=1346791&r1=1346790&r2=1346791&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java Wed Jun  6 08:56:55 2012
@@ -25,25 +25,33 @@ import java.util.Map.Entry;
 
 import org.apache.noggit.JSONWriter;
 
-// Immutable
+/**
+ * ZkNodeProps contains immutable properties for a shard/solr core.
+ */
 public class ZkNodeProps implements JSONWriter.Writable {
 
   private final Map<String,String> propMap;
 
+  /**
+   * Construct ZKNodeProps from map.
+   */
   public ZkNodeProps(Map<String,String> propMap) {
     this.propMap = new HashMap<String,String>();
     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);
   }
   
-  public ZkNodeProps() {
-    propMap = new HashMap<String,String>();
-  }
-  
+  /**
+   * Constructor that populates the from array of Strings in form key1, value1,
+   * key2, value2, ..., keyN, valueN
+   */
   public ZkNodeProps(String... keyVals) {
     if (keyVals.length % 2 != 0) {
       throw new IllegalArgumentException("arguments should be key,value");
@@ -53,15 +61,24 @@ public class ZkNodeProps implements JSON
       propMap.put(keyVals[i], keyVals[i+1]);
     }
   }
-  
+ 
+  /**
+   * Get property keys.
+   */
   public Set<String> keySet() {
     return Collections.unmodifiableSet(propMap.keySet());
   }
 
+  /**
+   * Get all properties as map.
+   */
   public Map<String,String> getProperties() {
     return Collections.unmodifiableMap(propMap);
   }
 
+  /**
+   * Create ZkNodeProps from json string that is typically stored in zookeeper.
+   */
   public static ZkNodeProps load(byte[] bytes) {
     Map<String, String> props = (Map<String, String>) ZkStateReader.fromJSON(bytes);
     return new ZkNodeProps(props);
@@ -72,6 +89,9 @@ public class ZkNodeProps implements JSON
     jsonWriter.write(propMap);
   }
   
+  /**
+   * Get property value.
+   */
   public String get(String key) {
     return propMap.get(key);
   }
@@ -85,7 +105,10 @@ public class ZkNodeProps implements JSON
     }
     return sb.toString();
   }
-  
+
+  /**
+   * Check if property key exists.
+   */
   public boolean containsKey(String key) {
     return propMap.containsKey(key);
   }

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1346791&r1=1346790&r2=1346791&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Wed Jun  6 08:56:55 2012
@@ -378,6 +378,9 @@ public class ZkStateReader {
 
 	}
   
+  /**
+   * Get shard leader url.
+   */
   public String getLeaderUrl(String collection, String shard) throws InterruptedException, KeeperException {
     return getLeaderUrl(collection, shard, 1000);
   }
@@ -389,6 +392,9 @@ public class ZkStateReader {
     return props.getCoreUrl();
   }
   
+  /**
+   * Get shard leader properties.
+   */
   public ZkNodeProps getLeaderProps(String collection, String shard) throws InterruptedException {
     return getLeaderProps(collection, shard, 1000);
   }
@@ -408,12 +414,23 @@ public class ZkStateReader {
     throw new RuntimeException("No registered leader was found, collection:" + collection + " slice:" + shard);
   }
 
+  /**
+   * Get path where shard leader properties live in zookeeper.
+   */
   public static String getShardLeadersPath(String collection, String shardId) {
     return COLLECTIONS_ZKNODE + "/" + collection + "/"
         + SHARD_LEADERS_ZKNODE + (shardId != null ? ("/" + shardId)
         : "");
   }
-  
+
+  /**
+   * Get CoreNodeName for a core. This name is unique across the collection.  
+   * @param nodeName in form: 127.0.0.1:54065_solr
+   */
+  public static String getCoreNodeName(String nodeName, String coreName) {
+    return nodeName + "_" + coreName;
+  }
+
   public List<ZkCoreNodeProps> getReplicaProps(String collection,
       String shardId, String thisNodeName, String coreName) {
     return getReplicaProps(collection, shardId, thisNodeName, coreName, null);