You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@helix.apache.org by ji...@apache.org on 2020/04/23 19:28:07 UTC

[helix] 21/23: Add registration logic for CustomizedView listeners (#944)

This is an automated email from the ASF dual-hosted git repository.

jiajunwang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/helix.git

commit 36ba93f5ad87a14a87a307c62722a726d0535e53
Author: Ali Reza Zamani Zadeh Najari <an...@linkedin.com>
AuthorDate: Mon Apr 13 10:53:44 2020 -0700

    Add registration logic for CustomizedView listeners (#944)
    
    In this commit, a new logic is added which target the scenario where user
    disables and enables specific types. In this case, since the CustomizedView
    path for that type is removed by the controller, the router looses its
    listener. In this commit, we added root change lister and re-registers the
    listens again.
---
 .../main/java/org/apache/helix/HelixConstants.java |  1 +
 .../main/java/org/apache/helix/HelixManager.java   |  7 ++++
 .../CustomizedViewRootChangeListener.java          | 39 ++++++++++++++++++++++
 .../apache/helix/manager/zk/CallbackHandler.java   | 16 +++++++++
 .../apache/helix/manager/zk/ZKHelixManager.java    |  9 +++++
 .../helix/spectator/RoutingTableProvider.java      | 34 ++++++++++++++++++-
 .../controller/stages/DummyClusterManager.java     |  6 ++++
 .../manager/TestParticipantManager.java            |  2 +-
 .../java/org/apache/helix/mock/MockManager.java    |  6 ++++
 .../helix/participant/MockZKHelixManager.java      |  5 +++
 10 files changed, 123 insertions(+), 2 deletions(-)

diff --git a/helix-core/src/main/java/org/apache/helix/HelixConstants.java b/helix-core/src/main/java/org/apache/helix/HelixConstants.java
index ca32c44..8691ffa 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixConstants.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixConstants.java
@@ -39,6 +39,7 @@ public interface HelixConstants {
     MESSAGE (PropertyType.MESSAGES),
     EXTERNAL_VIEW (PropertyType.EXTERNALVIEW),
     CUSTOMIZED_VIEW (PropertyType.CUSTOMIZEDVIEW),
+    CUSTOMIZED_VIEW_ROOT (PropertyType.CUSTOMIZEDVIEW),
     TARGET_EXTERNAL_VIEW (PropertyType.TARGETEXTERNALVIEW),
     CONTROLLER (PropertyType.CONTROLLER),
     MESSAGES_CONTROLLER (PropertyType.MESSAGES_CONTROLLER),
diff --git a/helix-core/src/main/java/org/apache/helix/HelixManager.java b/helix-core/src/main/java/org/apache/helix/HelixManager.java
index 9702015..c7e31a5 100644
--- a/helix-core/src/main/java/org/apache/helix/HelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/HelixManager.java
@@ -30,6 +30,7 @@ import org.apache.helix.api.listeners.CustomizedStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateConfigChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateRootChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -257,6 +258,12 @@ public interface HelixManager {
   void addCustomizedViewChangeListener(CustomizedViewChangeListener listener, String customizedStateType) throws Exception;
 
   /**
+   * @see CustomizedViewRootChangeListener#onCustomizedViewRootChange(List, NotificationContext)
+   * @param listener
+   */
+  void addCustomizedViewRootChangeListener(CustomizedViewRootChangeListener listener) throws Exception;
+
+  /**
    * @see ExternalViewChangeListener#onExternalViewChange(List, NotificationContext)
    * @param listener
    */
diff --git a/helix-core/src/main/java/org/apache/helix/api/listeners/CustomizedViewRootChangeListener.java b/helix-core/src/main/java/org/apache/helix/api/listeners/CustomizedViewRootChangeListener.java
new file mode 100644
index 0000000..a1ee62a
--- /dev/null
+++ b/helix-core/src/main/java/org/apache/helix/api/listeners/CustomizedViewRootChangeListener.java
@@ -0,0 +1,39 @@
+package org.apache.helix.api.listeners;
+
+/*
+ * 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.List;
+
+import org.apache.helix.NotificationContext;
+
+
+/**
+ * Interface to implement to respond to changes in the root path of customized View
+ */
+public interface CustomizedViewRootChangeListener {
+
+  /**
+   * Invoked when root path of CustomizedView changes
+   * @param customizedViewTypes CustomizedView types
+   * @param changeContext
+   */
+  void onCustomizedViewRootChange(List<String> customizedViewTypes,
+      NotificationContext changeContext);
+}
\ No newline at end of file
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
index 436873a..58460a3 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/CallbackHandler.java
@@ -50,6 +50,7 @@ import org.apache.helix.api.listeners.CustomizedStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateConfigChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateRootChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -89,6 +90,7 @@ import static org.apache.helix.HelixConstants.ChangeType.CUSTOMIZED_STATE;
 import static org.apache.helix.HelixConstants.ChangeType.CUSTOMIZED_STATE_CONFIG;
 import static org.apache.helix.HelixConstants.ChangeType.CUSTOMIZED_STATE_ROOT;
 import static org.apache.helix.HelixConstants.ChangeType.CUSTOMIZED_VIEW;
+import static org.apache.helix.HelixConstants.ChangeType.CUSTOMIZED_VIEW_ROOT;
 import static org.apache.helix.HelixConstants.ChangeType.EXTERNAL_VIEW;
 import static org.apache.helix.HelixConstants.ChangeType.IDEAL_STATE;
 import static org.apache.helix.HelixConstants.ChangeType.INSTANCE_CONFIG;
@@ -98,6 +100,7 @@ import static org.apache.helix.HelixConstants.ChangeType.MESSAGES_CONTROLLER;
 import static org.apache.helix.HelixConstants.ChangeType.RESOURCE_CONFIG;
 import static org.apache.helix.HelixConstants.ChangeType.TARGET_EXTERNAL_VIEW;
 
+
 @PreFetch(enabled = false)
 public class CallbackHandler implements IZkChildListener, IZkDataListener {
   private static Logger logger = LoggerFactory.getLogger(CallbackHandler.class);
@@ -313,6 +316,9 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener {
       case CUSTOMIZED_VIEW:
         listenerClass = CustomizedViewChangeListener.class;
         break;
+      case CUSTOMIZED_VIEW_ROOT:
+        listenerClass = CustomizedViewRootChangeListener.class;
+        break;
       case CONTROLLER:
         listenerClass = ControllerChangeListener.class;
     }
@@ -484,6 +490,16 @@ public class CallbackHandler implements IZkChildListener, IZkDataListener {
         List<ExternalView> externalViewList = preFetch(_propertyKey);
         externalViewListener.onExternalViewChange(externalViewList, changeContext);
 
+      } else if (_changeType == CUSTOMIZED_VIEW_ROOT) {
+        CustomizedViewRootChangeListener customizedViewRootChangeListener =
+            (CustomizedViewRootChangeListener) _listener;
+        List<String> customizedViewTypes = new ArrayList<>();
+        if (_preFetchEnabled) {
+          customizedViewTypes = _accessor.getChildNames(_accessor.keyBuilder().customizedViews());
+        }
+        customizedViewRootChangeListener.onCustomizedViewRootChange(customizedViewTypes,
+            changeContext);
+
       } else if (_changeType == CUSTOMIZED_VIEW) {
         CustomizedViewChangeListener customizedViewListener = (CustomizedViewChangeListener) _listener;
         List<CustomizedView> customizedViewListList = preFetch(_propertyKey);
diff --git a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
index ad56f95..e2429d7 100644
--- a/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
+++ b/helix-core/src/main/java/org/apache/helix/manager/zk/ZKHelixManager.java
@@ -61,6 +61,7 @@ import org.apache.helix.api.listeners.CustomizedStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateConfigChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateRootChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -618,6 +619,14 @@ public class ZKHelixManager implements HelixManager, IZkStateListener {
   }
 
   @Override
+  public void addCustomizedViewRootChangeListener(CustomizedViewRootChangeListener listener) throws Exception {
+    addListener(listener, new Builder(_clusterName).customizedViews(),
+        ChangeType.CUSTOMIZED_VIEW_ROOT, new EventType[] {
+            EventType.NodeChildrenChanged
+        });
+  }
+
+  @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     addListener(listener, new Builder(_clusterName).externalViews(), ChangeType.TARGET_EXTERNAL_VIEW,
         new EventType[] { EventType.NodeChildrenChanged });
diff --git a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
index 1e11965..cfd4350 100644
--- a/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
+++ b/helix-core/src/main/java/org/apache/helix/spectator/RoutingTableProvider.java
@@ -46,6 +46,7 @@ import org.apache.helix.PropertyType;
 import org.apache.helix.api.listeners.ConfigChangeListener;
 import org.apache.helix.api.listeners.CurrentStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
 import org.apache.helix.api.listeners.LiveInstanceChangeListener;
@@ -67,7 +68,8 @@ import org.slf4j.LoggerFactory;
 
 public class RoutingTableProvider
     implements ExternalViewChangeListener, InstanceConfigChangeListener, ConfigChangeListener,
-    LiveInstanceChangeListener, CurrentStateChangeListener, CustomizedViewChangeListener {
+               LiveInstanceChangeListener, CurrentStateChangeListener, CustomizedViewChangeListener,
+               CustomizedViewRootChangeListener {
   private static final Logger logger = LoggerFactory.getLogger(RoutingTableProvider.class);
   private static final long DEFAULT_PERIODIC_REFRESH_INTERVAL = 300000L; // 5 minutes
   private final Map<String, AtomicReference<RoutingTable>> _routingTableRefMap;
@@ -230,6 +232,15 @@ public class RoutingTableProvider
           }
           break;
         case CUSTOMIZEDVIEW:
+          // Add CustomizedView root change listener
+          try {
+            _helixManager.addCustomizedViewRootChangeListener(this);
+          } catch (Exception e) {
+            shutdown();
+            throw new HelixException(
+                "Failed to attach CustomizedView Root Listener to HelixManager!", e);
+          }
+          // Add individual listeners for each customizedStateType
           List<String> customizedStateTypes = _sourceDataTypeMap.get(propertyType);
           for (String customizedStateType : customizedStateTypes) {
             try {
@@ -683,6 +694,27 @@ public class RoutingTableProvider
     }
   }
 
+  @Override
+  @PreFetch(enabled = false)
+  public void onCustomizedViewRootChange(List<String> customizedViewTypes,
+      NotificationContext changeContext) {
+    logger.info(
+        "Registering the CustomizedView listeners again due to the CustomizedView root change.");
+    List<String> userRequestedTypes =
+        _sourceDataTypeMap.getOrDefault(PropertyType.CUSTOMIZEDVIEW, Collections.emptyList());
+    for (String customizedStateType : userRequestedTypes) {
+      try {
+        _helixManager.addCustomizedViewChangeListener(this, customizedStateType);
+      } catch (Exception e) {
+        shutdown();
+        throw new HelixException(
+            String.format("Failed to attach CustomizedView Listener to HelixManager for type %s!",
+                customizedStateType),
+            e);
+      }
+    }
+  }
+
   final AtomicReference<Map<String, LiveInstance>> _lastSeenSessions = new AtomicReference<>();
 
   /**
diff --git a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
index 83dd49e..ba214c7 100644
--- a/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
+++ b/helix-core/src/test/java/org/apache/helix/controller/stages/DummyClusterManager.java
@@ -40,6 +40,7 @@ import org.apache.helix.api.listeners.CustomizedStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateConfigChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateRootChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -160,6 +161,11 @@ public class DummyClusterManager implements HelixManager {
   }
 
   @Override
+  public void addCustomizedViewRootChangeListener(CustomizedViewRootChangeListener listener) throws Exception {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     // TODO Auto-generated method stub
   }
diff --git a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
index 7922153..75d14b3 100644
--- a/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
+++ b/helix-core/src/test/java/org/apache/helix/integration/manager/TestParticipantManager.java
@@ -151,7 +151,7 @@ public class TestParticipantManager extends ZkTestBase {
     // check HelixCallback Monitor
     Set<ObjectInstance> objs =
         _server.queryMBeans(buildCallbackMonitorObjectName(type, clusterName, instanceName), null);
-    Assert.assertEquals(objs.size(), 17);
+    Assert.assertEquals(objs.size(), 18);
 
     // check HelixZkClient Monitors
     objs =
diff --git a/helix-core/src/test/java/org/apache/helix/mock/MockManager.java b/helix-core/src/test/java/org/apache/helix/mock/MockManager.java
index 51b932d..4ac1a9f 100644
--- a/helix-core/src/test/java/org/apache/helix/mock/MockManager.java
+++ b/helix-core/src/test/java/org/apache/helix/mock/MockManager.java
@@ -42,6 +42,7 @@ import org.apache.helix.api.listeners.CustomizedStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateConfigChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateRootChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -163,6 +164,11 @@ public class MockManager implements HelixManager {
 
   }
 
+  public void addCustomizedViewRootChangeListener(CustomizedViewRootChangeListener listener) throws Exception {
+    // TODO Auto-generated method stub
+
+  }
+
   @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     // TODO Auto-generated method stub
diff --git a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
index e671482..7d28304 100644
--- a/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
+++ b/helix-core/src/test/java/org/apache/helix/participant/MockZKHelixManager.java
@@ -41,6 +41,7 @@ import org.apache.helix.api.listeners.CustomizedStateChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateConfigChangeListener;
 import org.apache.helix.api.listeners.CustomizedStateRootChangeListener;
 import org.apache.helix.api.listeners.CustomizedViewChangeListener;
+import org.apache.helix.api.listeners.CustomizedViewRootChangeListener;
 import org.apache.helix.api.listeners.ExternalViewChangeListener;
 import org.apache.helix.api.listeners.IdealStateChangeListener;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
@@ -195,6 +196,10 @@ public class MockZKHelixManager implements HelixManager {
 
   }
 
+  public void addCustomizedViewRootChangeListener(CustomizedViewRootChangeListener listener) throws Exception {
+    // TODO Auto-generated method stub
+  }
+
   @Override
   public void addTargetExternalViewChangeListener(ExternalViewChangeListener listener) throws Exception {
     // TODO Auto-generated method stub