You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by kh...@apache.org on 2020/09/15 19:47:32 UTC

[incubator-pinot] branch controller_api_patch created (now 36babc3)

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

kharekartik pushed a change to branch controller_api_patch
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


      at 36babc3  change in constructors of listeners

This branch includes the following new commits:

     new acd9397  Add listeners to cache zookeeper configs for helix
     new f347289  Fetch configs from cache rather than zookeeper
     new e30ffb9  Add change for Helix Configs
     new 36babc3  change in constructors of listeners

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 02/04: Fetch configs from cache rather than zookeeper

Posted by kh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kharekartik pushed a commit to branch controller_api_patch
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit f3472898ad5e81f0de3267c4b2333b629fc9d007
Author: Kartik Khare <kh...@Kartiks-MacBook-Pro.local>
AuthorDate: Tue Sep 15 19:56:11 2020 +0530

    Fetch configs from cache rather than zookeeper
---
 .../helix/core/PinotHelixResourceManager.java      | 70 +++++++++++++++++-----
 1 file changed, 54 insertions(+), 16 deletions(-)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
index 1536128..917a777 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
@@ -25,7 +25,6 @@ import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
-import com.google.common.collect.Lists;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -57,6 +56,8 @@ import org.apache.helix.InstanceType;
 import org.apache.helix.PropertyKey;
 import org.apache.helix.PropertyKey.Builder;
 import org.apache.helix.ZNRecord;
+import org.apache.helix.api.listeners.ExternalViewChangeListener;
+import org.apache.helix.api.listeners.LiveInstanceChangeListener;
 import org.apache.helix.model.CurrentState;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.HelixConfigScope;
@@ -103,6 +104,9 @@ import org.apache.pinot.controller.helix.core.assignment.instance.InstanceAssign
 import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignment;
 import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentFactory;
 import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
+import org.apache.pinot.controller.helix.core.listener.ClusterExternalViewChangeListener;
+import org.apache.pinot.controller.helix.core.listener.ClusterInstanceConfigChangeListener;
+import org.apache.pinot.controller.helix.core.listener.ClusterLiveInstanceChangeListener;
 import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager;
 import org.apache.pinot.controller.helix.core.rebalance.RebalanceResult;
 import org.apache.pinot.controller.helix.core.rebalance.TableRebalancer;
@@ -160,6 +164,9 @@ public class PinotHelixResourceManager {
   private SegmentDeletionManager _segmentDeletionManager;
   private PinotLLCRealtimeSegmentManager _pinotLLCRealtimeSegmentManager;
   private TableCache _tableCache;
+  private ClusterInstanceConfigChangeListener _clusterInstanceConfigChangeListener;
+  private ClusterLiveInstanceChangeListener _clusterLiveInstanceChangeListener;
+  private ClusterExternalViewChangeListener _clusterExternalViewChangeListener;
 
   public PinotHelixResourceManager(String zkURL, String helixClusterName, @Nullable String dataDir,
       long externalViewOnlineToOfflineTimeoutMillis, boolean isSingleTenantCluster, boolean enableBatchMessageMode,
@@ -223,6 +230,18 @@ public class PinotHelixResourceManager {
     boolean caseInsensitive = Boolean.parseBoolean(configs.get(Helix.ENABLE_CASE_INSENSITIVE_KEY)) || Boolean
         .parseBoolean(configs.get(Helix.DEPRECATED_ENABLE_CASE_INSENSITIVE_KEY));
     _tableCache = new TableCache(_propertyStore, caseInsensitive);
+
+    _clusterInstanceConfigChangeListener = new ClusterInstanceConfigChangeListener();
+    _clusterLiveInstanceChangeListener = new ClusterLiveInstanceChangeListener();
+    _clusterExternalViewChangeListener = new ClusterExternalViewChangeListener();
+    try {
+      addConfigListeners(_clusterInstanceConfigChangeListener);
+      addLiveInstanceListeners(_clusterLiveInstanceChangeListener);
+      addExternalViewListeners(_clusterExternalViewChangeListener);
+    } catch (Exception e) {
+      LOGGER.warn(
+          "Unable to add config listener in controller. This will result in incorrect response from controller's broker API");
+    }
   }
 
   /**
@@ -317,7 +336,7 @@ public class PinotHelixResourceManager {
    * Returns the config for all the Helix instances in the cluster.
    */
   public List<InstanceConfig> getAllHelixInstanceConfigs() {
-    return HelixHelper.getInstanceConfigs(_helixZkManager);
+    return _clusterInstanceConfigChangeListener.getInstanceConfigs();
   }
 
   /**
@@ -359,14 +378,15 @@ public class PinotHelixResourceManager {
         brokerTenantName = realtimeTableConfig.getTenantConfig().getBroker();
       }
     }
-    return HelixHelper.getInstancesWithTag(_helixZkManager, TagNameUtils.getBrokerTagForTenant(brokerTenantName));
+    return HelixHelper.getInstancesWithTag(_clusterInstanceConfigChangeListener.getInstanceConfigs(),
+        TagNameUtils.getBrokerTagForTenant(brokerTenantName));
   }
 
   /**
    * Get all instances with the given tag
    */
   public List<String> getInstancesWithTag(String tag) {
-    return HelixHelper.getInstancesWithTag(_helixZkManager, tag);
+    return HelixHelper.getInstancesWithTag(_clusterInstanceConfigChangeListener.getInstanceConfigs(), tag);
   }
 
   /**
@@ -833,7 +853,7 @@ public class PinotHelixResourceManager {
 
   public Set<String> getAllBrokerTenantNames() {
     Set<String> tenantSet = new HashSet<>();
-    List<InstanceConfig> instanceConfigs = getAllHelixInstanceConfigs();
+    List<InstanceConfig> instanceConfigs = _clusterInstanceConfigChangeListener.getInstanceConfigs();
     for (InstanceConfig instanceConfig : instanceConfigs) {
       for (String tag : instanceConfig.getTags()) {
         if (TagNameUtils.isBrokerTag(tag)) {
@@ -969,7 +989,7 @@ public class PinotHelixResourceManager {
   }
 
   public Set<String> getAllInstancesForServerTenant(String tenantName) {
-    return getAllInstancesForServerTenant(HelixHelper.getInstanceConfigs(_helixZkManager), tenantName);
+    return getAllInstancesForServerTenant(_clusterInstanceConfigChangeListener.getInstanceConfigs(), tenantName);
   }
 
   /**
@@ -981,7 +1001,7 @@ public class PinotHelixResourceManager {
   }
 
   public Set<String> getAllInstancesForBrokerTenant(String tenantName) {
-    return getAllInstancesForBrokerTenant(HelixHelper.getInstanceConfigs(_helixZkManager), tenantName);
+    return getAllInstancesForBrokerTenant(_clusterInstanceConfigChangeListener.getInstanceConfigs(), tenantName);
   }
 
   /**
@@ -1798,7 +1818,8 @@ public class PinotHelixResourceManager {
         _helixZkManager.getMessagingService().send(recipientCriteria, routingTableRebuildMessage, null, -1);
     if (numMessagesSent > 0) {
       // TODO: Would be nice if we can get the name of the instances to which messages were sent
-      LOGGER.info("Sent {} routing table rebuild messages to brokers for table: {}", numMessagesSent, tableNameWithType);
+      LOGGER
+          .info("Sent {} routing table rebuild messages to brokers for table: {}", numMessagesSent, tableNameWithType);
     } else {
       LOGGER.warn("No routing table rebuild message sent to brokers for table: {}", tableNameWithType);
     }
@@ -2010,8 +2031,8 @@ public class PinotHelixResourceManager {
   public List<String> getBrokerInstancesForTable(String tableName, TableType tableType) {
     TableConfig tableConfig = getTableConfig(tableName, tableType);
     Preconditions.checkNotNull(tableConfig);
-    return HelixHelper
-        .getInstancesWithTag(_helixZkManager, TagNameUtils.extractBrokerTag(tableConfig.getTenantConfig()));
+    return HelixHelper.getInstancesWithTag(_clusterInstanceConfigChangeListener.getInstanceConfigs(),
+        TagNameUtils.extractBrokerTag(tableConfig.getTenantConfig()));
   }
 
   public PinotResourceManagerResponse enableInstance(String instanceName) {
@@ -2163,7 +2184,7 @@ public class PinotHelixResourceManager {
    */
   public List<String> getOnlineUnTaggedBrokerInstanceList() {
     List<String> instanceList = HelixHelper.getInstancesWithTag(_helixZkManager, Helix.UNTAGGED_BROKER_INSTANCE);
-    List<String> liveInstances = _helixDataAccessor.getChildNames(_keyBuilder.liveInstances());
+    List<String> liveInstances = getOnlineInstanceList();
     instanceList.retainAll(liveInstances);
     return instanceList;
   }
@@ -2174,13 +2195,29 @@ public class PinotHelixResourceManager {
    */
   public List<String> getOnlineUnTaggedServerInstanceList() {
     List<String> instanceList = HelixHelper.getInstancesWithTag(_helixZkManager, Helix.UNTAGGED_SERVER_INSTANCE);
-    List<String> liveInstances = _helixDataAccessor.getChildNames(_keyBuilder.liveInstances());
+    List<String> liveInstances = getOnlineInstanceList();
     instanceList.retainAll(liveInstances);
     return instanceList;
   }
 
   public List<String> getOnlineInstanceList() {
-    return _helixDataAccessor.getChildNames(_keyBuilder.liveInstances());
+    return _clusterLiveInstanceChangeListener.getLiveInstances().stream().map(LiveInstance::getInstanceName)
+        .collect(Collectors.toList());
+  }
+
+  public void addConfigListeners(ClusterInstanceConfigChangeListener clusterInstanceConfigChangeListener)
+      throws Exception {
+    _helixZkManager.addInstanceConfigChangeListener(clusterInstanceConfigChangeListener);
+  }
+
+  public void addLiveInstanceListeners(LiveInstanceChangeListener liveInstanceChangeListener)
+      throws Exception {
+    _helixZkManager.addLiveInstanceChangeListener(liveInstanceChangeListener);
+  }
+
+  public void addExternalViewListeners(ExternalViewChangeListener externalViewChangeListener)
+      throws Exception {
+    _helixZkManager.addExternalViewChangeListener(externalViewChangeListener);
   }
 
   /**
@@ -2437,9 +2474,9 @@ public class PinotHelixResourceManager {
       }
       Thread.sleep(EXTERNAL_VIEW_CHECK_INTERVAL_MS);
     } while (System.currentTimeMillis() < endTimeMs);
-    throw new TimeoutException(String.format(
-        "Time out while waiting segments become ONLINE. (tableNameWithType = %s, segmentsToCheck = %s)",
-        tableNameWithType, segmentsToCheck));
+    throw new TimeoutException(String
+        .format("Time out while waiting segments become ONLINE. (tableNameWithType = %s, segmentsToCheck = %s)",
+            tableNameWithType, segmentsToCheck));
   }
 
   private Set<String> getOnlineSegmentsFromExternalView(String tableNameWithType) {
@@ -2458,6 +2495,7 @@ public class PinotHelixResourceManager {
     return onlineSegments;
   }
 
+
   /*
    * Uncomment and use for testing on a real cluster
   public static void main(String[] args) throws Exception {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 01/04: Add listeners to cache zookeeper configs for helix

Posted by kh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kharekartik pushed a commit to branch controller_api_patch
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit acd9397d28f1edd400c4b6fcdea8473a770d475e
Author: Kartik Khare <kh...@Kartiks-MacBook-Pro.local>
AuthorDate: Tue Sep 15 19:54:47 2020 +0530

    Add listeners to cache zookeeper configs for helix
---
 .../ClusterExternalViewChangeListener.java         | 21 +++++++++++++++++++
 .../ClusterInstanceConfigChangeListener.java       | 24 ++++++++++++++++++++++
 .../ClusterLiveInstanceChangeListener.java         | 21 +++++++++++++++++++
 3 files changed, 66 insertions(+)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterExternalViewChangeListener.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterExternalViewChangeListener.java
new file mode 100644
index 0000000..6400f47
--- /dev/null
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterExternalViewChangeListener.java
@@ -0,0 +1,21 @@
+package org.apache.pinot.controller.helix.core.listener;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.api.listeners.ExternalViewChangeListener;
+import org.apache.helix.model.ExternalView;
+
+
+public class ClusterExternalViewChangeListener implements ExternalViewChangeListener {
+  private List<ExternalView> _externalViewList = new ArrayList<>();
+
+  @Override
+  public void onExternalViewChange(List<ExternalView> externalViewList, NotificationContext changeContext) {
+    _externalViewList = externalViewList;
+  }
+
+  public List<ExternalView> getExternalViewList() {
+    return _externalViewList;
+  }
+}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java
new file mode 100644
index 0000000..940230c
--- /dev/null
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java
@@ -0,0 +1,24 @@
+package org.apache.pinot.controller.helix.core.listener;
+
+import org.apache.helix.NotificationContext;
+import org.apache.helix.api.listeners.InstanceConfigChangeListener;
+import org.apache.helix.model.InstanceConfig;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ClusterInstanceConfigChangeListener implements InstanceConfigChangeListener {
+    private List<InstanceConfig> _instanceConfigs = new ArrayList<>();
+
+    public ClusterInstanceConfigChangeListener() {
+    }
+
+    @Override
+    public void onInstanceConfigChange(List<InstanceConfig> instanceConfigs, NotificationContext context) {
+        _instanceConfigs = instanceConfigs;
+    }
+
+    public List<InstanceConfig> getInstanceConfigs() {
+        return _instanceConfigs;
+    }
+}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java
new file mode 100644
index 0000000..777b41a
--- /dev/null
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java
@@ -0,0 +1,21 @@
+package org.apache.pinot.controller.helix.core.listener;
+
+import java.util.ArrayList;
+import org.apache.helix.NotificationContext;
+import org.apache.helix.api.listeners.LiveInstanceChangeListener;
+import org.apache.helix.model.LiveInstance;
+
+import java.util.List;
+
+public class ClusterLiveInstanceChangeListener implements LiveInstanceChangeListener {
+    private List<LiveInstance> _liveInstances = new ArrayList<>();
+
+    @Override
+    public void onLiveInstanceChange(List<LiveInstance> liveInstances, NotificationContext changeContext) {
+        _liveInstances = liveInstances;
+    }
+
+    public List<LiveInstance> getLiveInstances() {
+        return _liveInstances;
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 03/04: Add change for Helix Configs

Posted by kh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kharekartik pushed a commit to branch controller_api_patch
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit e30ffb9ecb2be33ad5e1d6161deb838884b08fe1
Author: Kartik Khare <kh...@Kartiks-MacBook-Pro.local>
AuthorDate: Wed Sep 16 01:08:33 2020 +0530

    Add change for Helix Configs
---
 .../ClusterExternalViewChangeListener.java         | 21 -----------------
 .../ClusterInstanceConfigChangeListener.java       | 10 +++++++-
 .../ClusterLiveInstanceChangeListener.java         | 27 ++++++++++++++++------
 3 files changed, 29 insertions(+), 29 deletions(-)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterExternalViewChangeListener.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterExternalViewChangeListener.java
deleted file mode 100644
index 6400f47..0000000
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterExternalViewChangeListener.java
+++ /dev/null
@@ -1,21 +0,0 @@
-package org.apache.pinot.controller.helix.core.listener;
-
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.helix.NotificationContext;
-import org.apache.helix.api.listeners.ExternalViewChangeListener;
-import org.apache.helix.model.ExternalView;
-
-
-public class ClusterExternalViewChangeListener implements ExternalViewChangeListener {
-  private List<ExternalView> _externalViewList = new ArrayList<>();
-
-  @Override
-  public void onExternalViewChange(List<ExternalView> externalViewList, NotificationContext changeContext) {
-    _externalViewList = externalViewList;
-  }
-
-  public List<ExternalView> getExternalViewList() {
-    return _externalViewList;
-  }
-}
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java
index 940230c..88bdd40 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterInstanceConfigChangeListener.java
@@ -1,16 +1,21 @@
 package org.apache.pinot.controller.helix.core.listener;
 
+import org.apache.helix.HelixManager;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.api.listeners.InstanceConfigChangeListener;
 import org.apache.helix.model.InstanceConfig;
 
 import java.util.ArrayList;
 import java.util.List;
+import org.apache.pinot.common.utils.helix.HelixHelper;
+
 
 public class ClusterInstanceConfigChangeListener implements InstanceConfigChangeListener {
+    private HelixManager _helixManager;
     private List<InstanceConfig> _instanceConfigs = new ArrayList<>();
 
-    public ClusterInstanceConfigChangeListener() {
+    public ClusterInstanceConfigChangeListener(HelixManager helixManager) {
+        _helixManager = helixManager;
     }
 
     @Override
@@ -19,6 +24,9 @@ public class ClusterInstanceConfigChangeListener implements InstanceConfigChange
     }
 
     public List<InstanceConfig> getInstanceConfigs() {
+        if(_instanceConfigs.isEmpty()){
+            _instanceConfigs = HelixHelper.getInstanceConfigs(_helixManager);
+        }
         return _instanceConfigs;
     }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java
index 777b41a..e3623c7 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/listener/ClusterLiveInstanceChangeListener.java
@@ -1,21 +1,34 @@
 package org.apache.pinot.controller.helix.core.listener;
 
 import java.util.ArrayList;
+import org.apache.helix.HelixDataAccessor;
 import org.apache.helix.NotificationContext;
 import org.apache.helix.api.listeners.LiveInstanceChangeListener;
 import org.apache.helix.model.LiveInstance;
 
 import java.util.List;
+import org.apache.helix.PropertyKey.Builder;
+
 
 public class ClusterLiveInstanceChangeListener implements LiveInstanceChangeListener {
-    private List<LiveInstance> _liveInstances = new ArrayList<>();
+  private HelixDataAccessor _helixDataAccessor;
+  private Builder _keyBuilder;
+  private List<LiveInstance> _liveInstances = new ArrayList<>();
 
-    @Override
-    public void onLiveInstanceChange(List<LiveInstance> liveInstances, NotificationContext changeContext) {
-        _liveInstances = liveInstances;
-    }
+  public ClusterLiveInstanceChangeListener(HelixDataAccessor helixDataAccessor, Builder keyBuilder) {
+    _helixDataAccessor = helixDataAccessor;
+    _keyBuilder = keyBuilder;
+  }
+
+  @Override
+  public void onLiveInstanceChange(List<LiveInstance> liveInstances, NotificationContext changeContext) {
+    _liveInstances = liveInstances;
+  }
 
-    public List<LiveInstance> getLiveInstances() {
-        return _liveInstances;
+  public List<LiveInstance> getLiveInstances() {
+    if (_liveInstances.isEmpty()) {
+      _liveInstances = _helixDataAccessor.getProperty(_keyBuilder.liveInstances());
     }
+    return _liveInstances;
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[incubator-pinot] 04/04: change in constructors of listeners

Posted by kh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kharekartik pushed a commit to branch controller_api_patch
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 36babc3ef54be37809e8b0e14119080a3aada5ca
Author: Kartik Khare <kh...@Kartiks-MacBook-Pro.local>
AuthorDate: Wed Sep 16 01:10:38 2020 +0530

    change in constructors of listeners
---
 .../controller/helix/core/PinotHelixResourceManager.java    | 13 ++-----------
 1 file changed, 2 insertions(+), 11 deletions(-)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
index 917a777..27920db 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java
@@ -104,7 +104,6 @@ import org.apache.pinot.controller.helix.core.assignment.instance.InstanceAssign
 import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignment;
 import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentFactory;
 import org.apache.pinot.controller.helix.core.assignment.segment.SegmentAssignmentUtils;
-import org.apache.pinot.controller.helix.core.listener.ClusterExternalViewChangeListener;
 import org.apache.pinot.controller.helix.core.listener.ClusterInstanceConfigChangeListener;
 import org.apache.pinot.controller.helix.core.listener.ClusterLiveInstanceChangeListener;
 import org.apache.pinot.controller.helix.core.realtime.PinotLLCRealtimeSegmentManager;
@@ -166,7 +165,6 @@ public class PinotHelixResourceManager {
   private TableCache _tableCache;
   private ClusterInstanceConfigChangeListener _clusterInstanceConfigChangeListener;
   private ClusterLiveInstanceChangeListener _clusterLiveInstanceChangeListener;
-  private ClusterExternalViewChangeListener _clusterExternalViewChangeListener;
 
   public PinotHelixResourceManager(String zkURL, String helixClusterName, @Nullable String dataDir,
       long externalViewOnlineToOfflineTimeoutMillis, boolean isSingleTenantCluster, boolean enableBatchMessageMode,
@@ -231,13 +229,11 @@ public class PinotHelixResourceManager {
         .parseBoolean(configs.get(Helix.DEPRECATED_ENABLE_CASE_INSENSITIVE_KEY));
     _tableCache = new TableCache(_propertyStore, caseInsensitive);
 
-    _clusterInstanceConfigChangeListener = new ClusterInstanceConfigChangeListener();
-    _clusterLiveInstanceChangeListener = new ClusterLiveInstanceChangeListener();
-    _clusterExternalViewChangeListener = new ClusterExternalViewChangeListener();
+    _clusterInstanceConfigChangeListener = new ClusterInstanceConfigChangeListener(_helixZkManager);
+    _clusterLiveInstanceChangeListener = new ClusterLiveInstanceChangeListener(_helixDataAccessor, _keyBuilder);
     try {
       addConfigListeners(_clusterInstanceConfigChangeListener);
       addLiveInstanceListeners(_clusterLiveInstanceChangeListener);
-      addExternalViewListeners(_clusterExternalViewChangeListener);
     } catch (Exception e) {
       LOGGER.warn(
           "Unable to add config listener in controller. This will result in incorrect response from controller's broker API");
@@ -2215,11 +2211,6 @@ public class PinotHelixResourceManager {
     _helixZkManager.addLiveInstanceChangeListener(liveInstanceChangeListener);
   }
 
-  public void addExternalViewListeners(ExternalViewChangeListener externalViewChangeListener)
-      throws Exception {
-    _helixZkManager.addExternalViewChangeListener(externalViewChangeListener);
-  }
-
   /**
    * Provides admin endpoints for the provided data instances
    * @param instances instances for which to read endpoints


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org