mcvsubbu commented on a change in pull request #7174:
URL: https://github.com/apache/pinot/pull/7174#discussion_r684666685



##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerMessageHandlerFactory.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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;
+
+import org.apache.helix.NotificationContext;
+import org.apache.helix.messaging.handling.HelixTaskResult;
+import org.apache.helix.messaging.handling.MessageHandler;
+import org.apache.helix.messaging.handling.MessageHandlerFactory;
+import org.apache.helix.model.Message;
+import org.apache.pinot.common.messages.RunPeriodicTaskMessage;
+import org.apache.pinot.core.periodictask.PeriodicTaskScheduler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** Factory class for creating message handlers for incoming helix messages. */
+public class ControllerMessageHandlerFactory implements MessageHandlerFactory {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ControllerMessageHandlerFactory.class);
+  private static final String USER_DEFINED_MSG_STRING = 
Message.MessageType.USER_DEFINE_MSG.toString();
+
+  private final PeriodicTaskScheduler _periodicTaskScheduler;
+
+  public ControllerMessageHandlerFactory(PeriodicTaskScheduler 
periodicTaskScheduler) {
+    _periodicTaskScheduler = periodicTaskScheduler;
+  }
+
+  @Override
+  public MessageHandler createHandler(Message message, NotificationContext 
notificationContext) {
+    String messageType = message.getMsgSubType();
+    if 
(messageType.equals(RunPeriodicTaskMessage.RUN_PERIODIC_TASK_MSG_SUB_TYPE)) {
+      return new RunPeriodicTaskMessageHandler(new 
RunPeriodicTaskMessage(message), notificationContext, _periodicTaskScheduler);
+    }
+
+    LOGGER.warn("Unknown message type {} received by controller. ", 
messageType);
+    return null;
+  }
+
+  @Override
+  public String getMessageType() {
+    return USER_DEFINED_MSG_STRING;
+  }
+
+  @Override
+  public void reset() {

Review comment:
       Why are we overriding this to do nothing?

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotControllerPeriodicTaskRestletResource.java
##########
@@ -0,0 +1,104 @@
+/**
+ * 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.api.resources;
+
+import io.swagger.annotations.Api;
+import io.swagger.annotations.ApiOperation;
+import io.swagger.annotations.ApiParam;
+import java.util.ArrayList;
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import org.apache.helix.ClusterMessagingService;
+import org.apache.helix.Criteria;
+import org.apache.helix.InstanceType;
+import org.apache.pinot.common.messages.RunPeriodicTaskMessage;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.core.periodictask.PeriodicTaskScheduler;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@Api(tags = Constants.PERIODIC_TASK_TAG)
+@Path("/periodictask")
+public class PinotControllerPeriodicTaskRestletResource {
+
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(PinotControllerPeriodicTaskRestletResource.class);
+
+  @Inject
+  PinotHelixResourceManager _pinotHelixResourceManager;
+
+  @Inject
+  PeriodicTaskScheduler _periodicTaskScheduler;
+
+  @GET
+  @Produces(MediaType.APPLICATION_JSON)
+  @Path("/run")
+  @ApiOperation(value = "Run controller periodic task against the specified 
table. If no table name is specified, task will run against all tables.")
+  public boolean runPeriodicTask(
+      @ApiParam(value = "Periodic Task Name", required = true) 
@QueryParam("taskname") String periodicTaskName,
+      @ApiParam(value = "Table Name (with type)", required = false) 
@QueryParam("tablename") String tableName) {
+    if (!_periodicTaskScheduler.hasTask(periodicTaskName)) {
+      throw new WebApplicationException("Periodic task '" + periodicTaskName + 
"' not found.",
+          Response.Status.NOT_FOUND);
+    }
+
+    LOGGER.info("Sending periodic task execution message to all controllers 
for running task {} against {}.",
+        periodicTaskName, tableName != null ? tableName + " table" : "all 
tables");

Review comment:
       ```suggestion
           periodicTaskName, tableName != null ? " table '" + tableName + "'" : 
"all tables");
   ```
   This will ensure that if by some chance the user specifies some weird 
characters, we can find out here.

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/periodictask/BasePeriodicTask.java
##########
@@ -149,27 +182,17 @@ public final synchronized void stop() {
     }
     _started = false;
 
-    if (_running) {
-      long startTimeMs = System.currentTimeMillis();
-      long remainingTimeMs = MAX_PERIODIC_TASK_STOP_TIME_MILLIS;
-      LOGGER.info("Task: {} is running, wait for at most {}ms for it to 
finish", _taskName, remainingTimeMs);
-      while (_running && remainingTimeMs > 0L) {
-        long sleepTimeMs = Long.min(remainingTimeMs, 1000L);
-        remainingTimeMs -= sleepTimeMs;
-        try {
-          Thread.sleep(sleepTimeMs);
-        } catch (InterruptedException e) {
-          LOGGER.error("Caught InterruptedException while waiting for task: {} 
to finish", _taskName);
-          Thread.currentThread().interrupt();
-          break;
-        }
-      }
-      long waitTimeMs = System.currentTimeMillis() - startTimeMs;
-      if (_running) {
-        LOGGER.warn("Task: {} is not finished in {}ms", waitTimeMs);
+    try {
+      // check if task is done running, or wait for the task to get done, by 
trying to acquire runLock.
+      if (!_runLock.tryLock(MAX_PERIODIC_TASK_STOP_TIME_MILLIS, 
TimeUnit.MILLISECONDS)) {
+        LOGGER.warn("Task: {} did not finish within timeout of {}ms", 
MAX_PERIODIC_TASK_STOP_TIME_MILLIS);
       } else {
-        LOGGER.info("Task: {} is finished in {}ms", waitTimeMs);
+        LOGGER.warn("Task: {} finished within timeout of {}ms", 
MAX_PERIODIC_TASK_STOP_TIME_MILLIS);

Review comment:
       I think `info` is ok here

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerMessageHandlerFactory.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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;
+
+import org.apache.helix.NotificationContext;
+import org.apache.helix.messaging.handling.HelixTaskResult;
+import org.apache.helix.messaging.handling.MessageHandler;
+import org.apache.helix.messaging.handling.MessageHandlerFactory;
+import org.apache.helix.model.Message;
+import org.apache.pinot.common.messages.RunPeriodicTaskMessage;
+import org.apache.pinot.core.periodictask.PeriodicTaskScheduler;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/** Factory class for creating message handlers for incoming helix messages. */
+public class ControllerMessageHandlerFactory implements MessageHandlerFactory {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(ControllerMessageHandlerFactory.class);
+  private static final String USER_DEFINED_MSG_STRING = 
Message.MessageType.USER_DEFINE_MSG.toString();
+
+  private final PeriodicTaskScheduler _periodicTaskScheduler;
+
+  public ControllerMessageHandlerFactory(PeriodicTaskScheduler 
periodicTaskScheduler) {
+    _periodicTaskScheduler = periodicTaskScheduler;
+  }
+
+  @Override
+  public MessageHandler createHandler(Message message, NotificationContext 
notificationContext) {
+    String messageType = message.getMsgSubType();
+    if 
(messageType.equals(RunPeriodicTaskMessage.RUN_PERIODIC_TASK_MSG_SUB_TYPE)) {
+      return new RunPeriodicTaskMessageHandler(new 
RunPeriodicTaskMessage(message), notificationContext, _periodicTaskScheduler);
+    }
+
+    LOGGER.warn("Unknown message type {} received by controller. ", 
messageType);

Review comment:
       If you find other messages erroring out, can you please make them a 
warning and not throw exception on bad message? It is not upgrade friendly to 
throw exception on bad messages. Thanks.

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/periodictask/BasePeriodicTask.java
##########
@@ -87,13 +94,16 @@ public final synchronized void start() {
       LOGGER.warn("Task: {} is already started", _taskName);
       return;
     }
-    _started = true;
 
     try {
       setUpTask();
     } catch (Exception e) {
       LOGGER.error("Caught exception while setting up task: {}", _taskName, e);
     }
+
+    // mark _started as true only after state has completely initialized, so 
that run method doesn't end up seeing

Review comment:
       ok




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

Reply via email to