You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ka...@apache.org on 2014/12/08 04:02:52 UTC

[1/4] helix git commit: [HELIX-555] Allow ClusterStateVerifier classes to take ZkClient, master port

Repository: helix
Updated Branches:
  refs/heads/master af7782944 -> 01222c4f6


[HELIX-555] Allow ClusterStateVerifier classes to take ZkClient, master port


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/48cd91d7
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/48cd91d7
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/48cd91d7

Branch: refs/heads/master
Commit: 48cd91d76e45cbe8c6f8cbdc5659038ff50900b2
Parents: af77829
Author: Kanak Biscuitwala <ka...@hotmail.com>
Authored: Sun Nov 23 18:59:43 2014 -0800
Committer: Kanak Biscuitwala <ka...@hotmail.com>
Committed: Sun Nov 23 19:00:00 2014 -0800

----------------------------------------------------------------------
 .../helix/tools/ClusterStateVerifier.java       | 45 +++++++++++++-------
 1 file changed, 29 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/48cd91d7/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java b/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
index 65df706..5446152 100644
--- a/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
+++ b/helix-core/src/main/java/org/apache/helix/tools/ClusterStateVerifier.java
@@ -70,6 +70,7 @@ import org.apache.helix.model.builder.ResourceAssignmentBuilder;
 import org.apache.helix.util.ZKClientPool;
 import org.apache.log4j.Logger;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
 
 public class ClusterStateVerifier {
@@ -91,16 +92,14 @@ public class ClusterStateVerifier {
     private final ZkClient _zkclient;
 
     public ZkVerifier(String clusterName, ZkClient zkclient) {
-      if (zkclient == null || clusterName == null) {
-        throw new IllegalArgumentException("zkclient/clusterName can't be null");
-      }
-
+      Preconditions.checkArgument(zkclient != null && clusterName != null,
+          "zkclient/clusterName can't be null");
       _clusterName = clusterName;
       _zkclient = zkclient;
     }
 
     public ZkVerifier(String clusterName, String zkAddr) {
-      this(clusterName, ZKClientPool.getZkClient(zkAddr));
+      this(clusterName, validateAndGetClient(zkAddr, clusterName));
     }
 
     public ZkClient getZkClient() {
@@ -133,8 +132,7 @@ public class ClusterStateVerifier {
 
     @Override
     public void handleDataDeleted(String dataPath) throws Exception {
-      // TODO Auto-generated method stub
-
+      LOG.debug(String.format("Data at path %s deleted", dataPath));
     }
 
     @Override
@@ -152,6 +150,12 @@ public class ClusterStateVerifier {
 
   }
 
+  private static ZkClient validateAndGetClient(String zkAddr, String clusterName) {
+    Preconditions.checkArgument(zkAddr != null && clusterName != null,
+        "requires zkAddr and clusterName");
+    return ZKClientPool.getZkClient(zkAddr);
+  }
+
   /**
    * verifier that verifies best possible state and external view
    */
@@ -171,8 +175,13 @@ public class ClusterStateVerifier {
 
     public BestPossAndExtViewZkVerifier(String zkAddr, String clusterName,
         Map<String, Map<String, String>> errStates, Set<String> resources) {
-      super(clusterName, zkAddr);
-      this.zkAddr = zkAddr;
+      this(validateAndGetClient(zkAddr, clusterName), clusterName, errStates, resources);
+    }
+
+    public BestPossAndExtViewZkVerifier(ZkClient zkClient, String clusterName,
+        Map<String, Map<String, String>> errStates, Set<String> resources) {
+      super(clusterName, zkClient);
+      this.zkAddr = zkClient.getServers();
       this.errStates = errStates;
       this.resources = resources;
     }
@@ -181,7 +190,8 @@ public class ClusterStateVerifier {
     public boolean verify() {
       try {
         HelixDataAccessor accessor =
-            new ZKHelixDataAccessor(getClusterName(), new ZkBaseDataAccessor<ZNRecord>(getZkClient()));
+            new ZKHelixDataAccessor(getClusterName(), new ZkBaseDataAccessor<ZNRecord>(
+                getZkClient()));
 
         return ClusterStateVerifier.verifyBestPossAndExtView(accessor, errStates, getClusterName(),
             resources);
@@ -206,11 +216,16 @@ public class ClusterStateVerifier {
       super(clusterName, zkAddr);
     }
 
+    public MasterNbInExtViewVerifier(ZkClient zkClient, String clusterName) {
+      super(clusterName, zkClient);
+    }
+
     @Override
     public boolean verify() {
       try {
         ZKHelixDataAccessor accessor =
-            new ZKHelixDataAccessor(getClusterName(), new ZkBaseDataAccessor<ZNRecord>(getZkClient()));
+            new ZKHelixDataAccessor(getClusterName(), new ZkBaseDataAccessor<ZNRecord>(
+                getZkClient()));
 
         return ClusterStateVerifier.verifyMasterNbInExtView(accessor);
       } catch (Exception e) {
@@ -515,8 +530,7 @@ public class ClusterStateVerifier {
       } while (curTime <= startTime + timeout);
       return result;
     } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
+      LOG.error("Failed to verify cluster state by polling", e);
     } finally {
       long endTime = System.currentTimeMillis();
 
@@ -561,8 +575,7 @@ public class ClusterStateVerifier {
           result = verifier.verify();
         }
       } catch (Exception e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
+        LOG.error("Failed to verify cluster state by callback", e);
       }
     }
 
@@ -653,10 +666,10 @@ public class ClusterStateVerifier {
   }
 
   public static boolean verifyState(String[] args) {
-    // TODO Auto-generated method stub
     String clusterName = "storage-cluster";
     String zkServer = "localhost:2181";
     long timeoutValue = 0;
+    @SuppressWarnings("unused")
     long periodValue = 1000;
 
     Set<String> resourceSet = null;


[3/4] helix git commit: adding apache license

Posted by ka...@apache.org.
adding apache license


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/3d7626cb
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/3d7626cb
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/3d7626cb

Branch: refs/heads/master
Commit: 3d7626cbf52041789ac4d3b92179f7533de78a87
Parents: e2a033f
Author: Jon Heise <j....@gmail.com>
Authored: Thu Dec 4 23:43:46 2014 -0800
Committer: Jon Heise <j....@gmail.com>
Committed: Thu Dec 4 23:43:46 2014 -0800

----------------------------------------------------------------------
 contributors/py-helix-admin/helix/__init__.py    | 18 ++++++++++++++++++
 contributors/py-helix-admin/helix/cluster.py     | 18 ++++++++++++++++++
 contributors/py-helix-admin/helix/functions.py   | 19 +++++++++++++++++++
 .../py-helix-admin/helix/helixexceptions.py      | 19 +++++++++++++++++++
 contributors/py-helix-admin/helix/participant.py | 19 +++++++++++++++++++
 contributors/py-helix-admin/helix/partition.py   | 19 +++++++++++++++++++
 .../py-helix-admin/helix/resourcegroup.py        | 19 +++++++++++++++++++
 7 files changed, 131 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/__init__.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/__init__.py b/contributors/py-helix-admin/helix/__init__.py
index e69de29..fe95886 100644
--- a/contributors/py-helix-admin/helix/__init__.py
+++ b/contributors/py-helix-admin/helix/__init__.py
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#

http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/cluster.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/cluster.py b/contributors/py-helix-admin/helix/cluster.py
index f2b931e..7db7014 100644
--- a/contributors/py-helix-admin/helix/cluster.py
+++ b/contributors/py-helix-admin/helix/cluster.py
@@ -1,3 +1,21 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
 """base class for anything that connects to helix"""
 
 from participant import Participant

http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/functions.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/functions.py b/contributors/py-helix-admin/helix/functions.py
index b66cca7..5e36404 100644
--- a/contributors/py-helix-admin/helix/functions.py
+++ b/contributors/py-helix-admin/helix/functions.py
@@ -1,3 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
 """library to handle helix commands"""
 import json
 from restkit import Resource

http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/helixexceptions.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/helixexceptions.py b/contributors/py-helix-admin/helix/helixexceptions.py
index 63ffec1..8483991 100644
--- a/contributors/py-helix-admin/helix/helixexceptions.py
+++ b/contributors/py-helix-admin/helix/helixexceptions.py
@@ -1,3 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
 """library to handle helix exceptions"""
 
 

http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/participant.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/participant.py b/contributors/py-helix-admin/helix/participant.py
index 678f93b..6695432 100644
--- a/contributors/py-helix-admin/helix/participant.py
+++ b/contributors/py-helix-admin/helix/participant.py
@@ -1,3 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
 """base class for anything that connects to helix"""
 from helixexceptions import HelixException
 

http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/partition.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/partition.py b/contributors/py-helix-admin/helix/partition.py
index e028567..fbc0e66 100644
--- a/contributors/py-helix-admin/helix/partition.py
+++ b/contributors/py-helix-admin/helix/partition.py
@@ -1,3 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
 """base class for anything that connects to helix"""
 
 

http://git-wip-us.apache.org/repos/asf/helix/blob/3d7626cb/contributors/py-helix-admin/helix/resourcegroup.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/resourcegroup.py b/contributors/py-helix-admin/helix/resourcegroup.py
index 62c447f..feb9f09 100644
--- a/contributors/py-helix-admin/helix/resourcegroup.py
+++ b/contributors/py-helix-admin/helix/resourcegroup.py
@@ -1,3 +1,22 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
 """base class for anything that connects to helix"""
 
 import partition


[2/4] helix git commit: adding python helix admin library

Posted by ka...@apache.org.
adding python helix admin library


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/e2a033f8
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/e2a033f8
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/e2a033f8

Branch: refs/heads/master
Commit: e2a033f8563538a4cae1e73a3a7f95a302751a98
Parents: af77829
Author: Jon Heise <jh...@jheise-ld1.linkedin.biz>
Authored: Thu Dec 4 15:38:20 2014 -0800
Committer: Jon Heise <jh...@jheise-ld1.linkedin.biz>
Committed: Thu Dec 4 15:38:20 2014 -0800

----------------------------------------------------------------------
 contributors/py-helix-admin/helix/__init__.py   |   0
 contributors/py-helix-admin/helix/cluster.py    | 346 ++++++++++++
 contributors/py-helix-admin/helix/functions.py  | 543 +++++++++++++++++++
 .../py-helix-admin/helix/helixexceptions.py     |  16 +
 .../py-helix-admin/helix/participant.py         |  66 +++
 contributors/py-helix-admin/helix/partition.py  |  18 +
 .../py-helix-admin/helix/resourcegroup.py       |  34 ++
 7 files changed, 1023 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/__init__.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/__init__.py b/contributors/py-helix-admin/helix/__init__.py
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/cluster.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/cluster.py b/contributors/py-helix-admin/helix/cluster.py
new file mode 100644
index 0000000..f2b931e
--- /dev/null
+++ b/contributors/py-helix-admin/helix/cluster.py
@@ -0,0 +1,346 @@
+"""base class for anything that connects to helix"""
+
+from participant import Participant
+from partition import Partition
+from resourcegroup import ResourceGroup
+
+from helixexceptions import HelixException
+import functions
+
+
+class Cluster(object):
+    """Basic model of a cluster, holds participants, partitions, slices,
+    external view, ideal state, resource groups"""
+    ver = (1, 0)
+
+    def __init__(self, host, cluster):
+        super(Cluster, self).__init__()
+        self.host = host
+        self.cluster = cluster
+
+        # dynamically loaded data below
+        self._partitions = {}
+        self._participants = {}
+        self._resources = {}
+        self._ideal_state = {}
+        self._external_view = {}
+
+    def __str__(self):
+        return "{0} Object for {1}".format(self.__class__.__name__,
+                                           self.cluster)
+
+    def __repr__(self):
+        return "{0}({1}, {2})".format(self.__class__.__name__, self.cluster,
+                                      self.host)
+
+    def load_resources(self):
+        """queries helix for resource groups and loades them into model"""
+        try:
+            for cur_resource in functions.get_resource_groups(self.host,
+                                                              self.cluster):
+                data = functions.get_resource_group(self.host, self.cluster,
+                                                    cur_resource)
+                name = data["id"]
+                count = data["simpleFields"]["NUM_PARTITIONS"]
+                replicas = data["simpleFields"]["REPLICAS"]
+                statemode = data["simpleFields"]["STATE_MODEL_DEF_REF"]
+                resource = ResourceGroup(name,
+                                         count, replicas,
+                                         statemode, data)
+                partitions = data["mapFields"]
+                for part, hosts in partitions.items():
+                    phosts = []
+                    for host, status in hosts.items():
+                        participant = self.participants[host]
+                        participant.partitions[part] = status
+                        phosts.append(participant)
+
+                    partition = Partition(part, phosts)
+                    resource.add_partition(partition)
+
+                self._resources[cur_resource] = resource
+        except HelixException:
+            pass
+
+    @property
+    def resources(self):
+        """sanely handle resource loading and usage"""
+        if not self._resources:
+            self.load_resources()
+        return self._resources
+
+    @resources.setter
+    def resources(self, value):
+        """ensure an exception is raise on an attempt to set resource groups"""
+        raise HelixException("Resource groups cannont be added in this manner")
+
+    def _cluster_exists(self):
+        """verify cluster exists in helix"""
+        if self.cluster in functions.get_clusters(self.host):
+            return True
+        return False
+
+    def load_participants(self):
+        """create instances of storage node for participants in this cluster"""
+        self._participants = {}
+
+        try:
+            instances = functions.get_instances(self.host, self.cluster)
+            for instance in instances:
+                ident = instance["id"]
+                enabled = instance["simpleFields"]["HELIX_ENABLED"]
+                alive = instance["simpleFields"]["Alive"]
+                data = instance
+                participant = Participant(ident, alive, enabled, data)
+                self._participants[instance["id"]] = participant
+        except HelixException:
+            pass
+
+    @property
+    def participants(self):
+        """returns participants, if not loaded, loads them then returns"""
+        if not self._participants:
+            self.load_participants()
+        return self._participants
+
+    @participants.setter
+    def participants(self, value):
+        raise HelixException("Participants cannot added in this fashion!")
+
+    def load_partitions(self):
+        """query partitions from helix and load into model"""
+        self._partitions = {}
+        for resource in self.resources:
+            newstate = functions.get_ideal_state(self.host, self.cluster,
+                                                 resource)
+            self._partitions[resource] = {}
+            if newstate:
+                for part in newstate:
+                    hosts = [self.participants[x] for x in newstate[part]]
+                    partition = Partition(part, hosts)
+                    self._partitions[resource][part] = partition
+                    for host in newstate[part]:
+                        self.participants[host].partitions[part] = partition
+
+    @property
+    def partitions(self):
+        """return partitions"""
+        if not self._partitions:
+            self.load_partitions()
+        return self._partitions
+
+    def load_ideal_state(self):
+        """query ideal state from helix and load into model"""
+        self._ideal_state = {}
+        for resource in self.resources:
+            self._ideal_state[resource] = \
+                functions.get_ideal_state(self.host, self.cluster, resource)
+
+    @property
+    def ideal_state(self):
+        """return ideal state"""
+        if not self._ideal_state:
+            self.load_ideal_state()
+        return self._ideal_state
+
+    @ideal_state.setter
+    def ideal_state(self, value):
+        """setter for ideal state"""
+        raise HelixException("Cannot adjust Ideal State in this manner")
+
+    def load_external_view(self):
+        """query external view from helix and load into model"""
+        self._external_view = {}
+        for resource in self.resources:
+            self._external_view[resource] = \
+                functions.get_external_view(self.host, self.cluster, resource)
+
+    @property
+    def external_view(self):
+        """return external view"""
+        if not self._external_view:
+            self.load_external_view()
+        return self._external_view
+
+    @external_view.setter
+    def external_view(self, value):
+        """setter for external view"""
+        raise HelixException("External View cannot be modified!")
+
+    def get_config(self, config):
+        """ get requested config from helix"""
+        return functions.get_config(self.host, self.cluster, config)
+
+    def set_cluster_config(self, config):
+        """ set given configs in helix"""
+        return functions.set_config(self.host, self.cluster, config)
+
+    def set_resource_config(self, config, resource):
+        """ set given configs in helix"""
+        rname = resource
+        if isinstance(resource, ResourceGroup):
+            rname = resource.name
+        return functions.set_config(self.host, self.cluster, config,
+                                    resource=rname)
+
+    def set_participant_config(self, config, participant):
+        pname = participant
+        if isinstance(participant, Participant):
+            pname = participant.ident
+        """ set given configs in helix"""
+        return functions.set_config(self.host, self.cluster, config,
+                                    participant=pname)
+
+    def activate_cluster(self, grand, enabled=True):
+        """activate this cluster with the specified grand cluster"""
+        return functions.activate_cluster(self.host, self.cluster, grand,
+                                          enabled)
+
+    def deactivate_cluster(self, grand):
+        """deactivate this cluster against the given grandcluster"""
+        return functions.deactivate_cluster(self.host, self.cluster, grand)
+
+    def add_cluster(self):
+        """add cluster to helix"""
+        return functions.add_cluster(self.host, self.cluster)
+
+    def add_instance(self, instances, port):
+        """add instance to cluster"""
+        return functions.add_instance(self.host, self.cluster, instances, port)
+
+    def rebalance(self, resource, replicas, key=""):
+        """rebalance a resource group"""
+        return functions.rebalance(self.host, self.cluster, resource,
+                                   replicas, key)
+
+    def add_resource(self, resource, partitions, state_model_def, mode=""):
+        """add resource to cluster"""
+        return functions.add_resource(self.host, self.cluster, resource,
+                                      partitions, state_model_def, mode)
+
+    def enable_instance(self, instance, enabled=True):
+        """enable instance, assumes instance a participant object"""
+        ident = None
+        if isinstance(instance, Participant):
+            ident = instance.ident
+        elif isinstance(instance, str):
+            ident = instance
+        else:
+            raise HelixException("Instance must be a string or participant")
+        return functions.enable_instance(self.host, self.cluster, ident,
+                                         enabled)
+
+    def disable_instance(self, instance):
+        """disable instance, assumes instance is a participant object"""
+        return self.enable_instance(instance, enabled=False)
+
+    def enable_partition(self, resource, partition, instance, enabled=True):
+        """enable partition, assumes instance and partition are
+        helix objects"""
+        ident = None
+        part_id = None
+
+        if isinstance(instance, Participant):
+            ident = instance.ident
+        elif isinstance(instance, str):
+            ident = instance
+        else:
+            raise HelixException("Instance must be a string or participant")
+
+        if isinstance(partition, Partition):
+            part_id = partition.name
+        elif isinstance(partition, str):
+            part_id = partition
+        else:
+            raise HelixException("Partition must be a string or partition")
+
+        return functions.enable_partition(self.host, self.cluster, resource,
+                                          part_id, ident, enabled)
+
+    def disable_partition(self, resource, partition, instance):
+        """disable partition, conveience function for enable partition"""
+        return self.enable_partition(resource, partition, instance,
+                                     enabled=False)
+
+    def enable_resource(self, resource, enabled=True):
+        """enable/disable resource"""
+        resource_name = None
+        if isinstance(resource, ResourceGroup):
+            resource_name = resource.name
+        elif isinstance(resource, str):
+            resource_name = resource
+        else:
+            raise HelixException(
+                "Resource must be a string or a resource group object")
+
+        return functions.enable_resource(self.host, self.cluster,
+                                         resource_name, enabled)
+
+    def disable_resource(self, resource):
+        """disable given function"""
+        return self.enable_resource(resource, enabled=False)
+
+    def add_resource_tag(self, resource, tag):
+        """add a tag to a resource"""
+        resource_name = None
+        if isinstance(resource, ResourceGroup):
+            resource_name = resource.name
+        elif isinstance(resource, str):
+            resource_name = resource
+        else:
+            raise HelixException("Resource must be resource object or string")
+
+        return functions.add_resource_tag(self.host, self.cluster,
+                                          resource_name, tag)
+
+    # del resource not yet available in api
+    # def del_resource_tag(self, resource, tag):
+    # """del a tag to a resource"""
+    #     resource_name = None
+    #     if isinstance(resource, ResourceGroup):
+    #         resource_name = resource.name
+    #     elif isinstance(resource, str):
+    #         resource_name = resource
+    #     else:
+    #         raise HelixException("Resource must be resource object or str")
+    #
+    #     return functions.del_resource_tag(self.host, self.cluster,
+    #                                       resource_name, tag)
+
+    def add_instance_tag(self, instance, tag):
+        ident = None
+
+        if isinstance(instance, Participant):
+            ident = instance.ident
+        elif isinstance(instance, str):
+            ident = instance
+        else:
+            raise HelixException("Instance must be a string or participant")
+
+        return functions.add_instance_tag(self.host, self.cluster, ident, tag)
+
+    def del_instance_tag(self, instance, tag):
+        ident = None
+
+        if isinstance(instance, Participant):
+            ident = instance.ident
+        elif isinstance(instance, str):
+            ident = instance
+        else:
+            raise HelixException("Instance must be a string or participant")
+
+        return functions.del_instance_tag(self.host, self.cluster, ident, tag)
+
+    def del_instance(self, instance):
+        """remove instance from cluster, assumes instance is a
+        participant object"""
+        return functions.del_instance(self.host, self.cluster, instance.ident)
+
+    def del_resource(self, resource):
+        """remove resource group from cluster, assumes resource is a
+        resource object"""
+        return functions.del_resource(self.host, self.cluster, resource.name)
+
+    def del_cluster(self):
+        """remove cluster from helix"""
+        return functions.del_cluster(self.host, self.cluster)

http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/functions.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/functions.py b/contributors/py-helix-admin/helix/functions.py
new file mode 100644
index 0000000..b66cca7
--- /dev/null
+++ b/contributors/py-helix-admin/helix/functions.py
@@ -0,0 +1,543 @@
+"""library to handle helix commands"""
+import json
+from restkit import Resource
+
+from helixexceptions import HelixException
+from helixexceptions import HelixAlreadyExistsException
+from helixexceptions import HelixDoesNotExistException
+
+
+def _post_payload(host, path, data, **kwargs):
+    """generic function to handle posting data
+    :rtype : return body of page
+    :param host: host to send data to
+    :param path: path to interact with
+    :param data: data to send
+    :param kwargs:  additional keyword args
+    """
+
+    if "http://" not in host:
+        host = "http://{0}".format(host)
+
+    res = Resource(host)
+
+    payload = "jsonParameters={0}".format(json.dumps(data))
+    for key, value in kwargs.items():
+        payload += '&{0}={1}'.format(key, json.dumps(value))
+    headers = {"Content-Type": "application/json"}
+    # print "path is %s" % path
+    page = res.post(path=path, payload=payload, headers=headers)
+    body = page.body_string()
+    if body:
+        body = json.loads(body)
+
+        if isinstance(body, dict) and "ERROR" in body:
+            raise HelixException(body["ERROR"])
+
+    # test what was returned, see if any exceptions need to be raise
+    # if not body:
+    # raise HelixException("body for path {0} is empty".format(path))
+    # else:
+    # print "BODY IS EMPTY FOR ", path
+    # print "BODY is %s." % body
+
+    return body
+
+
+def _get_page(host, path):
+    """if we're specifying a cluster then verify that a cluster is set"""
+
+    if "http://" not in host:
+        host = "http://{0}".format(host)
+
+    res = Resource(host)
+
+    page = res.get(path=path)
+    data = page.body_string()
+    body = None
+    try:
+        body = json.loads(data)
+    except ValueError:
+        body = json.loads(data[:-3])
+
+    # test what was returned, see if any exceptions need to be raise
+    if not body:
+        raise HelixException("body for path {0} is empty".format(path))
+
+    if isinstance(body, dict) and "ERROR" in body:
+        raise HelixException(body["ERROR"])
+
+    return body
+
+
+def _delete_page(host, path):
+    """delete page at a given path"""
+    retval = None
+    if "http://" not in host:
+        host = "http://{0}".format(host)
+
+    res = Resource(host)
+
+    page = res.delete(path)
+    data = page.body_string()
+    if data:
+        retval = json.loads(data)
+
+    return retval
+
+
+def get_clusters(host):
+    """ querys helix cluster for all clusters """
+    return _get_page(host, "/clusters")["listFields"]["clusters"]
+
+
+def get_resource_groups(host, cluster):
+    """ querys helix cluster for resources groups of the current cluster"""
+    return _get_page(host, "/clusters/{0}/resourceGroups".format(cluster))[
+        "listFields"]["ResourceGroups"]
+
+
+def get_resource_tags(host, cluster):
+    """returns a dict of resource tags for a cluster"""
+    return _get_page(host, "/clusters/{0}/resourceGroups".format(cluster))[
+        "mapFields"]["ResourceTags"]
+
+
+def get_resource_group(host, cluster, resource):
+    """ gets the ideal state of the specified resource group of the
+    current cluster"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixException(
+            "{0} is not a resource group of {1}".format(resource, cluster))
+
+    return _get_page(host, "/clusters/{0}/resourceGroups/{1}".format(cluster,
+                                                                     resource))
+
+
+def get_ideal_state(host, cluster, resource):
+    """ gets the ideal state of the specified resource group of the
+    current cluster"""
+
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixException(
+            "{0} is not a resource group of {1}".format(resource, cluster))
+
+    return _get_page(host, "/clusters/{0}/resourceGroups/{1}/idealState".
+                     format(cluster, resource))["mapFields"]
+
+
+def get_external_view(host, cluster, resource):
+    """return the external view for a given cluster and resource"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixException(
+            "{0} is not a resource group of {1}".format(resource, cluster))
+
+    return _get_page(host,
+                     "/clusters/{0}/resourceGroups/{1}/externalView".format(
+                         cluster, resource))["mapFields"]
+
+
+def get_instances(host, cluster):
+    """get list of instances registered to the cluster"""
+    if not cluster:
+        raise HelixException("Cluster must be set before "
+                             "calling this function")
+
+    return _get_page(host, "/clusters/{0}/instances".format(cluster))[
+        "instanceInfo"]
+
+
+def get_instance_detail(host, cluster, name):
+    """get details of an instance"""
+    return _get_page(host, "/clusters/{0}/instances/{1}".format(cluster, name))
+
+
+def get_config(host, cluster, config):
+    """get requested config"""
+    return _get_page(host, "/clusters/{0}/configs/{1}".format(cluster, config))
+
+
+def add_cluster(host, cluster):
+    """add a cluster to helix"""
+    if cluster in get_clusters(host):
+        raise HelixAlreadyExistsException(
+            "Cluster {0} already exists".format(cluster))
+
+    data = {"command": "addCluster",
+            "clusterName": cluster}
+
+    page = _post_payload(host, "/clusters", data)
+    return page
+
+
+def add_instance(host, cluster, instances, port):
+    """add a list of instances to a cluster"""
+    if cluster not in get_clusters(host):
+        raise HelixDoesNotExistException(
+            "Cluster {0} does not exist".format(cluster))
+
+    if not isinstance(instances, list):
+        instances = [instances]
+    instances = ["{0}:{1}".format(instance, port) for instance in instances]
+    try:
+        newinstances = set(instances)
+        oldinstances = set(
+            [x["id"].replace('_', ':') for x in get_instances(host, cluster)])
+        instances = list(newinstances - oldinstances)
+    except HelixException:
+        # this will get thrown if instances is empty,
+        # which if we're just populating should happen
+        pass
+
+    if instances:
+        data = {"command": "addInstance",
+                "instanceNames": ";".join(instances)}
+
+        instance_path = "/clusters/{0}/instances".format(cluster)
+        # print "adding to", instance_path
+        page = _post_payload(host, instance_path, data)
+        return page
+
+    else:
+        raise HelixAlreadyExistsException(
+            "All instances given already exist in cluster")
+
+
+def rebalance(host, cluster, resource, replicas, key=""):
+    """rebalance the given resource group"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixException(
+            "{0} is not a resource group of {1}".format(resource, cluster))
+
+    data = {"command": "rebalance",
+            "replicas": replicas}
+
+    if key:
+        data["key"] = key
+    page = _post_payload(host,
+                         "/clusters/{0}/resourceGroups/{1}/idealState".format(
+                             cluster, resource), data)
+    return page
+
+
+def activate_cluster(host, cluster, grand_cluster, enabled=True):
+    """activate the cluster with the grand cluster"""
+    if grand_cluster not in get_clusters(host):
+        raise HelixException(
+            "grand cluster {0} does not exist".format(grand_cluster))
+
+    data = {'command': 'activateCluster',
+            'grandCluster': grand_cluster}
+
+    if enabled:
+        data["enabled"] = "true"
+    else:
+        data["enabled"] = "false"
+
+    page = _post_payload(host, "/clusters/{0}".format(cluster), data)
+    return page
+
+
+def deactivate_cluster(host, cluster, grand_cluster):
+    """deactivate the cluster with the grand cluster"""
+    return activate_cluster(host, cluster, grand_cluster, enabled=False)
+
+
+def add_resource(host, cluster, resource, partitions,
+                 state_model_def, mode=""):
+    """Add given resource group"""
+    if resource in get_resource_groups(host, cluster):
+        raise HelixAlreadyExistsException(
+            "ResourceGroup {0} already exists".format(resource))
+
+    data = {"command": "addResource",
+            "resourceGroupName": resource,
+            "partitions": partitions,
+            "stateModelDefRef": state_model_def}
+
+    if mode:
+        data["mode"] = mode
+
+    return _post_payload(host, "/clusters/{0}/resourceGroups".format(cluster),
+                         data)
+
+
+def enable_resource(host, cluster, resource, enabled=True):
+    """enable or disable specified resource"""
+    data = {"command": "enableResource"}
+    if enabled:
+        data["enabled"] = "true"
+    else:
+        data["enabled"] = "false"
+
+    return _post_payload(host, "/clusters/{0}/resourceGroups/{1}".format(
+        cluster, resource), data)
+
+
+def disable_resource(host, cluster, resource):
+    """function for disabling resources"""
+    return enable_resource(host, cluster, resource, enabled=False)
+
+
+def alter_ideal_state(host, cluster, resource, newstate):
+    """alter ideal state"""
+    data = {"command": "alterIdealState"}
+    return _post_payload(host,
+                         "/clusters/{0}/resourceGroups/{1}/idealState".format(
+                             cluster, resource), data,
+                         newIdealState=newstate)
+
+
+def enable_instance(host, cluster, instance, enabled=True):
+    """enable instance within cluster"""
+    data = {"command": "enableInstance"}
+    if enabled:
+        data["enabled"] = "true"
+    else:
+        data["enabled"] = "false"
+
+    return _post_payload(host, "/clusters/{0}/instances/{1}".format(cluster,
+                                                                    instance),
+                         data)
+
+
+def disable_instance(host, cluster, instance):
+    """wrapper for ease of use for disabling an instance"""
+    return enable_instance(host, cluster, instance, enabled=False)
+
+
+def swap_instance(host, cluster, old, new):
+    """swap instance"""
+    data = {"command": "swapInstance",
+            "oldInstance": old,
+            "newInstance": new}
+
+    return _post_payload(host, "/cluster/{0}/instances".format(cluster), data)
+
+
+def enable_partition(host, cluster, resource, partition, instance,
+                     enabled=True):
+    """enable Partition """
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixDoesNotExistException(
+            "ResourceGroup {0} does not exist".format(resource))
+
+    data = {"command": "enablePartition",
+            "resource": resource,
+            "partition": partition,
+            "enabled": enabled}
+    return _post_payload(host, "/clusters/{0}/instances/{1}".format(cluster,
+                                                                    instance),
+                         data)
+
+
+def disable_partition(host, cluster, resource, partitions, instance):
+    """disable Partition """
+    return enable_partition(host, cluster, resource, partitions, instance,
+                            enabled=False)
+
+
+def reset_partition(host, cluster, resource, partitions, instance):
+    """reset partition"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixDoesNotExistException(
+            "ResourceGroup {0} does not exist".format(resource))
+
+    data = {"command": "resetPartition",
+            "resource": resource,
+            "partition": " ".join(partitions)}
+    return _post_payload(host, "/clusters/{0}/instances/{1}".format(cluster,
+                                                                    instance),
+                         data)
+
+
+def reset_resource(host, cluster, resource):
+    """reset resource"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixDoesNotExistException(
+            "ResourceGroup {0} does not exist".format(resource))
+
+    data = {"command": "resetResource"}
+    return _post_payload(host,
+                         "/clusters/{0}/resourceGroups/{1}".format(cluster,
+                                                                   resource),
+                         data)
+
+
+def reset_instance(host, cluster, instance):
+    """reset instance"""
+    if instance not in get_instances(host, cluster):
+        raise HelixDoesNotExistException(
+            "Instance {0} does not exist".format(instance))
+
+    data = {"command": "resetInstance"}
+    return _post_payload(host, "/clusters/{0}/instances/{1}".format(cluster,
+                                                                    instance),
+                         data)
+
+
+def add_instance_tag(host, cluster, instance, tag):
+    """add tag to an instance"""
+    data = {"command": "addInstanceTag",
+            "instanceGroupTag": tag}
+    return _post_payload(host,
+                         "/clusters/{0}/instances/{1}".format(
+                             cluster, instance), data)
+
+
+def del_instance_tag(host, cluster, instance, tag):
+    """remove tag from instance"""
+    data = {"command": "removeInstanceTag",
+            "instanceGroupTag": tag}
+    return _post_payload(host,
+                         "/clusters/{0}/instances/{1}".format(
+                             cluster, instance), data)
+
+
+def add_resource_tag(host, cluster, resource, tag):
+    """add tag to resource group"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixDoesNotExistException(
+            "ResourceGroup {0} does not exist".format(resource))
+
+    data = {"command": "addResourceProperty",
+            "INSTANCE_GROUP_TAG": tag}
+    return _post_payload(host,
+                         "/clusters/{0}/resourceGroups/{1}/idealState".format(
+                             cluster, resource), data)
+
+
+"""
+del resource currently does not exist in helix api
+def del_resource_tag(host, cluster, resource, tag):
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixDoesNotExistException(
+            "ResourceGroup {0} does not exist".format(resource))
+
+    data = {"command": "removeResourceProperty",
+            "INSTANCE_GROUP_TAG": tag}
+    return _post_payload(host,
+                         "/clusters/{0}/resourceGroups/{1}/idealState".format(
+                             cluster, resource), data)
+"""
+
+
+def get_instance_taginfo(host, cluster):
+    return _get_page(host, "/clusters/{0}/instances".format(
+        cluster))["tagInfo"]
+
+
+def expand_cluster(host, cluster):
+    """expand cluster"""
+    data = {"command": "expandCluster"}
+
+    return _post_payload(host, "/clusters/{0}/".format(cluster), data)
+
+
+def expand_resource(host, cluster, resource):
+    """expand resource"""
+    data = {"command": "expandResource"}
+
+    return _post_payload(host,
+                         "/clusters/{0}/resourceGroup/{1}/idealState".format(
+                             cluster, resource), data)
+
+
+def add_resource_property(host, cluster, resource, properties):
+    """add resource property properties must be a dictionary of properties"""
+    properties["command"] = "addResourceProperty"
+
+    return _post_payload(host,
+                         "/clusters/{0}/resourceGroup/{1}/idealState".format(
+                             cluster, resource), properties)
+
+
+def _handle_config(host, cluster, configs, command, participant=None,
+                   resource=None):
+    """helper function to set or delete configs in helix"""
+    data = {"command": "{0}Config".format(command),
+            "configs": ",".join(
+                ["{0}={1}".format(x, y) for x, y in configs.items()])}
+
+    address = "/clusters/{0}/configs/".format(cluster)
+    if participant:
+        address += "participant/{0}".format(participant)
+    elif resource:
+        address += "resource/{0}".format(resource)
+    else:
+        address += "cluster"
+
+    return _post_payload(host, address, data)
+
+
+def set_config(host, cluster, configs, participant=None, resource=None):
+    """sets config in helix"""
+    return _handle_config(host, cluster, configs, "set", participant, resource)
+
+
+def remove_config(host, cluster, configs, participant=None, resource=None):
+    """sets config in helix"""
+    return _handle_config(host, "remove", cluster, configs, participant,
+                          resource)
+
+
+def get_zk_path(host, path):
+    """get zookeeper path"""
+    return _get_page(host, "zkPath/{0}".format(path))
+
+
+def del_zk_path(host, path):
+    """delete zookeeper path"""
+    return _delete_page(host, "zkPath/{0}".format(path))
+
+
+def get_zk_child(host, path):
+    """get zookeeper child"""
+    return _get_page(host, "zkChild/{0}".format(path))
+
+
+def del_zk_child(host, path):
+    """delete zookeeper child"""
+    return _delete_page(host, "zkChild/{0}".format(path))
+
+
+def add_state_model(host, cluster, newstate):
+    """add state model"""
+    data = {"command": "addStateModel"}
+
+    return _post_payload(host, "/clusters/{0}/StateModelDefs".format(cluster),
+                         data, newStateModelDef=newstate)
+
+
+def del_instance(host, cluster, instance):
+    """delete instance"""
+    if instance not in [x["id"] for x in get_instances(host, cluster)]:
+        raise HelixDoesNotExistException(
+            "Instance {0} does not exist.".format(instance))
+
+    page = _delete_page(host,
+                        "/clusters/{0}/instances/{1}".format(cluster,
+                                                             instance))
+    return page
+
+
+def del_resource(host, cluster, resource):
+    """delete specified resource from cluster"""
+    if resource not in get_resource_groups(host, cluster):
+        raise HelixDoesNotExistException(
+            "ResourceGroup {0} does not exist".format(resource))
+
+    page = _delete_page(host, "/clusters/{0}/resourceGroups/{1}".format(
+        cluster, resource))
+    return page
+
+
+def del_cluster(host, cluster):
+    """delete cluster"""
+    page = _delete_page(host, "/clusters/{0}".format(cluster))
+
+    return page
+
+
+def send_message(host, cluster, path, **kwargs):
+    pass

http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/helixexceptions.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/helixexceptions.py b/contributors/py-helix-admin/helix/helixexceptions.py
new file mode 100644
index 0000000..63ffec1
--- /dev/null
+++ b/contributors/py-helix-admin/helix/helixexceptions.py
@@ -0,0 +1,16 @@
+"""library to handle helix exceptions"""
+
+
+class HelixException(Exception):
+    """Base helix exception"""
+    pass
+
+
+class HelixAlreadyExistsException(HelixException):
+    """Exception is thrown when an entry in helix already exists"""
+    pass
+
+
+class HelixDoesNotExistException(HelixException):
+    """Exception is thrown when an entry in helix does not exist"""
+    pass

http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/participant.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/participant.py b/contributors/py-helix-admin/helix/participant.py
new file mode 100644
index 0000000..678f93b
--- /dev/null
+++ b/contributors/py-helix-admin/helix/participant.py
@@ -0,0 +1,66 @@
+"""base class for anything that connects to helix"""
+from helixexceptions import HelixException
+
+
+class Participant(object):
+    """Basic model for a helix participant"""
+
+    def __init__(self, ident, alive, enabled, data):
+        super(Participant, self).__init__()
+        self.ident = ident
+        self.hostname, self.port = ident.split("_")
+        self.partitions = {}
+        self.data = data
+        self.enabled = None
+        self._tags = []
+        self._disabled_partitions = []
+
+        if isinstance(enabled, str) or isinstance(enabled, unicode):
+            if enabled == "true":
+                self.enabled = True
+            else:
+                self.enabled = False
+        elif isinstance(enabled, bool):
+            self.enabled = enabled
+
+        self.alive = bool(alive)
+        self.update()
+
+    def __repr__(self):
+        return "{0}('{1}', {2}, {3}, {4})".format(self.__class__.__name__,
+                                                  self.ident, self.alive,
+                                                  self.enabled, self.data)
+
+    def __str__(self):
+        return "Id: {0} Enabled: {1} Alive: {2}".format(self.ident,
+                                                        self.enabled,
+                                                        self.alive)
+
+    def update(self, data=None):
+        """update data for participant then update values"""
+        if data:
+            self.data = data
+
+        if "TAG_LIST" in self.data["listFields"]:
+            self._tags = self.data["listFields"]["TAG_LIST"]
+
+        if "HELIX_DISABLED_PARTITION" in self.data["listFields"]:
+            self._disabled_partitions = \
+                self.data["listFields"]["HELIX_DISABLED_PARTITION"]
+
+    @property
+    def tags(self):
+        return self._tags
+
+    @tags.setter
+    def tags(self, value):
+        """ensure an exception is raise on an attempt to set tags this way"""
+        raise HelixException("Tags must be set on a cluster object")
+
+    @property
+    def disabled_partitions(self):
+        return self._disabled_partitions
+
+    @disabled_partitions.setter
+    def disabled_partitions(self, value):
+        raise HelixException("Partitions must be disabled on a cluster object")

http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/partition.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/partition.py b/contributors/py-helix-admin/helix/partition.py
new file mode 100644
index 0000000..e028567
--- /dev/null
+++ b/contributors/py-helix-admin/helix/partition.py
@@ -0,0 +1,18 @@
+"""base class for anything that connects to helix"""
+
+
+class Partition(object):
+    """Object to deal helix partitions"""
+
+    def __init__(self, name, hosts):
+        super(Partition, self).__init__()
+        self.name = name
+        self.hosts = hosts
+
+    def __str__(self):
+        return "Partition {0} - Hosts: {1}".format(self.name, ", ".join(
+            [x.ident for x in self.hosts]))
+
+    def __repr__(self):
+        return "{0}('{1}', {2})".format(self.__class__.__name__, self.name,
+                                        self.hosts)

http://git-wip-us.apache.org/repos/asf/helix/blob/e2a033f8/contributors/py-helix-admin/helix/resourcegroup.py
----------------------------------------------------------------------
diff --git a/contributors/py-helix-admin/helix/resourcegroup.py b/contributors/py-helix-admin/helix/resourcegroup.py
new file mode 100644
index 0000000..62c447f
--- /dev/null
+++ b/contributors/py-helix-admin/helix/resourcegroup.py
@@ -0,0 +1,34 @@
+"""base class for anything that connects to helix"""
+
+import partition
+from helixexceptions import HelixException
+
+
+class ResourceGroup(object):
+    """Object to deal with resource groups"""
+
+    def __init__(self, name, count, replicas, statemode, data):
+        super(ResourceGroup, self).__init__()
+        self.name = name
+        self.count = count
+        self.replicas = replicas
+        self.state_model_def_ref = statemode
+        self.data = data
+        self.partitions = {}
+
+    def __str__(self):
+        return "Resource: {0} - Count: {1}".format(self.name, self.count)
+
+    def __repr__(self):
+        return "{0}('{1}', {2}, {3}, {4}, {5})".format(self.__class__.__name__,
+                                                       self.name, self.count,
+                                                       self.replicas,
+                                                       self.
+                                                       state_model_def_ref,
+                                                       self.data)
+
+    def add_partition(self, part):
+        """add a partition to this resource group"""
+        if not isinstance(part, partition.Partition):
+            raise HelixException("Argument part must be Partition or subclass")
+        self.partitions[part.name] = part


[4/4] helix git commit: Merge branch 'master' of https://github.com/jheise/helix

Posted by ka...@apache.org.
Merge branch 'master' of https://github.com/jheise/helix


Project: http://git-wip-us.apache.org/repos/asf/helix/repo
Commit: http://git-wip-us.apache.org/repos/asf/helix/commit/01222c4f
Tree: http://git-wip-us.apache.org/repos/asf/helix/tree/01222c4f
Diff: http://git-wip-us.apache.org/repos/asf/helix/diff/01222c4f

Branch: refs/heads/master
Commit: 01222c4f6289722a5bc230095138a6475fa71852
Parents: 48cd91d 3d7626c
Author: Kanak Biscuitwala <ka...@hotmail.com>
Authored: Sun Dec 7 18:59:33 2014 -0800
Committer: Kanak Biscuitwala <ka...@hotmail.com>
Committed: Sun Dec 7 18:59:33 2014 -0800

----------------------------------------------------------------------
 contributors/py-helix-admin/helix/__init__.py   |  18 +
 contributors/py-helix-admin/helix/cluster.py    | 364 ++++++++++++
 contributors/py-helix-admin/helix/functions.py  | 562 +++++++++++++++++++
 .../py-helix-admin/helix/helixexceptions.py     |  35 ++
 .../py-helix-admin/helix/participant.py         |  85 +++
 contributors/py-helix-admin/helix/partition.py  |  37 ++
 .../py-helix-admin/helix/resourcegroup.py       |  53 ++
 7 files changed, 1154 insertions(+)
----------------------------------------------------------------------