You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ne...@apache.org on 2019/01/18 21:22:37 UTC

[incubator-pinot] branch master updated: Move instantiation of objects which depend on _helixResourceManager to after invocation of PinotHelixResourceManager::start (#3713)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e6dfe31  Move instantiation of objects which depend on _helixResourceManager to after invocation of PinotHelixResourceManager::start (#3713)
e6dfe31 is described below

commit e6dfe310c2d53fa4031d63ee5cb7015b29762d3e
Author: Neha Pawar <np...@linkedin.com>
AuthorDate: Fri Jan 18 13:22:32 2019 -0800

    Move instantiation of objects which depend on _helixResourceManager to after invocation of PinotHelixResourceManager::start (#3713)
    
    Certain objects which are instantiated in the ControllerStarter need an instance of PinotHelixResourceManager. Even though PinotHelixResourceManager is instantiated in the ControllerStarter constructor right at the beginning, it cannot be used until PinotHelixResourceManager::start is invoked. The start is invoked in ControllerStarter::start(). Every object that depends on PinotHelixResourceManager should be created after this invocation of start
    This PR moves all the dependents to after PinotHelixResourceManager being ready
---
 .../apache/pinot/controller/ControllerStarter.java | 71 +++++++++-------
 .../helix/ControllerPeriodicTaskStarterTest.java   | 97 ++++++++++++++++++++++
 .../pinot/controller/helix/ControllerTest.java     | 18 ++--
 3 files changed, 149 insertions(+), 37 deletions(-)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java b/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java
index 370a13c..5d1546e 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/ControllerStarter.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.controller;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.primitives.Longs;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.yammer.metrics.core.MetricsRegistry;
@@ -79,35 +80,31 @@ public class ControllerStarter {
   private final ControllerConf _config;
   private final ControllerAdminApiApplication _adminApp;
   private final PinotHelixResourceManager _helixResourceManager;
-  private final RetentionManager _retentionManager;
   private final MetricsRegistry _metricsRegistry;
   private final ControllerMetrics _controllerMetrics;
-  private final PinotRealtimeSegmentManager _realtimeSegmentsManager;
-  private final SegmentStatusChecker _segmentStatusChecker;
   private final ExecutorService _executorService;
-  private final ControllerPeriodicTaskScheduler _controllerPeriodicTaskScheduler;
 
   // Can only be constructed after resource manager getting started
   private OfflineSegmentIntervalChecker _offlineSegmentIntervalChecker;
   private RealtimeSegmentValidationManager _realtimeSegmentValidationManager;
   private BrokerResourceValidationManager _brokerResourceValidationManager;
   private RealtimeSegmentRelocator _realtimeSegmentRelocator;
-  private PinotHelixTaskResourceManager _helixTaskResourceManager;
+  private RetentionManager _retentionManager;
+  private SegmentStatusChecker _segmentStatusChecker;
   private PinotTaskManager _taskManager;
+  private ControllerPeriodicTaskScheduler _controllerPeriodicTaskScheduler;
+  private PinotHelixTaskResourceManager _helixTaskResourceManager;
+  private PinotRealtimeSegmentManager _realtimeSegmentsManager;
 
   public ControllerStarter(ControllerConf conf) {
     _config = conf;
     _adminApp = new ControllerAdminApiApplication(_config.getQueryConsoleWebappPath(), _config.getQueryConsoleUseHttps());
+    // Do not use this before the invocation of {@link PinotHelixResourceManager::start()}, which happens in {@link ControllerStarter::start()}
     _helixResourceManager = new PinotHelixResourceManager(_config);
-    _retentionManager = new RetentionManager(_helixResourceManager, _config);
     _metricsRegistry = new MetricsRegistry();
     _controllerMetrics = new ControllerMetrics(_metricsRegistry);
-    _realtimeSegmentsManager = new PinotRealtimeSegmentManager(_helixResourceManager);
     _executorService =
         Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("restapi-multiget-thread-%d").build());
-    _segmentStatusChecker = new SegmentStatusChecker(_helixResourceManager, _config, _controllerMetrics);
-    _realtimeSegmentRelocator = new RealtimeSegmentRelocator(_helixResourceManager, _config);
-    _controllerPeriodicTaskScheduler = new ControllerPeriodicTaskScheduler();
   }
 
   public PinotHelixResourceManager getHelixResourceManager() {
@@ -134,6 +131,8 @@ public class ControllerStarter {
     return _taskManager;
   }
 
+
+
   public void start() {
     LOGGER.info("Starting Pinot controller");
 
@@ -184,31 +183,14 @@ public class ControllerStarter {
     // Helix resource manager must be started in order to create PinotLLCRealtimeSegmentManager
     LOGGER.info("Starting realtime segment manager");
     PinotLLCRealtimeSegmentManager.create(_helixResourceManager, _config, _controllerMetrics);
+    _realtimeSegmentsManager = new PinotRealtimeSegmentManager(_helixResourceManager);
     _realtimeSegmentsManager.start(_controllerMetrics);
 
     // Setting up periodic tasks
-    LOGGER.info("Setting up periodic tasks");
-    List<PeriodicTask> periodicTasks = new ArrayList<>();
-    _taskManager = new PinotTaskManager(_helixTaskResourceManager, _helixResourceManager, _config, _controllerMetrics);
-    periodicTasks.add(_taskManager);
-    periodicTasks.add(_retentionManager);
-    _offlineSegmentIntervalChecker =
-        new OfflineSegmentIntervalChecker(_config, _helixResourceManager, new ValidationMetrics(_metricsRegistry));
-    _realtimeSegmentValidationManager =
-        new RealtimeSegmentValidationManager(_config, _helixResourceManager, PinotLLCRealtimeSegmentManager.getInstance(),
-            new ValidationMetrics(_metricsRegistry));
-    _brokerResourceValidationManager =
-        new BrokerResourceValidationManager(_config, _helixResourceManager);
-
-    periodicTasks.add(_offlineSegmentIntervalChecker);
-    periodicTasks.add(_realtimeSegmentValidationManager);
-    periodicTasks.add(_brokerResourceValidationManager);
-    periodicTasks.add(_segmentStatusChecker);
-    periodicTasks.add(_realtimeSegmentRelocator);
-
+    List<PeriodicTask> controllerPeriodicTasks = setupControllerPeriodicTasks();
     LOGGER.info("Init controller periodic tasks scheduler");
-    _controllerPeriodicTaskScheduler.init(periodicTasks);
-
+    _controllerPeriodicTaskScheduler = new ControllerPeriodicTaskScheduler();
+    _controllerPeriodicTaskScheduler.init(controllerPeriodicTasks);
 
     LOGGER.info("Creating rebalance segments factory");
     RebalanceSegmentStrategyFactory.createInstance(helixManager);
@@ -312,6 +294,32 @@ public class ControllerStarter {
     _controllerMetrics.initializeGlobalMeters();
   }
 
+  @VisibleForTesting
+  protected List<PeriodicTask> setupControllerPeriodicTasks() {
+    LOGGER.info("Setting up periodic tasks");
+    List<PeriodicTask> periodicTasks = new ArrayList<>();
+    _taskManager = new PinotTaskManager(_helixTaskResourceManager, _helixResourceManager, _config, _controllerMetrics);
+    periodicTasks.add(_taskManager);
+    _retentionManager = new RetentionManager(_helixResourceManager, _config);
+    periodicTasks.add(_retentionManager);
+    _offlineSegmentIntervalChecker =
+        new OfflineSegmentIntervalChecker(_config, _helixResourceManager, new ValidationMetrics(_metricsRegistry));
+    periodicTasks.add(_offlineSegmentIntervalChecker);
+    _realtimeSegmentValidationManager =
+        new RealtimeSegmentValidationManager(_config, _helixResourceManager, PinotLLCRealtimeSegmentManager.getInstance(),
+            new ValidationMetrics(_metricsRegistry));
+    periodicTasks.add(_realtimeSegmentValidationManager);
+    _brokerResourceValidationManager =
+        new BrokerResourceValidationManager(_config, _helixResourceManager);
+    periodicTasks.add(_brokerResourceValidationManager);
+    _segmentStatusChecker = new SegmentStatusChecker(_helixResourceManager, _config, _controllerMetrics);
+    periodicTasks.add(_segmentStatusChecker);
+    _realtimeSegmentRelocator = new RealtimeSegmentRelocator(_helixResourceManager, _config);
+    periodicTasks.add(_realtimeSegmentRelocator);
+
+    return periodicTasks;
+  }
+
   public void stop() {
     try {
       LOGGER.info("Stopping controller leadership manager");
@@ -330,7 +338,6 @@ public class ControllerStarter {
       LOGGER.info("Stopping realtime segment manager");
       _realtimeSegmentsManager.stop();
 
-
       LOGGER.info("Stopping resource manager");
       _helixResourceManager.stop();
 
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerPeriodicTaskStarterTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerPeriodicTaskStarterTest.java
new file mode 100644
index 0000000..9620139
--- /dev/null
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerPeriodicTaskStarterTest.java
@@ -0,0 +1,97 @@
+/**
+ * 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.
+ */
+package org.apache.pinot.controller.helix;
+
+import java.util.List;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.controller.ControllerStarter;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.core.periodictask.PeriodicTask;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+public class ControllerPeriodicTaskStarterTest extends ControllerTest {
+  private MockControllerStarter _mockControllerStarter;
+
+  @BeforeClass
+  public void setup() {
+    startZk();
+  }
+
+  /**
+   * Test that controller starts up and helixResourceManager is non null before initiating periodic tasks
+   */
+  @Test
+  public void testHelixResourceManagerDuringControllerStart() {
+    startController();
+  }
+
+  @AfterClass
+  public void teardown() {
+    stopController();
+  }
+
+  @Override
+  protected void startControllerStarter(ControllerConf config) {
+    _mockControllerStarter = new MockControllerStarter(config);
+    _mockControllerStarter.start();
+    _helixResourceManager = _mockControllerStarter.getHelixResourceManager();
+  }
+
+  @Override
+  protected void stopControllerStarter() {
+    Assert.assertNotNull(_mockControllerStarter);
+
+    _mockControllerStarter.stop();
+    _mockControllerStarter = null;
+  }
+
+  private class MockControllerStarter extends ControllerStarter {
+
+    private static final int NUM_PERIODIC_TASKS = 7;
+
+    private List<PeriodicTask> _controllerPeriodicTasks;
+
+    public MockControllerStarter(ControllerConf conf) {
+      super(conf);
+    }
+
+    @Override
+    protected List<PeriodicTask> setupControllerPeriodicTasks() {
+      PinotHelixResourceManager helixResourceManager = getHelixResourceManager();
+      Assert.assertNotNull(helixResourceManager);
+      Assert.assertNotNull(helixResourceManager.getHelixAdmin());
+      Assert.assertNotNull(helixResourceManager.getHelixZkManager());
+      Assert.assertNotNull(helixResourceManager.getHelixClusterName());
+      Assert.assertNotNull(helixResourceManager.getPropertyStore());
+
+      _controllerPeriodicTasks = super.setupControllerPeriodicTasks();
+      Assert.assertNotNull(_controllerPeriodicTasks);
+      Assert.assertEquals(_controllerPeriodicTasks.size(), NUM_PERIODIC_TASKS);
+      return _controllerPeriodicTasks;
+    }
+
+    List<PeriodicTask> getControllerPeriodicTasks() {
+      return _controllerPeriodicTasks;
+    }
+   }
+}
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java
index 9ed2859..a4ca263 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/ControllerTest.java
@@ -127,22 +127,30 @@ public abstract class ControllerTest {
       _zkClient.deleteRecursive("/" + helixClusterName);
     }
 
-    _controllerStarter = new ControllerStarter(config);
-    _controllerStarter.start();
+    startControllerStarter(config);
 
-    _helixResourceManager = _controllerStarter.getHelixResourceManager();
     _helixManager = _helixResourceManager.getHelixZkManager();
     _helixAdmin = _helixResourceManager.getHelixAdmin();
     _propertyStore = _helixResourceManager.getPropertyStore();
   }
 
+  protected void startControllerStarter(ControllerConf config) {
+    _controllerStarter = new ControllerStarter(config);
+    _controllerStarter.start();
+    _helixResourceManager = _controllerStarter.getHelixResourceManager();
+  }
+
   protected void stopController() {
+    stopControllerStarter();
+    FileUtils.deleteQuietly(new File(_controllerDataDir));
+    _zkClient.close();
+  }
+
+  protected void stopControllerStarter() {
     Assert.assertNotNull(_controllerStarter);
 
     _controllerStarter.stop();
     _controllerStarter = null;
-    FileUtils.deleteQuietly(new File(_controllerDataDir));
-    _zkClient.close();
   }
 
   protected Schema createDummySchema(String tableName) {


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