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



##########
File path: 
pinot-common/src/main/java/org/apache/pinot/common/messages/RunPeriodicTaskMessage.java
##########
@@ -0,0 +1,64 @@
+/**
+ * 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.common.messages;
+
+import com.google.common.base.Preconditions;
+import java.util.UUID;
+import javax.annotation.Nonnull;
+import org.apache.helix.ZNRecord;
+import org.apache.helix.model.Message;
+
+
+/**
+ * Upon receiving this message, Controller will execute the specified 
PeriodicTask against the tables for which it is
+ * the lead controller. The message is sent whenever API call for executing a 
PeriodicTask is invoked.
+ */
+public class RunPeriodicTaskMessage extends Message {
+  public static final String RUN_PERIODIC_TASK_MSG_SUB_TYPE = "PERIODIC_TASK";
+  private static final String PERIODIC_TASK_NAME_KEY = "periodicTaskName";
+  private static final String TABLE_NAME_KEY = "tableName";
+
+  /**
+   * @param periodicTaskName Name of the task that will be run.
+   * @param tableName Table name against which task will run.
+   */
+  public RunPeriodicTaskMessage(@Nonnull String periodicTaskName, String 
tableName) {
+    super(MessageType.USER_DEFINE_MSG, UUID.randomUUID().toString());
+    setMsgSubType(RUN_PERIODIC_TASK_MSG_SUB_TYPE);
+    setExecutionTimeout(-1);
+    ZNRecord znRecord = getRecord();
+    znRecord.setSimpleField(PERIODIC_TASK_NAME_KEY, periodicTaskName);
+    znRecord.setSimpleField(TABLE_NAME_KEY, tableName);
+  }
+
+  public RunPeriodicTaskMessage(Message message) {
+    super(message.getRecord());
+    String msgSubType = message.getMsgSubType();
+    
Preconditions.checkArgument(msgSubType.equals(RUN_PERIODIC_TASK_MSG_SUB_TYPE),

Review comment:
       Isnt this check redundant?

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerMessageHandlerFactory.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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 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.error("Bad message type {} received by controller. ", messageType);

Review comment:
       ```suggestion
       LOGGER.warn("Unknown message type {} received by controller. ", 
messageType);
   ```
   
   We may add new message types later, and not be able to upgrade all 
controllers at the same time to understand the new message. Better to drop any 
unknown message with a warning or even just INFO. It is a valid upgrade use case

##########
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:
       Hmm.. The run() method should never be called unless the start() method 
has completely initialized, right? Or, are you trying to handle the case where 
we are trying to start the task after a stop() has been called? In case of 
latter, please keep a variable called  `_stopped`

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/periodictask/PeriodicTaskScheduler.java
##########
@@ -108,4 +108,48 @@ 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;
+  }
+
+  private PeriodicTask getPeriodicTask(String periodicTaskName) {
+    for (PeriodicTask task : _tasksWithValidInterval) {
+      if (task.getTaskName().equals(periodicTaskName)) {
+        return task;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Execute specified {@link PeriodicTask} immediately. If the task is 
already running, wait for the running task
+   * to finish before executing the task again.
+   */
+  public void execute(String periodicTaskName, String tableName) {
+    PeriodicTask periodicTask = getPeriodicTask(periodicTaskName);
+    LOGGER.info("Immediately executing periodic task {}", periodicTaskName);

Review comment:
       A misleading log if periodicTaskName is null (as per check below), so 
move it at least one line down. Include table name in the log if given (or 
include "all tables" in the log).
   
   A rare corner case can exist here where we rename a periodic task (or 
add/remove periodic tasks) in some release, and each controller has a slightly 
different list of periodic tasks. So, addition to returning 4xx at the API, it 
is good to check at the destination controllers as well.  Please add a comment 
to this effect so that someone reading the code may not wonder why we are 
checking it at the API level, and also here.
   

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerMessageHandlerFactory.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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 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.error("Bad message type {} received by controller. ", messageType);
+    return null;
+  }
+
+  @Override
+  public String getMessageType() {
+    return Message.MessageType.USER_DEFINE_MSG.toString();
+  }
+
+  @Override
+  public void reset() {
+  }
+
+  /** Message handler for "Run Periodic Task" message. */
+  private static class RunPeriodicTaskMessageHandler extends MessageHandler {
+    private final String _periodicTaskName;
+    private final String _tableName;
+    private final PeriodicTaskScheduler _periodicTaskScheduler;
+
+    RunPeriodicTaskMessageHandler(RunPeriodicTaskMessage message, 
NotificationContext context, PeriodicTaskScheduler periodicTaskScheduler) {
+      super(message, context);
+      _periodicTaskName = message.getPeriodicTaskName();
+      _tableName = message.getTableName();
+      _periodicTaskScheduler = periodicTaskScheduler;
+    }
+
+    @Override
+    public HelixTaskResult handleMessage()
+        throws InterruptedException {
+      LOGGER.info("Handle RunPeriodicTaskMessage by executing task {}", 
_periodicTaskName);

Review comment:
       ```suggestion
         LOGGER.info("Handling RunPeriodicTaskMessage by executing task {}", 
_periodicTaskName);
   ```

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/ControllerMessageHandlerFactory.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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 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.error("Bad message type {} received by controller. ", messageType);
+    return null;
+  }
+
+  @Override
+  public String getMessageType() {
+    return Message.MessageType.USER_DEFINE_MSG.toString();

Review comment:
       private static?

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/periodictask/PeriodicTaskScheduler.java
##########
@@ -108,4 +108,48 @@ 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;
+  }
+
+  private PeriodicTask getPeriodicTask(String periodicTaskName) {
+    for (PeriodicTask task : _tasksWithValidInterval) {
+      if (task.getTaskName().equals(periodicTaskName)) {
+        return task;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Execute specified {@link PeriodicTask} immediately. If the task is 
already running, wait for the running task
+   * to finish before executing the task again.
+   */
+  public void execute(String periodicTaskName, String tableName) {

Review comment:
       ```suggestion
     public void execute(String periodicTaskName, @Nullable String tableName) {
   ```
   Also, suggest rename method to `scheduleNow()` ?

##########
File path: 
pinot-core/src/main/java/org/apache/pinot/core/periodictask/PeriodicTaskScheduler.java
##########
@@ -108,4 +108,48 @@ 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;
+  }
+
+  private PeriodicTask getPeriodicTask(String periodicTaskName) {
+    for (PeriodicTask task : _tasksWithValidInterval) {
+      if (task.getTaskName().equals(periodicTaskName)) {
+        return task;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Execute specified {@link PeriodicTask} immediately. If the task is 
already running, wait for the running task
+   * to finish before executing the task again.
+   */
+  public void execute(String periodicTaskName, String tableName) {
+    PeriodicTask periodicTask = getPeriodicTask(periodicTaskName);
+    LOGGER.info("Immediately executing periodic task {}", periodicTaskName);
+    if (periodicTask == null) {
+      throw new IllegalArgumentException("Unknown Periodic Task " + 
periodicTaskName);
+    }
+
+    _executorService.schedule(() -> {
+      try {
+        // To prevent thread conflict, this call will block if the same task 
is already running (see

Review comment:
       TBD whether we should queue and re-run the task immediately after one 
run. I would suggest don't bother. If it is running, just return with an INFO 
message saying so.
   
   I am eager to hear the requirements here and what others have to say.

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotControllerPeriodicTaskRestletResource.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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 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", required = false) 
@QueryParam("tablename") String tableName) {
+    if (!_periodicTaskScheduler.hasTask(periodicTaskName)) {
+      throw new WebApplicationException("Periodic task '" + periodicTaskName + 
"' not found.",

Review comment:
       Will be good to return a list of valid names here. "Periodic task should 
be one of : ...." . That way, administrators need not look up the code to get 
task names.
   
   Or, add another API to list the task names is also OK. 
   
   Then we don't have to document the task names, and are free to change it 
across releases. just document the api to get the name of tasks.

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotControllerPeriodicTaskRestletResource.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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 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", 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 for {} to all 
controllers.", periodicTaskName);

Review comment:
       Add table name to this log message, if provided (otherwise include the 
string "for all tables")

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/MinionInstancesCleanupTask.java
##########
@@ -54,8 +54,11 @@ public MinionInstancesCleanupTask(PinotHelixResourceManager 
pinotHelixResourceMa
         
controllerConf.getMinionInstancesCleanupTaskMinOfflineTimeBeforeDeletionInSeconds()
 * 1000L;
   }
 
+  /**
+   * @param filter Currently not used, but can be used to specify how this 
task should be run.
+   */
   @Override
-  protected void runTask() {
+  protected void runTask(String filter) {

Review comment:
       Instead of adding a filter that is to be interpreted by each subclass, 
please add  `shouldRunForTable()` in base periodic task, and use that in all 
the sub-classes (that operate on a per-table basis). You can then add all logic 
there. 




-- 
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