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/12/04 20:11:53 UTC

git commit: [HELIX-333] Support remove in ControllerContextProvider, rb=16006

Updated Branches:
  refs/heads/master 03001d992 -> 992bca8a0


[HELIX-333] Support remove in ControllerContextProvider, rb=16006


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

Branch: refs/heads/master
Commit: 992bca8a08eac87f62da549539dee731e073b36f
Parents: 03001d9
Author: Kanak Biscuitwala <ka...@apache.org>
Authored: Wed Dec 4 10:44:19 2013 -0800
Committer: Kanak Biscuitwala <ka...@apache.org>
Committed: Wed Dec 4 11:11:42 2013 -0800

----------------------------------------------------------------------
 .../context/ControllerContextProvider.java      | 71 +++++++++++++++++---
 .../controller/stages/PersistContextStage.java  |  9 +++
 .../TestCustomizedIdealStateRebalancer.java     |  2 +-
 3 files changed, 72 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/992bca8a/helix-core/src/main/java/org/apache/helix/controller/context/ControllerContextProvider.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/context/ControllerContextProvider.java b/helix-core/src/main/java/org/apache/helix/controller/context/ControllerContextProvider.java
index 1541585..93ed78c 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/context/ControllerContextProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/context/ControllerContextProvider.java
@@ -21,21 +21,26 @@ package org.apache.helix.controller.context;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.helix.api.id.ContextId;
 import org.apache.log4j.Logger;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 /**
  * An interface for getting and setting {@link ControllerContext} objects, which will eventually
- * be persisted and acessible across runs of the controller pipeline.
+ * be persisted and accessible across runs of the controller pipeline.
  */
 public class ControllerContextProvider {
   private static final Logger LOG = Logger.getLogger(ControllerContextProvider.class);
 
   private Map<ContextId, ControllerContext> _persistedContexts;
   private Map<ContextId, ControllerContext> _pendingContexts;
+  private Set<ContextId> _removedContexts;
 
   /**
    * Instantiate with already-persisted controller contexts
@@ -44,6 +49,7 @@ public class ControllerContextProvider {
   public ControllerContextProvider(Map<ContextId, ControllerContext> contexts) {
     _persistedContexts = contexts != null ? contexts : new HashMap<ContextId, ControllerContext>();
     _pendingContexts = Maps.newHashMap();
+    _removedContexts = Sets.newHashSet();
   }
 
   /**
@@ -51,8 +57,8 @@ public class ControllerContextProvider {
    * @param contextId the context id to look up
    * @return a ControllerContext, or null if not found
    */
-  public ControllerContext getControllerContext(ContextId contextId) {
-    return getControllerContext(contextId, ControllerContext.class);
+  public ControllerContext getContext(ContextId contextId) {
+    return getContext(contextId, ControllerContext.class);
   }
 
   /**
@@ -62,7 +68,7 @@ public class ControllerContextProvider {
    * @return a typed ControllerContext, or null if no context with given id is available for this
    *         type
    */
-  public <T extends ControllerContext> T getControllerContext(ContextId contextId,
+  public <T extends ControllerContext> T getContext(ContextId contextId,
       Class<T> contextClass) {
     try {
       if (_pendingContexts.containsKey(contextId)) {
@@ -81,8 +87,8 @@ public class ControllerContextProvider {
    * @param contextId the id to set
    * @param context the context object
    */
-  public void putControllerContext(ContextId contextId, ControllerContext context) {
-    putControllerContext(contextId, context, true);
+  public void putContext(ContextId contextId, ControllerContext context) {
+    putContext(contextId, context, true);
   }
 
   /**
@@ -92,16 +98,44 @@ public class ControllerContextProvider {
    * @param overwriteAllowed true if existing objects can be overwritten, false otherwise
    * @return true if saved, false if an object with that id exists and overwrite is not allowed
    */
-  public boolean putControllerContext(ContextId contextId, ControllerContext context,
+  public boolean putContext(ContextId contextId, ControllerContext context,
       boolean overwriteAllowed) {
+    // avoid persisting null contexts
+    if (context == null) {
+      LOG.error("Cannot save a null context, id: " + contextId);
+      return false;
+    }
     if (overwriteAllowed || !exists(contextId)) {
       _pendingContexts.put(contextId, context);
+      if (_removedContexts.contains(contextId)) {
+        // no need to mark as removed if it's being added again
+        _removedContexts.remove(contextId);
+      }
       return true;
     }
     return false;
   }
 
   /**
+   * Remove a controller context
+   * @param contextId the id to remove
+   * @return ControllerContext that was removed, or null
+   */
+  public ControllerContext removeContext(ContextId contextId) {
+    ControllerContext removed = null;
+    if (_persistedContexts.containsKey(contextId)) {
+      removed = _persistedContexts.remove(contextId);
+    }
+    if (_pendingContexts.containsKey(contextId)) {
+      // check pending second since it might overwrite a persisted context
+      removed = _pendingContexts.remove(contextId);
+    }
+    // mark as removed even if pending; this is so that remove, put, remove works
+    _removedContexts.add(contextId);
+    return removed;
+  }
+
+  /**
    * Check if a context exists
    * @param contextId the id to look up
    * @return true if a context exists with that id, false otherwise
@@ -111,10 +145,29 @@ public class ControllerContextProvider {
   }
 
   /**
+   * Get all contexts, both persisted and pending
+   * @return an immutable map of context id to context
+   */
+  public Map<ContextId, ControllerContext> getContexts() {
+    Map<ContextId, ControllerContext> aggregateMap = Maps.newHashMap();
+    aggregateMap.putAll(_persistedContexts);
+    aggregateMap.putAll(_pendingContexts);
+    return ImmutableMap.copyOf(aggregateMap);
+  }
+
+  /**
    * Get all contexts that have been put, but not yet persisted
-   * @return a map of context id to context
+   * @return an immutable map of context id to context
    */
   public Map<ContextId, ControllerContext> getPendingContexts() {
-    return _pendingContexts;
+    return ImmutableMap.copyOf(_pendingContexts);
+  }
+
+  /**
+   * Get all context ids that have been marked for removal
+   * @return a set of context ids
+   */
+  public Set<ContextId> getRemovedContexts() {
+    return ImmutableSet.copyOf(_removedContexts);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/992bca8a/helix-core/src/main/java/org/apache/helix/controller/stages/PersistContextStage.java
----------------------------------------------------------------------
diff --git a/helix-core/src/main/java/org/apache/helix/controller/stages/PersistContextStage.java b/helix-core/src/main/java/org/apache/helix/controller/stages/PersistContextStage.java
index e63041a..7c2cc0f 100644
--- a/helix-core/src/main/java/org/apache/helix/controller/stages/PersistContextStage.java
+++ b/helix-core/src/main/java/org/apache/helix/controller/stages/PersistContextStage.java
@@ -21,6 +21,7 @@ package org.apache.helix.controller.stages;
 
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.HelixManager;
@@ -44,6 +45,14 @@ public class PersistContextStage extends AbstractBaseStage {
     PropertyKey.Builder keyBuilder = accessor.keyBuilder();
     ControllerContextProvider contextProvider =
         event.getAttribute(AttributeName.CONTEXT_PROVIDER.toString());
+
+    // remove marked contexts
+    Set<ContextId> removedContexts = contextProvider.getRemovedContexts();
+    for (ContextId contextId : removedContexts) {
+      accessor.removeProperty(keyBuilder.controllerContext(contextId.stringify()));
+    }
+
+    // persist pending contexts
     Map<ContextId, ControllerContext> pendingContexts = contextProvider.getPendingContexts();
     List<PropertyKey> keys = Lists.newArrayList();
     List<ControllerContextHolder> properties = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-helix/blob/992bca8a/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 11805fe..7bf4f94 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
@@ -92,7 +92,7 @@ public class TestCustomizedIdealStateRebalancer extends
 
       // set some basic context
       ContextId contextId = ContextId.from(config.getResourceId().stringify());
-      _contextProvider.putControllerContext(contextId, new BasicControllerContext(contextId));
+      _contextProvider.putContext(contextId, new BasicControllerContext(contextId));
       return resourceMapping;
     }