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 19:27:25 UTC

[incubator-pinot] branch pinot_helix_resource_manager_start updated: Add test to check that we have valid _helixResourceManager before proceeding with periodic tasks

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

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


The following commit(s) were added to refs/heads/pinot_helix_resource_manager_start by this push:
     new b5390af  Add test to check that we have valid _helixResourceManager before proceeding with periodic tasks
b5390af is described below

commit b5390af8edb4ef19bb6608ebcc128d7379be2b43
Author: Neha Pawar <np...@linkedin.com>
AuthorDate: Fri Jan 18 11:27:14 2019 -0800

    Add test to check that we have valid _helixResourceManager before proceeding with periodic tasks
---
 .../apache/pinot/controller/ControllerStarter.java | 53 +++++++-----
 .../helix/ControllerPeriodicTaskStarterTest.java   | 97 ++++++++++++++++++++++
 .../pinot/controller/helix/ControllerTest.java     | 18 ++--
 3 files changed, 141 insertions(+), 27 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 7d9f3df..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;
@@ -130,6 +131,8 @@ public class ControllerStarter {
     return _taskManager;
   }
 
+
+
   public void start() {
     LOGGER.info("Starting Pinot controller");
 
@@ -184,30 +187,10 @@ public class ControllerStarter {
     _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);
-    _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);
-
+    List<PeriodicTask> controllerPeriodicTasks = setupControllerPeriodicTasks();
     LOGGER.info("Init controller periodic tasks scheduler");
     _controllerPeriodicTaskScheduler = new ControllerPeriodicTaskScheduler();
-    _controllerPeriodicTaskScheduler.init(periodicTasks);
+    _controllerPeriodicTaskScheduler.init(controllerPeriodicTasks);
 
     LOGGER.info("Creating rebalance segments factory");
     RebalanceSegmentStrategyFactory.createInstance(helixManager);
@@ -311,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");
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