mcvsubbu commented on a change in pull request #7174: URL: https://github.com/apache/pinot/pull/7174#discussion_r687260857
########## File path: pinot-core/src/main/java/org/apache/pinot/core/periodictask/PeriodicTaskScheduler.java ########## @@ -108,4 +111,59 @@ public synchronized void stop() { _tasksWithValidInterval.parallelStream().forEach(PeriodicTask::stop); } } + + /** @return true if task with given name exists; otherwise, false. */ + public boolean hasTask(String periodicTaskName) { + for (PeriodicTask task : _tasksWithValidInterval) { + if (task.getTaskName().equals(periodicTaskName)) { + return true; + } + } + + return false; + } + + /** @return List of tasks name that will run periodically. */ + public List<String> getTaskNameList() { + List<String> taskNameList = new ArrayList<>(); Review comment: why create a new list? why not just return a `Collections.unmodifiableList` of this same one? ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotControllerPeriodicTaskRestletResource.java ########## @@ -0,0 +1,96 @@ +/** + * 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.List; +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 a periodic task against 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", required = false) @QueryParam("tablename") String tableName, + @ApiParam(value = "Table type suffix", required = false, example = "OFFLINE|REALTIME", defaultValue = "OFFLINE") @QueryParam("tabletype") String tableType) { + if (!_periodicTaskScheduler.hasTask(periodicTaskName)) { + throw new WebApplicationException("Periodic task '" + periodicTaskName + "' not found.", + Response.Status.NOT_FOUND); + } + Review comment: please add code to send 404 if table is not found. ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java ########## @@ -54,14 +54,22 @@ public ControllerPeriodicTask(String taskName, long runFrequencyInSeconds, long _controllerMetrics = controllerMetrics; } + // Determine if this task can run on the specified table. Task can run on all tables for which the controller is lead + // if "tablename" property is not set. However, if "tablename" property is set (by calling the /periodictask/run + // controller API), then the task will only run on the specified by the "tablename" property key. + private boolean runTaskForTable(String tableNameWithType) { + return _leadControllerManager.isLeaderForTable(tableNameWithType) && (_activePeriodicTaskProperties == null + || ((String) _activePeriodicTaskProperties.get("tablename")).equalsIgnoreCase(tableNameWithType)); Review comment: ```suggestion || ((String) _activePeriodicTaskProperties.get("tablename")).equals(tableNameWithType)); ``` Table `FooBar` is different from table `fooBar` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/periodictask/BasePeriodicTask.java ########## @@ -111,22 +126,49 @@ protected void setUpTask() { */ @Override public final void run() { - _running = true; + try { + // Don't allow a task to run more than once at a time. + _runLock.lock(); + _running = true; - if (_started) { - long startTime = System.currentTimeMillis(); - LOGGER.info("Start running task: {}", _taskName); - try { - runTask(); - } catch (Exception e) { - LOGGER.error("Caught exception while running task: {}", _taskName, e); + if (_started) { + long startTime = System.currentTimeMillis(); + LOGGER.info("Start running task: {}", _taskName); + try { + runTask(); + } catch (Exception e) { + LOGGER.error("Caught exception while running task: {}", _taskName, e); + } + LOGGER.info("Finish running task: {} in {}ms", _taskName, System.currentTimeMillis() - startTime); + } else { + LOGGER.warn("Task: {} is skipped because it is not started or already stopped", _taskName); } - LOGGER.info("Finish running task: {} in {}ms", _taskName, System.currentTimeMillis() - startTime); - } else { - LOGGER.warn("Task: {} is skipped because it is not started or already stopped", _taskName); + + _running = false; + } finally { + _runLock.unlock(); + _running = false; } + } - _running = false; + @Override + public void run(@Nullable java.util.Properties periodicTaskProperties) { + Properties savedPeriodicTaskProperties = _activePeriodicTaskProperties; Review comment: ```suggestion Properties savedPeriodicTaskProperties; ``` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/periodictask/BasePeriodicTask.java ########## @@ -111,22 +126,49 @@ protected void setUpTask() { */ @Override public final void run() { - _running = true; + try { + // Don't allow a task to run more than once at a time. + _runLock.lock(); + _running = true; - if (_started) { - long startTime = System.currentTimeMillis(); - LOGGER.info("Start running task: {}", _taskName); - try { - runTask(); - } catch (Exception e) { - LOGGER.error("Caught exception while running task: {}", _taskName, e); + if (_started) { + long startTime = System.currentTimeMillis(); + LOGGER.info("Start running task: {}", _taskName); + try { + runTask(); + } catch (Exception e) { + LOGGER.error("Caught exception while running task: {}", _taskName, e); + } + LOGGER.info("Finish running task: {} in {}ms", _taskName, System.currentTimeMillis() - startTime); + } else { + LOGGER.warn("Task: {} is skipped because it is not started or already stopped", _taskName); } - LOGGER.info("Finish running task: {} in {}ms", _taskName, System.currentTimeMillis() - startTime); - } else { - LOGGER.warn("Task: {} is skipped because it is not started or already stopped", _taskName); + + _running = false; Review comment: redundant ########## File path: pinot-core/src/main/java/org/apache/pinot/core/periodictask/PeriodicTaskScheduler.java ########## @@ -108,4 +111,59 @@ public synchronized void stop() { _tasksWithValidInterval.parallelStream().forEach(PeriodicTask::stop); } } + + /** @return true if task with given name exists; otherwise, false. */ + public boolean hasTask(String periodicTaskName) { + for (PeriodicTask task : _tasksWithValidInterval) { + if (task.getTaskName().equals(periodicTaskName)) { + return true; + } + } + + return false; + } + + /** @return List of tasks name that will run periodically. */ + public List<String> getTaskNameList() { + List<String> taskNameList = new ArrayList<>(); + for (PeriodicTask task : _tasksWithValidInterval) { + taskNameList.add(task.getTaskName()); + } + + return taskNameList; + } + + private PeriodicTask getPeriodicTask(String periodicTaskName) { + for (PeriodicTask task : _tasksWithValidInterval) { + if (task.getTaskName().equals(periodicTaskName)) { + return task; + } + } + return null; + } + + /** Execute {@link PeriodicTask} immediately on the specified table. */ + public void scheduleNow(String periodicTaskName, @Nullable Properties periodicTaskProperties) { + // Each controller may have a slightly different list of periodic tasks if we add, remove, or rename periodic Review comment: Let us clarify that this is a scenario only during rollout of controllers ########## File path: pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/periodictask/ControllerPeriodicTask.java ########## @@ -54,14 +54,22 @@ public ControllerPeriodicTask(String taskName, long runFrequencyInSeconds, long _controllerMetrics = controllerMetrics; } + // Determine if this task can run on the specified table. Task can run on all tables for which the controller is lead + // if "tablename" property is not set. However, if "tablename" property is set (by calling the /periodictask/run + // controller API), then the task will only run on the specified by the "tablename" property key. + private boolean runTaskForTable(String tableNameWithType) { Review comment: ```suggestion private boolean shouldRunTaskForTable(String tableNameWithType) { ``` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/periodictask/PeriodicTaskScheduler.java ########## @@ -108,4 +111,59 @@ public synchronized void stop() { _tasksWithValidInterval.parallelStream().forEach(PeriodicTask::stop); } } + + /** @return true if task with given name exists; otherwise, false. */ + public boolean hasTask(String periodicTaskName) { + for (PeriodicTask task : _tasksWithValidInterval) { + if (task.getTaskName().equals(periodicTaskName)) { + return true; + } + } + + return false; + } + + /** @return List of tasks name that will run periodically. */ + public List<String> getTaskNameList() { + List<String> taskNameList = new ArrayList<>(); + for (PeriodicTask task : _tasksWithValidInterval) { + taskNameList.add(task.getTaskName()); + } + + return taskNameList; + } + + private PeriodicTask getPeriodicTask(String periodicTaskName) { + for (PeriodicTask task : _tasksWithValidInterval) { + if (task.getTaskName().equals(periodicTaskName)) { + return task; + } + } + return null; + } + + /** Execute {@link PeriodicTask} immediately on the specified table. */ + public void scheduleNow(String periodicTaskName, @Nullable Properties periodicTaskProperties) { + // Each controller may have a slightly different list of periodic tasks if we add, remove, or rename periodic + // task. To avoid this situation, we check again (besides the check at controller API level) whether the + // periodic task exists. + PeriodicTask periodicTask = getPeriodicTask(periodicTaskName); + if (periodicTask == null) { + throw new IllegalArgumentException("Unknown Periodic Task " + periodicTaskName); + } + + LOGGER.info("Immediately executing periodic task {}", periodicTaskName); Review comment: This is misleading if the task ends up waiting on a lock, and that could be a very long time. You may want to say "scheduling the task to run" -- 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