lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From markrmil...@apache.org
Subject svn commit: r1209575 - in /lucene/dev/branches/solrcloud/solr: core/src/java/org/apache/solr/cloud/ core/src/test/org/apache/solr/cloud/ solrj/src/java/org/apache/solr/common/cloud/
Date Fri, 02 Dec 2011 16:46:49 GMT
Author: markrmiller
Date: Fri Dec  2 16:46:49 2011
New Revision: 1209575

URL: http://svn.apache.org/viewvc?rev=1209575&view=rev
Log:
SOLR-2821: Improve how cluster state is managed in ZooKeeper.

Modified:
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreAssignment.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreState.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/Overseer.java
    lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/CloudStateTest.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudStateUtility.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreAssignment.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreAssignment.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreAssignment.java
(original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreAssignment.java
Fri Dec  2 16:46:49 2011
@@ -17,92 +17,80 @@ package org.apache.solr.cloud;
  * the License.
  */
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+
+import org.apache.noggit.CharArr;
+import org.apache.noggit.JSONUtil;
+import org.apache.noggit.ObjectBuilder;
 
 public class CoreAssignment {
-  
+
+  private static String COLLECTION="_collection";
+  private static String CORE="_core";
+
   public String getCoreName() {
-    return coreName;
+    return properties.get(CORE);
   }
 
   public String getCollectionName() {
-    return collectionName;
+    return properties.get(COLLECTION);
   }
 
-  private final String coreName;
-  private final String collectionName;
   private final Map<String, String> properties;
   
   public Map<String,String> getProperties() {
     return properties;
   }
 
-  public CoreAssignment(String coreName, String collectionName, Map<String, String>
properties) {
-    this.coreName = coreName;
-    this.collectionName = collectionName;
-    this.properties = Collections.unmodifiableMap(properties);
+  private CoreAssignment(Map<String, String> props) {
+    this.properties = Collections.unmodifiableMap(props);
   }
   
-  public static byte[] tobytes(CoreAssignment... states) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    
-    try {
-      dos.write(states.length);
-      for (CoreAssignment state : states) {
-        dos.writeUTF(state.coreName);
-        dos.writeUTF(state.collectionName);
-        dos.write(state.properties.size());
-        for(Entry<String, String> prop: state.properties.entrySet()) {
-          dos.writeUTF(prop.getKey());
-          if(prop.getValue()==null) {
-            throw new NullPointerException("value was null for key:" + prop.getKey());
-          }
-          dos.writeUTF(prop.getValue());
-        }
+  public CoreAssignment(String coreName, String collectionName, Map<String,String>
properties) {
+    HashMap<String,String> props = new HashMap<String,String>();
+    props.putAll(properties);
+    props.put(COLLECTION, collectionName);
+    props.put(CORE, coreName);
+    this.properties = Collections.unmodifiableMap(props);
+  }
+  
+  public static byte[] tobytes(CoreAssignment... assignments) throws IOException {
+    CharArr out = new CharArr();
+    out.append(JSONUtil.ARRAY_START);
+    boolean first = true;
+    for (CoreAssignment assignment : assignments) {
+      if (first) {
+        first = false;
+      } else {
+        out.append(JSONUtil.VALUE_SEPARATOR);
       }
-      return baos.toByteArray();
-    } finally {
-      dos.close();
+      out.append(JSONUtil.toJSON(assignment.properties));
+      
     }
+    
+    out.append(JSONUtil.ARRAY_END);
+
+    return out.toString().getBytes("utf-8");
   }
   
   public static CoreAssignment[] fromBytes(byte[] bytes) throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-    DataInputStream dis = new DataInputStream(bais);
-    try {
-      int count = dis.read();
-      CoreAssignment[] states = new CoreAssignment[count];
-      for (int i = 0; i < count; i++) {
-        String coreName = dis.readUTF();
-        String collectionName = dis.readUTF();
-        int propcount = dis.read();
-        HashMap<String, String> props = new HashMap<String, String>();
-        for(int j=0;j<propcount;j++) {
-          String key = dis.readUTF();
-          String value = dis.readUTF();
-          props.put(key, value);
-        }
-        
-        states[i] = new CoreAssignment(coreName, collectionName, props);
-      }
-      return states;
-    } finally {
-      dis.close();
+    ArrayList<CoreAssignment> states = new ArrayList<CoreAssignment>(); 
+    List<Map<String, String>> stateMaps = (List<Map<String, String>>)ObjectBuilder.fromJSON(new
String(bytes,"utf-8"));
+    for (Map<String,String> state : stateMaps) {
+      states.add(new CoreAssignment(state));
     }
+    
+    return states.toArray(new CoreAssignment[states.size()]);
   }
   
   @Override
   public int hashCode() {
-    return coreName.hashCode();
+    return getCoreName().hashCode();
   }
   
   @Override
@@ -112,7 +100,7 @@ public class CoreAssignment {
   
   @Override
   public String toString() {
-    return "coll:" + collectionName + " core:" + coreName + " props:" + properties;
+    return "coll:" + getCollectionName() + " core:" + getCoreName() + " props:" + properties;
   }
   
 }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreState.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreState.java
(original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/CoreState.java
Fri Dec  2 16:46:49 2011
@@ -17,92 +17,81 @@ package org.apache.solr.cloud;
  * the License.
  */
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+
+import org.apache.noggit.CharArr;
+import org.apache.noggit.JSONUtil;
+import org.apache.noggit.ObjectBuilder;
 
 public class CoreState {
+
   
+  private static String COLLECTION="_collection";
+  private static String CORE="_core";
+
   public String getCoreName() {
-    return coreName;
+    return properties.get(CORE);
   }
 
   public String getCollectionName() {
-    return collectionName;
+    return properties.get(COLLECTION);
   }
 
-  private final String coreName;
-  private final String collectionName;
   private final Map<String, String> properties;
   
   public Map<String,String> getProperties() {
     return properties;
   }
 
-  public CoreState(String coreName, String collectionName, Map<String, String> properties)
{
-    this.coreName = coreName;
-    this.collectionName = collectionName;
-    this.properties = Collections.unmodifiableMap(properties);
+  private CoreState(Map<String, String> props) {
+    this.properties = Collections.unmodifiableMap(props);
+  }
+  
+  public CoreState(String coreName, String collectionName, Map<String,String> properties)
{
+    HashMap<String,String> props = new HashMap<String,String>();
+    props.putAll(properties);
+    props.put(COLLECTION, collectionName);
+    props.put(CORE, coreName);
+    this.properties = Collections.unmodifiableMap(props);
   }
   
   public static byte[] tobytes(CoreState... states) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    
-    try {
-      dos.write(states.length);
-      for (CoreState state : states) {
-        dos.writeUTF(state.coreName);
-        dos.writeUTF(state.collectionName);
-        dos.write(state.properties.size());
-        for(Entry<String, String> prop: state.properties.entrySet()) {
-          dos.writeUTF(prop.getKey());
-          if(prop.getValue()==null) {
-            throw new NullPointerException("value was null for key:" + prop.getKey());
-          }
-          dos.writeUTF(prop.getValue());
-        }
+    CharArr out = new CharArr();
+    out.append(JSONUtil.ARRAY_START);
+    boolean first = true;
+    for (CoreState state : states) {
+      if (first) {
+        first = false;
+      } else {
+        out.append(JSONUtil.VALUE_SEPARATOR);
       }
-      return baos.toByteArray();
-    } finally {
-      dos.close();
+      out.append(JSONUtil.toJSON(state.properties));
+      
     }
+    
+    out.append(JSONUtil.ARRAY_END);
+
+    return out.toString().getBytes("utf-8");
   }
   
   public static CoreState[] fromBytes(byte[] bytes) throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-    DataInputStream dis = new DataInputStream(bais);
-    try {
-      int count = dis.read();
-      CoreState[] states = new CoreState[count];
-      for (int i = 0; i < count; i++) {
-        String coreName = dis.readUTF();
-        String collectionName = dis.readUTF();
-        int propcount = dis.read();
-        HashMap<String, String> props = new HashMap<String, String>();
-        for(int j=0;j<propcount;j++) {
-          String key = dis.readUTF();
-          String value = dis.readUTF();
-          props.put(key, value);
-        }
-        
-        states[i] = new CoreState(coreName, collectionName, props);
-      }
-      return states;
-    } finally {
-      dis.close();
+    ArrayList<CoreState> states = new ArrayList<CoreState>(); 
+    List<Map<String, String>> stateMaps = (List<Map<String, String>>)ObjectBuilder.fromJSON(new
String(bytes,"utf-8"));
+    for (Map<String,String> state : stateMaps) {
+      states.add(new CoreState(state));
     }
+    
+    return states.toArray(new CoreState[states.size()]);
   }
   
   @Override
   public int hashCode() {
-    return coreName.hashCode();
+    return getCoreName().hashCode();
   }
   
   @Override
@@ -112,7 +101,7 @@ public class CoreState {
   
   @Override
   public String toString() {
-    return "coll:" + collectionName + " core:" + coreName + " props:" + properties;
+    return "coll:" + getCollectionName() + " core:" + getCoreName() + " props:" + properties;
   }
   
 }

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/Overseer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/Overseer.java Fri
Dec  2 16:46:49 2011
@@ -78,6 +78,8 @@ public class Overseer implements NodeSta
     } catch (KeeperException e) {
       // TODO Auto-generated catch block
       e.printStackTrace();
+    } catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"",e);
     }
   }
   
@@ -98,7 +100,7 @@ public class Overseer implements NodeSta
   }
 
   public synchronized void createClusterStateWatchersAndUpdate()
-      throws KeeperException, InterruptedException {
+      throws KeeperException, InterruptedException, IOException {
     // We need to fetch the current cluster state and the set of live nodes
     
     if (!zkClient.exists(ZkStateReader.CLUSTER_STATE)) {

Modified: lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/CloudStateTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/CloudStateTest.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/CloudStateTest.java
(original)
+++ lucene/dev/branches/solrcloud/solr/core/src/test/org/apache/solr/cloud/CloudStateTest.java
Fri Dec  2 16:46:49 2011
@@ -28,29 +28,50 @@ import org.apache.solr.common.cloud.Slic
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.junit.Test;
 
-// TODO: assert something
 public class CloudStateTest extends SolrTestCaseJ4 {
   @Test
   public void testStoreAndRead() throws Exception {
     Map<String,Map<String,Slice>> collectionStates = new HashMap<String,Map<String,Slice>>();
     Set<String> liveNodes = new HashSet<String>();
+    liveNodes.add("node1");
+    liveNodes.add("node2");
     
     Map<String,Slice> slices = new HashMap<String,Slice>();
     Map<String,ZkNodeProps> sliceToProps = new HashMap<String,ZkNodeProps>();
     Map<String,String> props = new HashMap<String,String>();
 
     props.put("prop1", "value");
+    props.put("prop2", "value2");
     ZkNodeProps zkNodeProps = new ZkNodeProps(props);
     sliceToProps.put("node1", zkNodeProps);
     Slice slice = new Slice("shard1", sliceToProps);
     slices.put("shard1", slice);
+    Slice slice2 = new Slice("shard2", sliceToProps);
+    slices.put("shard2", slice2);
     collectionStates.put("collection1", slices);
+    collectionStates.put("collection2", slices);
     
     CloudState cloudState = new CloudState(liveNodes, collectionStates);
     byte[] bytes = CloudState.store(cloudState);
     
     CloudState loadedCloudState = CloudState.load(bytes, liveNodes);
     
-    //System.out.println("cloud state:" + loadedCloudState);
+    assertEquals("Provided liveNodes not used properly", 2, loadedCloudState
+        .getLiveNodes().size());
+    assertEquals("No collections found", 2, loadedCloudState.getCollections().size());
+    assertEquals("Poperties not copied properly", zkNodeProps.get("prop1"), loadedCloudState.getSlice("collection1",
"shard1").getShards().get("node1").get("prop1"));
+    assertEquals("Poperties not copied properly", zkNodeProps.get("prop2"), loadedCloudState.getSlice("collection1",
"shard1").getShards().get("node1").get("prop2"));
+
+    loadedCloudState = CloudState.load(new byte[0], liveNodes);
+    
+    assertEquals("Provided liveNodes not used properly", 2, loadedCloudState
+        .getLiveNodes().size());
+    assertEquals("Should not have collections", 0, loadedCloudState.getCollections().size());
+
+    loadedCloudState = CloudState.load((byte[])null, liveNodes);
+    
+    assertEquals("Provided liveNodes not used properly", 2, loadedCloudState
+        .getLiveNodes().size());
+    assertEquals("Should not have collections", 0, loadedCloudState.getCollections().size());
   }
 }

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
(original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudState.java
Fri Dec  2 16:46:49 2011
@@ -17,39 +17,24 @@ package org.apache.solr.common.cloud;
  * limitations under the License.
  */
 
-import java.io.BufferedWriter;
-import java.io.ByteArrayInputStream;
 import java.io.IOException;
-import java.io.StringWriter;
-import java.io.UnsupportedEncodingException;
-import java.io.Writer;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Set;
 
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.XMLErrorLogger;
+import org.apache.noggit.CharArr;
+import org.apache.noggit.JSONUtil;
+import org.apache.noggit.ObjectBuilder;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
 
 // quasi immutable :(
 public class CloudState {
 	protected static Logger log = LoggerFactory.getLogger(CloudState.class);
-	private static final XMLErrorLogger xmllog = new XMLErrorLogger(log);
 	private final Map<String, Map<String, Slice>> collectionStates;
 	private final Set<String> liveNodes;
 
@@ -126,152 +111,87 @@ public class CloudState {
 		return sb.toString();
 	}
 
-	public static CloudState load(SolrZkClient zkClient, Set<String> liveNodes) throws
KeeperException, InterruptedException {
+	public static CloudState load(SolrZkClient zkClient, Set<String> liveNodes) throws
KeeperException, InterruptedException, IOException {
     byte[] state = zkClient.getData(ZkStateReader.CLUSTER_STATE,
         null, null);
     return load(state, liveNodes);
 	}
 	
-	public static CloudState load(byte[] state, Set<String> liveNodes) throws KeeperException,
InterruptedException {
-	  Map<String,Map<String,Slice>> colStates = new HashMap<String, Map<String,
Slice>>();
-	  
-		if(state != null && state.length > 0) {
-			InputSource is = new InputSource(new ByteArrayInputStream(state));
-			DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
-	
-			try {
-				DocumentBuilder db = dbf.newDocumentBuilder();
-	
-				db.setErrorHandler(xmllog);
-				Document doc = db.parse(is);
-	
-				
-				
-				Element root = doc.getDocumentElement();
-	
-				NodeList collectionStates = root.getChildNodes();
-				for (int x = 0; x < collectionStates.getLength(); x++) {
-					Node collectionState = collectionStates.item(x);
-					String collectionName = collectionState.getAttributes()
-							.getNamedItem("name").getNodeValue();
-					NodeList slices = collectionState.getChildNodes();
-					for (int y = 0; y < slices.getLength(); y++) {
-						Node slice = slices.item(y);
-						Node sliceName = slice.getAttributes().getNamedItem("name");
-						
-						NodeList shardsNodeList = slice.getChildNodes();
-						Map<String, ZkNodeProps> shards = new HashMap<String, ZkNodeProps>();
-						for (int z = 0; z < shardsNodeList.getLength(); z++) {
-							Node shard = shardsNodeList.item(z);
-							String shardName = shard.getAttributes()
-									.getNamedItem("name").getNodeValue();
-							NodeList propsList = shard.getChildNodes();
-							Map<String,String> props = new HashMap<String,String>();
-							
-							for (int i = 0; i < propsList.getLength(); i++) {
-								Node prop = propsList.item(i);
-								String propName = prop.getAttributes()
-										.getNamedItem("name").getNodeValue();
-								String propValue = prop.getTextContent();
-								props.put(propName, propValue);
-							}
-							shards.put(shardName, new ZkNodeProps(props));
-						}
-            Map<String,Slice> s = null;
-            if (!colStates.containsKey(collectionName)) {
-              s = new HashMap<String,Slice>();
-              colStates.put(collectionName, s);
-            } else {
-              s = colStates.get(collectionName);
+	public static CloudState load(byte[] bytes, Set<String> liveNodes) throws KeeperException,
InterruptedException, IOException {
+    if (bytes == null || bytes.length == 0) {
+      return new CloudState(liveNodes, Collections.EMPTY_MAP);
+    }
+    
+    LinkedHashMap<String, Object> stateMap = (LinkedHashMap<String, Object>)
ObjectBuilder.fromJSON(new String(bytes, "utf-8"));
+    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);
+      HashMap<String, Slice> slices = new HashMap<String,Slice>();
+      for(String sliceName: collection.keySet()) {
+        Map<String, Map<String, String>> sliceMap = (Map<String, Map<String,
String>>)collection.get(sliceName);
+        HashMap<String, ZkNodeProps> shards = new HashMap<String,ZkNodeProps>();
+        for(String shardName: sliceMap.keySet()) {
+          shards.put(shardName, new ZkNodeProps(sliceMap.get(shardName)));
+        }
+        Slice slice = new Slice(sliceName, shards);
+        slices.put(sliceName, slice);
+      }
+      state.put(collectionName, slices);
+    }
+    return new CloudState(liveNodes, state);
+	}
+
+  public static byte[] store(CloudState state)
+      throws IOException {
+    CharArr out = new CharArr();
+    out.append(JSONUtil.OBJECT_START);
+    int collCount = state.getCollections().size();
+
+    for (String collectionName : state.getCollections()) {
+      JSONUtil.writeString(collectionName, 0, collectionName.length(), out);
+      out.append(JSONUtil.NAME_SEPARATOR);
+      Map<String, Slice> slices = state.getSlices(collectionName);
+      out.append(JSONUtil.OBJECT_START);
+      int sliceCount = slices.keySet().size();
+      for(String sliceName: slices.keySet()) {
+        JSONUtil.writeString(sliceName, 0, sliceName.length(), out);
+        out.append(JSONUtil.NAME_SEPARATOR);
+        Slice slice = slices.get(sliceName);
+        Map<String, ZkNodeProps> shards = slice.getShards();
+        out.append(JSONUtil.OBJECT_START);
+        int shardCount = shards.keySet().size();
+        for(String shardName: shards.keySet()) {
+          ZkNodeProps props = shards.get(shardName);
+          JSONUtil.writeString(shardName, 0, shardName.length(), out);
+          out.append(JSONUtil.NAME_SEPARATOR);
+          out.append(JSONUtil.OBJECT_START);
+          int propCount = props.keySet().size();
+          for(String key: props.keySet()) {
+            JSONUtil.writeString(key, 0, key.length(), out);
+            out.append(JSONUtil.NAME_SEPARATOR);
+            JSONUtil.writeString(props.get(key), 0, props.get(key).length(), out);
+            if (--propCount != 0) {
+              out.append(JSONUtil.VALUE_SEPARATOR);
             }
-            String sn = sliceName.getTextContent();
-            Slice sl = s.get(sn);
-
-            if (sl == null) {
-              sl = new Slice(sliceName.getTextContent(), shards);
-              s.put(sn, sl);
-            } else {
-              sl = new Slice(sliceName.getTextContent(), shards);
-            }
-			      
-//			      Slice existingSlice = colStates.get(collection).get(slice.getName());
-//			      shards.putAll(existingSlice.getShards());
-//			      shards.putAll(slice.getShards());
-//			      Slice updatedSlice = new Slice(slice.getName(), shards);
-//			      collectionStates.get(collection).put(slice.getName(), updatedSlice);
-//				    }
-//						
-//						Slice s = new Slice(sliceName.getNodeValue(), shards);
-//	
-//						colStates.put(collectionName, s);
-					}
-				}
-			} catch (SAXException e) {
-        log.error("", e);
-        throw new ZooKeeperException(
-            SolrException.ErrorCode.SERVER_ERROR, "", e);
-			} catch (IOException e) {
-        log.error("", e);
-        throw new ZooKeeperException(
-            SolrException.ErrorCode.SERVER_ERROR, "", e);
-			} catch (ParserConfigurationException e) {
-        log.error("", e);
-        throw new ZooKeeperException(
-            SolrException.ErrorCode.SERVER_ERROR, "", e);
-			} finally {
-				// some XML parsers are broken and don't close the byte stream (but
-				// they should according to spec)
-				IOUtils.closeQuietly(is.getByteStream());
-			}
-		}
-		
-		CloudState cloudState = new CloudState(liveNodes, colStates);
-
-		return cloudState;
-	}
-
-	public static byte[] store(CloudState state)
-			throws UnsupportedEncodingException, IOException {
-		StringWriter stringWriter = new StringWriter();
-		Writer w = new BufferedWriter(stringWriter);
-		w.write("<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n");
-		w.write("<clusterstate>");
-		Map<String, Map<String, Slice>> collectionStates = state
-				.getCollectionStates();
-		for (String collectionName : collectionStates.keySet()) {
-			w.write("<collectionstate name=\"" + collectionName + "\">");
-			Map<String, Slice> collection = collectionStates
-					.get(collectionName);
-			for (String sliceName : collection.keySet()) {
-				w.write("<shard name=\"" + sliceName + "\">");
-				Slice slice = collection.get(sliceName);
-				Map<String, ZkNodeProps> shards = slice.getShards();
-				for (String shardName : shards.keySet()) {
-					w.write("<replica name=\"" + shardName + "\">");
-					ZkNodeProps props = shards.get(shardName);
-					for (String propName : props.keySet()) {
-						w.write("<str name=\"" + propName + "\">"
-								+ props.get(propName) + "</str>");
-					}
-					w.write("</replica>");
-
-				}
-				w.write("</shard>");
-			}
-			w.write("</collectionstate>");
-		}
-		w.write("</clusterstate>");
-		w.flush();
-		w.close();
-		String xml = stringWriter.toString();
-
-		return xml.getBytes("UTF-8");
-
-	}
-
-//  public void setLiveNodes(List<String> liveNodes) {
-//    Set<String> liveNodesSet = new HashSet<String>(liveNodes.size());
-//    liveNodesSet.addAll(liveNodes);
-//    this.liveNodes = liveNodesSet;
-//  }
+          }
+          out.append(JSONUtil.OBJECT_END);
+          if (--shardCount != 0) {
+            out.append(JSONUtil.VALUE_SEPARATOR);
+          }
+        }
+        out.append(JSONUtil.OBJECT_END);
+        if (--sliceCount != 0) {
+          out.append(JSONUtil.VALUE_SEPARATOR);
+        }
+
+      }
+      out.append(JSONUtil.OBJECT_END);
+      if (--collCount != 0) {
+        out.append(JSONUtil.VALUE_SEPARATOR);
+      }
+    }
+    out.append(JSONUtil.OBJECT_END);
+    return new String(out.getArray()).getBytes("utf-8");
+  }
 }

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudStateUtility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudStateUtility.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudStateUtility.java
(original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/CloudStateUtility.java
Fri Dec  2 16:46:49 2011
@@ -24,7 +24,7 @@ public class CloudStateUtility {
   }
 
   public static CloudState get(SolrZkClient zkClient, Stat stat)
-      throws KeeperException, InterruptedException {
+      throws KeeperException, InterruptedException, IOException {
     
     List<String> liveNodes = zkClient.getChildren(
         ZkStateReader.LIVE_NODES_ZKNODE, null);

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
(original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/Slice.java
Fri Dec  2 16:46:49 2011
@@ -21,7 +21,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-
 // immutable
 public class Slice {
   private final Map<String,ZkNodeProps> shards;
@@ -35,7 +34,7 @@ public class Slice {
   public Map<String,ZkNodeProps> getShards() {
     return Collections.unmodifiableMap(shards);
   }
-  
+
   public Map<String,ZkNodeProps> getShardsCopy() {
     Map<String,ZkNodeProps> shards = new HashMap<String,ZkNodeProps>();
     for (Map.Entry<String,ZkNodeProps> entry : this.shards.entrySet()) {

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
(original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkNodeProps.java
Fri Dec  2 16:46:49 2011
@@ -24,8 +24,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Map.Entry;
 
+import org.apache.noggit.JSONUtil;
+import org.apache.noggit.ObjectBuilder;
+
 // Immutable
 public class ZkNodeProps  {
+
   private final Map<String,String> propMap;
 
   public ZkNodeProps(Map<String,String> propMap) {
@@ -47,25 +51,12 @@ public class ZkNodeProps  {
   }
   
   public static ZkNodeProps load(byte[] bytes) throws IOException {
-    ZkNodeProps props = new ZkNodeProps();
-    String stringRep = new String(bytes, "UTF-8");
-    String[] lines = stringRep.split("\n");
-    for (String line : lines) {
-      int sepIndex = line.indexOf('=');
-      String key = line.substring(0, sepIndex);
-      String value = line.substring(sepIndex + 1, line.length());
-      props.propMap.put(key, value);
-    }
-    return props;
+    Map<String, String> props = (Map<String, String>) ObjectBuilder.fromJSON(new
String(bytes, "utf-8"));
+    return new ZkNodeProps(props);
   }
 
   public byte[] store() throws IOException {
-    StringBuilder sb = new StringBuilder();
-    Set<Entry<String,String>> entries = propMap.entrySet();
-    for(Entry<String,String> entry : entries) {
-      sb.append(entry.getKey() + "=" + entry.getValue() + "\n");
-    }
-    return sb.toString().getBytes("UTF-8");
+    return new String(JSONUtil.toJSON(this.propMap)).getBytes("utf-8");
   }
   
   public String get(String key) {
@@ -81,7 +72,7 @@ public class ZkNodeProps  {
     }
     return sb.toString();
   }
-
+  
   public boolean containsKey(String key) {
     return propMap.containsKey(key);
   }

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1209575&r1=1209574&r2=1209575&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
(original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
Fri Dec  2 16:46:49 2011
@@ -46,7 +46,7 @@ public class ZkStateReader {
   public static final String SHARD_ID_PROP = "shard_id";
   
   public static final String LIVE_NODES_ZKNODE = "/live_nodes";
-  public static final String CLUSTER_STATE = "/clusterstate.xml";
+  public static final String CLUSTER_STATE = "/clusterstate.json";
   
   public static final String RECOVERING = "recovering";
   public static final String ACTIVE = "active";
@@ -100,6 +100,8 @@ public class ZkStateReader {
               log.error("", e);
               throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
                   "", e);
+            } catch (IOException e) {
+              throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"",e);
             }
 
           }
@@ -119,7 +121,7 @@ public class ZkStateReader {
   }
   
   public synchronized void createClusterStateWatchersAndUpdate() throws KeeperException,
-      InterruptedException {
+      InterruptedException, IOException {
     // We need to fetch the current cluster state and the set of live nodes
     
     if (!zkClient.exists(CLUSTER_STATE)) {
@@ -169,7 +171,11 @@ public class ZkStateReader {
         } /*
            * catch(IOException e){ log.error("", e); throw new
            * ZooKeeperException( SolrException.ErrorCode.SERVER_ERROR, "", e); }
-           */
+           */ 
+        catch (IOException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Could not serialize cloud state", e);
+        } 
       }
       
     });
@@ -288,6 +294,8 @@ public class ZkStateReader {
               log.error("", e);
               throw new ZooKeeperException(
                   SolrException.ErrorCode.SERVER_ERROR, "", e);
+            } catch (IOException e) {
+              throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "",e);
             }
             // update volatile
             ZkStateReader.this.cloudState = cloudState;



Mime
View raw message