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 2013/10/19 00:27:16 UTC

[1/2] git commit: [HELIX-273] Rebalancer interface should remain unchanged in 0.6.2

Updated Branches:
  refs/heads/helix-0.6.2-release 0c3796b30 -> 7db3d060e


[HELIX-273] Rebalancer interface should remain unchanged in 0.6.2


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

Branch: refs/heads/helix-0.6.2-release
Commit: 40bb97a558a930586a8d63a4ed8b1692568f6431
Parents: a48b89e
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Fri Oct 18 14:19:27 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Oct 18 14:28:50 2013 -0700

----------------------------------------------------------------------
 .../controller/rebalancer/AutoRebalancer.java   |  9 +--
 .../controller/rebalancer/CustomRebalancer.java |  9 +--
 .../helix/controller/rebalancer/Rebalancer.java | 14 -----
 .../rebalancer/SemiAutoRebalancer.java          |  9 +--
 .../rebalancer/internal/MappingCalculator.java  | 46 ++++++++++++++++
 .../stages/BestPossibleStateCalcStage.java      | 26 ++++++---
 .../controller/stages/ResourceAssignment.java   | 58 ++++++++++++++++++++
 .../controller/stages/ResourceMapping.java      | 58 --------------------
 .../TestCustomizedIdealStateRebalancer.java     | 10 ++--
 9 files changed, 143 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
index 4dd5ea6..508b7ac 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/AutoRebalancer.java
@@ -32,10 +32,11 @@ import java.util.TreeMap;
 
 import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
-import org.apache.helix.controller.stages.ResourceMapping;
+import org.apache.helix.controller.stages.ResourceAssignment;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.DefaultPlacementScheme;
 import org.apache.helix.controller.strategy.AutoRebalanceStrategy.ReplicaPlacementScheme;
@@ -58,7 +59,7 @@ import org.apache.log4j.Logger;
  * The output is a preference list and a mapping based on that preference list, i.e. partition p
  * has a replica on node k with state s.
  */
-public class AutoRebalancer implements Rebalancer {
+public class AutoRebalancer implements Rebalancer, MappingCalculator {
   // These should be final, but are initialized in init rather than a constructor
   private HelixManager _manager;
   private AutoRebalanceStrategy _algorithm;
@@ -188,7 +189,7 @@ public class AutoRebalancer implements Rebalancer {
   }
 
   @Override
-  public ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache,
+  public ResourceAssignment computeBestPossiblePartitionState(ClusterDataCache cache,
       IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Processing resource:" + resource.getResourceName());
@@ -196,7 +197,7 @@ public class AutoRebalancer implements Rebalancer {
     String stateModelDefName = idealState.getStateModelDefRef();
     StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
     calculateAutoBalancedIdealState(cache, idealState, stateModelDef);
-    ResourceMapping partitionMapping = new ResourceMapping();
+    ResourceAssignment partitionMapping = new ResourceAssignment();
     for (Partition partition : resource.getPartitions()) {
       Map<String, String> currentStateMap =
           currentStateOutput.getCurrentStateMap(resource.getResourceName(), partition);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
index 17dc5c8..69037d9 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/CustomRebalancer.java
@@ -25,9 +25,10 @@ import java.util.Set;
 
 import org.apache.helix.HelixDefinedState;
 import org.apache.helix.HelixManager;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
-import org.apache.helix.controller.stages.ResourceMapping;
+import org.apache.helix.controller.stages.ResourceAssignment;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.LiveInstance;
 import org.apache.helix.model.Partition;
@@ -44,7 +45,7 @@ import org.apache.log4j.Logger;
  * The output is a verified mapping based on that preference list, i.e. partition p has a replica
  * on node k with state s, where s may be a dropped or error state if necessary.
  */
-public class CustomRebalancer implements Rebalancer {
+public class CustomRebalancer implements Rebalancer, MappingCalculator {
 
   private static final Logger LOG = Logger.getLogger(CustomRebalancer.class);
 
@@ -59,14 +60,14 @@ public class CustomRebalancer implements Rebalancer {
   }
 
   @Override
-  public ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache,
+  public ResourceAssignment computeBestPossiblePartitionState(ClusterDataCache cache,
       IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) {
     String stateModelDefName = idealState.getStateModelDefRef();
     StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Processing resource:" + resource.getResourceName());
     }
-    ResourceMapping partitionMapping = new ResourceMapping();
+    ResourceAssignment partitionMapping = new ResourceAssignment();
     for (Partition partition : resource.getPartitions()) {
       Map<String, String> currentStateMap =
           currentStateOutput.getCurrentStateMap(resource.getResourceName(), partition);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
index a0cfbb7..f5a4ae8 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/Rebalancer.java
@@ -22,9 +22,7 @@ package org.apache.helix.controller.rebalancer;
 import org.apache.helix.HelixManager;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
-import org.apache.helix.controller.stages.ResourceMapping;
 import org.apache.helix.model.IdealState;
-import org.apache.helix.model.Resource;
 
 /**
  * Allows one to come up with custom implementation of a rebalancer.<br/>
@@ -49,16 +47,4 @@ public interface Rebalancer {
   IdealState computeNewIdealState(String resourceName, IdealState currentIdealState,
       final CurrentStateOutput currentStateOutput, final ClusterDataCache clusterData);
 
-  /**
-   * Given an ideal state for a resource and the liveness of instances, compute the best possible
-   * state assignment for each partition's replicas.
-   * @param cache
-   * @param idealState
-   * @param resource
-   * @param currentStateOutput
-   *          Provides the current state and pending state transitions for all partitions
-   * @return
-   */
-  ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache, IdealState idealState,
-      Resource resource, CurrentStateOutput currentStateOutput);
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
index bc682ff..420e7ab 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/SemiAutoRebalancer.java
@@ -24,10 +24,11 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.helix.HelixManager;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.controller.rebalancer.util.ConstraintBasedAssignment;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
-import org.apache.helix.controller.stages.ResourceMapping;
+import org.apache.helix.controller.stages.ResourceAssignment;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
@@ -43,7 +44,7 @@ import org.apache.log4j.Logger;
  * The output is a mapping based on that preference list, i.e. partition p has a replica on node k
  * with state s.
  */
-public class SemiAutoRebalancer implements Rebalancer {
+public class SemiAutoRebalancer implements Rebalancer, MappingCalculator {
 
   private static final Logger LOG = Logger.getLogger(SemiAutoRebalancer.class);
 
@@ -58,14 +59,14 @@ public class SemiAutoRebalancer implements Rebalancer {
   }
 
   @Override
-  public ResourceMapping computeBestPossiblePartitionState(ClusterDataCache cache,
+  public ResourceAssignment computeBestPossiblePartitionState(ClusterDataCache cache,
       IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput) {
     String stateModelDefName = idealState.getStateModelDefRef();
     StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
     if (LOG.isDebugEnabled()) {
       LOG.debug("Processing resource:" + resource.getResourceName());
     }
-    ResourceMapping partitionMapping = new ResourceMapping();
+    ResourceAssignment partitionMapping = new ResourceAssignment();
     for (Partition partition : resource.getPartitions()) {
       Map<String, String> currentStateMap =
           currentStateOutput.getCurrentStateMap(resource.getResourceName(), partition);

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/rebalancer/internal/MappingCalculator.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/rebalancer/internal/MappingCalculator.java b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/internal/MappingCalculator.java
new file mode 100644
index 0000000..731ed0b
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/rebalancer/internal/MappingCalculator.java
@@ -0,0 +1,46 @@
+package org.apache.helix.controller.rebalancer.internal;
+
+/*
+ * 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.
+ */
+
+import org.apache.helix.controller.stages.ClusterDataCache;
+import org.apache.helix.controller.stages.CurrentStateOutput;
+import org.apache.helix.controller.stages.ResourceAssignment;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Resource;
+
+/**
+ * Extends Rebalancer functionality by converting an IdealState to a ResourceAssignment.<br/>
+ * <br/>
+ * WARNING: this is an internal interface and is subject to change across releases
+ */
+public interface MappingCalculator {
+  /**
+   * Given an ideal state for a resource and the liveness of instances, compute the best possible
+   * state assignment for each partition's replicas.
+   * @param cache
+   * @param idealState
+   * @param resource
+   * @param currentStateOutput
+   *          Provides the current state and pending state transitions for all partitions
+   * @return
+   */
+  public ResourceAssignment computeBestPossiblePartitionState(ClusterDataCache cache,
+      IdealState idealState, Resource resource, CurrentStateOutput currentStateOutput);
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
index 598c318..cc171a0 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
@@ -27,10 +27,11 @@ import org.apache.helix.controller.rebalancer.AutoRebalancer;
 import org.apache.helix.controller.rebalancer.CustomRebalancer;
 import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
+import org.apache.helix.controller.rebalancer.internal.MappingCalculator;
 import org.apache.helix.model.IdealState;
+import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
-import org.apache.helix.model.IdealState.RebalanceMode;
 import org.apache.helix.util.HelixUtil;
 import org.apache.log4j.Logger;
 
@@ -90,6 +91,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       }
 
       Rebalancer rebalancer = null;
+      MappingCalculator mappingCalculator = null;
       if (idealState.getRebalanceMode() == RebalanceMode.USER_DEFINED
           && idealState.getRebalancerClassName() != null) {
         String rebalancerClassName = idealState.getRebalancerClassName();
@@ -98,6 +100,7 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
         try {
           rebalancer =
               (Rebalancer) (HelixUtil.loadClass(getClass(), rebalancerClassName).newInstance());
+          mappingCalculator = new SemiAutoRebalancer();
         } catch (Exception e) {
           logger.warn("Exception while invoking custom rebalancer class:" + rebalancerClassName, e);
         }
@@ -105,19 +108,28 @@ public class BestPossibleStateCalcStage extends AbstractBaseStage {
       if (rebalancer == null) {
         if (idealState.getRebalanceMode() == RebalanceMode.FULL_AUTO) {
           rebalancer = new AutoRebalancer();
+          mappingCalculator = new AutoRebalancer();
         } else if (idealState.getRebalanceMode() == RebalanceMode.SEMI_AUTO) {
           rebalancer = new SemiAutoRebalancer();
+          mappingCalculator = new SemiAutoRebalancer();
         } else {
           rebalancer = new CustomRebalancer();
+          mappingCalculator = new CustomRebalancer();
         }
+        idealState =
+            rebalancer.computeNewIdealState(resourceName, idealState, currentStateOutput, cache);
       }
 
-      ResourceMapping partitionStateAssignment =
-          rebalancer.computeBestPossiblePartitionState(cache, idealState, resource,
-              currentStateOutput);
-      for (Partition partition : resource.getPartitions()) {
-        Map<String, String> newStateMap = partitionStateAssignment.getInstanceStateMap(partition);
-        output.setState(resourceName, partition, newStateMap);
+      // Use the internal MappingCalculator interface to compute the final assignment
+      // The next release will support rebalancers that compute the mapping from start to finish
+      if (mappingCalculator != null) {
+        ResourceAssignment partitionStateAssignment =
+            mappingCalculator.computeBestPossiblePartitionState(cache, idealState, resource,
+                currentStateOutput);
+        for (Partition partition : resource.getPartitions()) {
+          Map<String, String> newStateMap = partitionStateAssignment.getInstanceStateMap(partition);
+          output.setState(resourceName, partition, newStateMap);
+        }
       }
     }
     return output;

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceAssignment.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceAssignment.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceAssignment.java
new file mode 100644
index 0000000..16fd884
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceAssignment.java
@@ -0,0 +1,58 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.
+ */
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.helix.model.Partition;
+
+/**
+ * Represents the assignments of replicas for an entire resource, keyed on partitions of the
+ * resource. Each partition has its replicas assigned to a node, and each replica is in a state.
+ */
+public class ResourceAssignment {
+
+  private final Map<Partition, Map<String, String>> _resourceMap;
+
+  public ResourceAssignment() {
+    this(new HashMap<Partition, Map<String, String>>());
+  }
+
+  public ResourceAssignment(Map<Partition, Map<String, String>> resourceMap) {
+    _resourceMap = resourceMap;
+  }
+
+  public Map<Partition, Map<String, String>> getResourceMap() {
+    return _resourceMap;
+  }
+
+  public Map<String, String> getInstanceStateMap(Partition partition) {
+    if (_resourceMap.containsKey(partition)) {
+      return _resourceMap.get(partition);
+    }
+    return Collections.emptyMap();
+  }
+
+  public void addReplicaMap(Partition partition, Map<String, String> replicaMap) {
+    _resourceMap.put(partition, replicaMap);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java b/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java
deleted file mode 100644
index 2609791..0000000
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/ResourceMapping.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package org.apache.helix.controller.stages;
-
-/*
- * 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.
- */
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.helix.model.Partition;
-
-/**
- * Represents the assignments of replicas for an entire resource, keyed on partitions of the
- * resource. Each partition has its replicas assigned to a node, and each replica is in a state.
- */
-public class ResourceMapping {
-
-  private final Map<Partition, Map<String, String>> _resourceMap;
-
-  public ResourceMapping() {
-    this(new HashMap<Partition, Map<String, String>>());
-  }
-
-  public ResourceMapping(Map<Partition, Map<String, String>> resourceMap) {
-    _resourceMap = resourceMap;
-  }
-
-  public Map<Partition, Map<String, String>> getResourceMap() {
-    return _resourceMap;
-  }
-
-  public Map<String, String> getInstanceStateMap(Partition partition) {
-    if (_resourceMap.containsKey(partition)) {
-      return _resourceMap.get(partition);
-    }
-    return Collections.emptyMap();
-  }
-
-  public void addReplicaMap(Partition partition, Map<String, String> replicaMap) {
-    _resourceMap.put(partition, replicaMap);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/40bb97a5/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
----------------------------------------------------------------------
diff --git a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
index 7811c0d..4575def 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/TestCustomizedIdealStateRebalancer.java
@@ -24,9 +24,9 @@ import java.util.Map;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
-import org.apache.helix.ZNRecord;
 import org.apache.helix.PropertyKey.Builder;
-import org.apache.helix.controller.rebalancer.SemiAutoRebalancer;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.controller.rebalancer.Rebalancer;
 import org.apache.helix.controller.stages.ClusterDataCache;
 import org.apache.helix.controller.stages.CurrentStateOutput;
 import org.apache.helix.manager.zk.ZKHelixDataAccessor;
@@ -46,7 +46,7 @@ public class TestCustomizedIdealStateRebalancer extends
   String db2 = TEST_DB + "2";
   static boolean testRebalancerCreated = false;
 
-  public static class TestRebalancer extends SemiAutoRebalancer {
+  public static class TestRebalancer implements Rebalancer {
 
     @Override
     public void init(HelixManager manager) {
@@ -86,7 +86,7 @@ public class TestCustomizedIdealStateRebalancer extends
     Assert.assertTrue(result);
     Thread.sleep(1000);
     HelixDataAccessor accessor =
-        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor(_zkClient));
+        new ZKHelixDataAccessor(CLUSTER_NAME, new ZkBaseDataAccessor<ZNRecord>(_zkClient));
     Builder keyBuilder = accessor.keyBuilder();
     ExternalView ev = accessor.getProperty(keyBuilder.externalView(db2));
     Assert.assertEquals(ev.getPartitionSet().size(), 60);
@@ -116,7 +116,7 @@ public class TestCustomizedIdealStateRebalancer extends
     public boolean verify() {
       try {
         HelixDataAccessor accessor =
-            new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor(_client));
+            new ZKHelixDataAccessor(_clusterName, new ZkBaseDataAccessor<ZNRecord>(_client));
         Builder keyBuilder = accessor.keyBuilder();
         int numberOfPartitions =
             accessor.getProperty(keyBuilder.idealStates(_resourceName)).getRecord().getListFields()


[2/2] git commit: Merge branch 'helix-0.6.2-release' of https://git-wip-us.apache.org/repos/asf/incubator-helix into helix-0.6.2-release

Posted by ka...@apache.org.
Merge branch 'helix-0.6.2-release' of https://git-wip-us.apache.org/repos/asf/incubator-helix into helix-0.6.2-release


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

Branch: refs/heads/helix-0.6.2-release
Commit: 7db3d060e99aa06e4241401485e37c65bb6e2298
Parents: 40bb97a 0c3796b
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Fri Oct 18 15:26:36 2013 -0700
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Fri Oct 18 15:26:36 2013 -0700

----------------------------------------------------------------------
 .../org/apache/helix/manager/zk/ZkClient.java   |  41 ++-
 .../java/org/apache/helix/ZkTestHelper.java     |  48 +++-
 .../apache/helix/manager/zk/TestZkFlapping.java | 272 +++++++++++++++++++
 3 files changed, 348 insertions(+), 13 deletions(-)
----------------------------------------------------------------------