You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "tibrewalpratik17 (via GitHub)" <gi...@apache.org> on 2024/02/21 10:43:41 UTC

[PR] [WIP] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

tibrewalpratik17 opened a new pull request, #12459:
URL: https://github.com/apache/pinot/pull/12459

   Iabel:
   - `feature`
   - `release-notes` (**)
   
   Currently, all minion-scheduled jobs run on `minion_untagged` node instaces. This doesn't allow isolation among tables / tasks at minion level.
   Adhoc-minion-tasks already acknowledge a task-config `minionInstanceTag` and schedule those adhoc-runs only on those set of instances. Extending the same for periodic-scheduled tasks.
   
   Resolving some TODOs / bugs in this PR itself where if `generateTasks` schedules tasks for some tables we still end up emitting a failed metric for all the tables. This PR makes the change of iterating through all tables and emitting metrics for each one of them.
   
   Will update the PR for review post-testing in our clusters.
   Raising a draft PR for any early feedback / review.


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] [WIP] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-1956434070

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/12459?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   All modified and coverable lines are covered by tests :white_check_mark:
   > Comparison is base [(`1d490c1`)](https://app.codecov.io/gh/apache/pinot/commit/1d490c1ac3268103a16d77ddfa70f8f8602f9e96?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 61.77% compared to head [(`6e80d6e`)](https://app.codecov.io/gh/apache/pinot/pull/12459?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 46.67%.
   > Report is 8 commits behind head on master.
   
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #12459       +/-   ##
   =============================================
   - Coverage     61.77%   46.67%   -15.10%     
   - Complexity      207      947      +740     
   =============================================
     Files          2436     1829      -607     
     Lines        133130    96603    -36527     
     Branches      20623    15650     -4973     
   =============================================
   - Hits          82241    45092    -37149     
   - Misses        44850    48268     +3418     
   + Partials       6039     3243     -2796     
   ```
   
   | [Flag](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration1](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [integration2](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [java-11](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [java-21](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.67% <ø> (-14.99%)` | :arrow_down: |
   | [skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.67% <ø> (-15.06%)` | :arrow_down: |
   | [skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [temurin](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.67% <ø> (-15.10%)` | :arrow_down: |
   | [unittests](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.67% <ø> (-15.10%)` | :arrow_down: |
   | [unittests1](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `46.67% <ø> (-0.23%)` | :arrow_down: |
   | [unittests2](https://app.codecov.io/gh/apache/pinot/pull/12459/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/pinot/pull/12459?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1534722096


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/BaseTaskGenerator.java:
##########
@@ -120,4 +123,23 @@ public List<PinotTaskConfig> generateTasks(TableConfig tableConfig, Map<String,
       throws Exception {
     throw new UnknownTaskTypeException("Adhoc task generation is not supported for task type - " + this.getTaskType());
   }
+
+  @Override
+  public void generateTasks(List<TableConfig> tableConfigs, List<PinotTaskConfig> pinotTaskConfigs)
+      throws Exception {
+    pinotTaskConfigs.addAll(generateTasks(tableConfigs));
+  }
+
+  @Override
+  public String getMinionInstanceTag(TableConfig tableConfig) {
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig != null) {
+      Map<String, String> configs = tableTaskConfig.getConfigsForTaskType(getTaskType());
+      if (!configs.isEmpty()) {
+        return configs.getOrDefault(PinotTaskManager.MINION_INSTANCE_TAG_CONFIG,

Review Comment:
   It's difficult to add in TableConfigUtils as we don't have HelixManager present in that class. 



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1517538480


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,88 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> pinotTaskConfig = taskGenerator.generateTasks(List.of(tableConfig));

Review Comment:
   Added this method in PinotTaskGenerator and BaseTaskGenerator.



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1540440738


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -615,12 +615,15 @@ public Map<String, String> scheduleTasks(@ApiParam(value = "Task type") @QueryPa
     tableName = DatabaseUtils.translateTableName(tableName, headers);

Review Comment:
   Would it also make sense to extend this API to take in `minionInstanceTag`? 
   I understand that `tasks/execute` API already provides that capability. 
   
   
   We plan to use this feature as follows: 
   * Say all minion periodic tasks are usually executed in the minion_untagged hosts.  
   * If our minion purge tasks are running behind for a table, we plan to deploy new set of hosts with a special tag e.g. `minion_emergency`, and use the schedule/execute API for one-off scheduling of these jobs on these hosts. 
   
   
   



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1546744519


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -615,12 +615,15 @@ public Map<String, String> scheduleTasks(@ApiParam(value = "Task type") @QueryPa
     tableName = DatabaseUtils.translateTableName(tableName, headers);

Review Comment:
   Hmm this is a good suggestion. Let me do it in a follow-up as this patch has been open from quite some time. Tracking improvements in #12698. Will add this one 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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "swaminathanmanish (via GitHub)" <gi...@apache.org>.
swaminathanmanish commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1529417960


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -483,15 +484,15 @@ public void registerTaskGenerator(PinotTaskGenerator taskGenerator) {
    * Public API to schedule tasks (all task types) for all tables. It might be called from the non-leader controller.
    * Returns a map from the task type to the task scheduled.

Review Comment:
   Can you update doc as well ? 



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,85 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> presentTaskConfig =
+            pinotMinionInstanceToTaskConfigs.computeIfAbsent(minionInstanceTag, k -> new ArrayList<>());
+        taskGenerator.generateTasks(List.of(tableConfig), presentTaskConfig);
+        pinotMinionInstanceToTaskConfigs.put(minionInstanceTag, presentTaskConfig);
+        long successRunTimestamp = System.currentTimeMillis();
+        _taskManagerStatusCache.saveTaskGeneratorInfo(tableName, taskType,
             taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addSuccessRunTs(successRunTimestamp));
         // before the first task schedule, the follow two gauge metrics will be empty
         // TODO: find a better way to report task generation information
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.TIME_MS_SINCE_LAST_SUCCESSFUL_MINION_TASK_GENERATION,
             () -> System.currentTimeMillis() - successRunTimestamp);
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.LAST_MINION_TASK_GENERATION_ENCOUNTERS_ERROR, 0L);
-      }
-    } catch (Exception e) {
-      StringWriter errors = new StringWriter();
-      try (PrintWriter pw = new PrintWriter(errors)) {
-        e.printStackTrace(pw);
-      }
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
-            taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addErrorRunMessage(successRunTimestamp,
+      } catch (Exception e) {
+        StringWriter errors = new StringWriter();
+        try (PrintWriter pw = new PrintWriter(errors)) {
+          e.printStackTrace(pw);
+        }
+        long failureRunTimestamp = System.currentTimeMillis();
+        _taskManagerStatusCache.saveTaskGeneratorInfo(tableName, taskType,
+            taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addErrorRunMessage(failureRunTimestamp,
                 errors.toString()));
         // before the first task schedule, the follow gauge metric will be empty
         // TODO: find a better way to report task generation information
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.LAST_MINION_TASK_GENERATION_ENCOUNTERS_ERROR, 1L);
+        LOGGER.error("Failed to generate tasks for task type {} for table {}", taskType, tableName, e);
       }
-      throw e;
     }
     if (!isLeader) {
       taskGenerator.nonLeaderCleanUp();
     }
-    String taskType = taskGenerator.getTaskType();
-    int numTasks = pinotTaskConfigs.size();
-    if (numTasks > 0) {
-      LOGGER.info("Submitting {} tasks for task type: {} with task configs: {}", numTasks, taskType, pinotTaskConfigs);
-      _controllerMetrics.addMeteredTableValue(taskType, ControllerMeter.NUMBER_TASKS_SUBMITTED, numTasks);
-      return _helixTaskResourceManager.submitTask(pinotTaskConfigs, taskGenerator.getTaskTimeoutMs(),
-          taskGenerator.getNumConcurrentTasksPerInstance(), taskGenerator.getMaxAttemptsPerTask());
-    }
-    LOGGER.info("No task to schedule for task type: {}", taskType);
-    return null;
+    int numErrorTasksScheduled = 0;
+    List<String> submittedTaskNames = new ArrayList<>();
+    for (String minionInstanceTag : pinotMinionInstanceToTaskConfigs.keySet()) {
+      List<PinotTaskConfig> pinotTaskConfigs = pinotMinionInstanceToTaskConfigs.get(minionInstanceTag);
+      int numTasks = pinotTaskConfigs.size();
+      try {
+        if (numTasks > 0) {
+          // This might lead to lot of logs, maybe sum it up and move outside the loop
+          LOGGER.info("Submitting {} tasks for task type: {} to minionInstance: {} with task configs: {}", numTasks,
+              taskType, minionInstanceTag, pinotTaskConfigs);
+          String submittedTaskName = _helixTaskResourceManager.submitTask(pinotTaskConfigs, minionInstanceTag,
+              taskGenerator.getTaskTimeoutMs(), taskGenerator.getNumConcurrentTasksPerInstance(),
+              taskGenerator.getMaxAttemptsPerTask());
+          submittedTaskNames.add(submittedTaskName);
+          _controllerMetrics.addMeteredTableValue(taskType, ControllerMeter.NUMBER_TASKS_SUBMITTED, numTasks);
+        }
+      } catch (Exception e) {
+        numErrorTasksScheduled++;
+        LOGGER.error("Failed to schedule task type {} with task configs: {}", taskType, pinotTaskConfigs, e);

Review Comment:
   Can you add minionInstanceTag as well?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/BaseTaskGenerator.java:
##########
@@ -120,4 +123,23 @@ public List<PinotTaskConfig> generateTasks(TableConfig tableConfig, Map<String,
       throws Exception {
     throw new UnknownTaskTypeException("Adhoc task generation is not supported for task type - " + this.getTaskType());
   }
+
+  @Override
+  public void generateTasks(List<TableConfig> tableConfigs, List<PinotTaskConfig> pinotTaskConfigs)
+      throws Exception {
+    pinotTaskConfigs.addAll(generateTasks(tableConfigs));
+  }
+
+  @Override
+  public String getMinionInstanceTag(TableConfig tableConfig) {
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig != null) {
+      Map<String, String> configs = tableTaskConfig.getConfigsForTaskType(getTaskType());
+      if (!configs.isEmpty()) {

Review Comment:
   Do we need null check here, if there are no tasks for a type ? 



##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java:
##########
@@ -1005,9 +1025,13 @@ public void testRealtimeTableProcessAllModeMultiLevelConcat()
     long[] expectedNumBucketsToProcess200Days = {0, 0, 1, 1, 0, 0, 1, 1};
     String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(tableName);
     int numTasks = 0;
-    for (String tasks = taskManager.scheduleTasks(realtimeTableName).get(MinionConstants.MergeRollupTask.TASK_TYPE);
-        tasks != null; tasks =
-        taskManager.scheduleTasks(realtimeTableName).get(MinionConstants.MergeRollupTask.TASK_TYPE), numTasks++) {
+    List<String> taskList;

Review Comment:
   Can we make sure that we test the default path as well? (untagged minion)



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/BaseTaskGenerator.java:
##########
@@ -120,4 +123,23 @@ public List<PinotTaskConfig> generateTasks(TableConfig tableConfig, Map<String,
       throws Exception {
     throw new UnknownTaskTypeException("Adhoc task generation is not supported for task type - " + this.getTaskType());
   }
+
+  @Override
+  public void generateTasks(List<TableConfig> tableConfigs, List<PinotTaskConfig> pinotTaskConfigs)
+      throws Exception {
+    pinotTaskConfigs.addAll(generateTasks(tableConfigs));
+  }
+
+  @Override
+  public String getMinionInstanceTag(TableConfig tableConfig) {
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig != null) {
+      Map<String, String> configs = tableTaskConfig.getConfigsForTaskType(getTaskType());
+      if (!configs.isEmpty()) {
+        return configs.getOrDefault(PinotTaskManager.MINION_INSTANCE_TAG_CONFIG,

Review Comment:
   Since the user has to manually fill this in, can we add table config validation to ensure that the minion tag configured here has actually been created? 
   
   If there are typos, it'll be some work to troubleshoot this, if not validated during config time. 



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "zhtaoxiang (via GitHub)" <gi...@apache.org>.
zhtaoxiang commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1542088076


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -531,78 +532,98 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
 
   /**
    * Helper method to schedule task with the given task generator for the given tables that have the task enabled.
-   * Returns the task name, or {@code null} if no task is scheduled.
+   * Returns the list of task names, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> minionInstanceTagToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> presentTaskConfig =
+            minionInstanceTagToTaskConfigs.computeIfAbsent(minionInstanceTag, k -> new ArrayList<>());
+        taskGenerator.generateTasks(List.of(tableConfig), presentTaskConfig);
+        minionInstanceTagToTaskConfigs.put(minionInstanceTag, presentTaskConfig);

Review Comment:
   This put is not needed,  `computeIfAbsent` will put the newly generated list into the map



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "swaminathanmanish (via GitHub)" <gi...@apache.org>.
swaminathanmanish commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2005348069

   > minionInstanceTag
   
   Just a thought to get clarity - 
   @Jackie-Jiang , @snleee, @tibrewalpratik17  - Why should minionInstanceTag be at  taskTypeConfigsMap instead of being a table level config ? Do we want isolation at the table level or task level? 


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2005385172

   @swaminathanmanish +1 on your point. I think that this feature naturally fits better to the `tenant` config?


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2035085026

   Unrelated test failure in `PulsarConsumerTest`. Merging this PR.


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "swaminathanmanish (via GitHub)" <gi...@apache.org>.
swaminathanmanish commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1529466472


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -667,13 +667,12 @@ public Map<String, Object> getCronSchedulerJobDetails(
   @Produces(MediaType.APPLICATION_JSON)
   @Authenticate(AccessType.UPDATE)
   @ApiOperation("Schedule tasks and return a map from task type to task name scheduled")
-  public Map<String, String> scheduleTasks(@ApiParam(value = "Task type") @QueryParam("taskType") String taskType,
+  public Map<String, List<String>> scheduleTasks(@ApiParam(value = "Task type") @QueryParam("taskType") String taskType,

Review Comment:
   Please update the doc as well



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "swaminathanmanish (via GitHub)" <gi...@apache.org>.
swaminathanmanish commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2014128786

   > > To get more clarity -
   > > @Jackie-Jiang , @snleee, @tibrewalpratik17 - Why should minionInstanceTag be at taskTypeConfigsMap instead of being a table level config ? Do we want isolation at the table level or task level?
   > > @swaminathanmanish +1 on your point. I think that this feature naturally fits better to the tenant config?
   > 
   > Yes agreed to have a `minionTenant` concept as a table level config. Based on a discussion here - [#11240 (comment)](https://github.com/apache/pinot/issues/11240#issuecomment-1662819235) we would also like to give the flexibility to override it as task-level too just in case there are lot of tasks for a table and we want to isolate those too or group similar tasks of different tables in dedicated minion-nodes (may be different SKUs based on tasks). I raised this patch to quickly give the option of isolating tasks first. Post this, can work on a patch to introduce `minionTenant` concept in Pinot. Anyways, since this task-level config will have higher precedence over table's tenant config, this change will be backward compatible even then. What do you all think about this?
   > 
   > Also, will address comments shortly and update the patch.
   
   Thanks a lot @tibrewalpratik17 . Would you be able to create a github issue to track this effort? 


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1517490304


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -667,13 +667,12 @@ public Map<String, Object> getCronSchedulerJobDetails(
   @Produces(MediaType.APPLICATION_JSON)
   @Authenticate(AccessType.UPDATE)
   @ApiOperation("Schedule tasks and return a map from task type to task name scheduled")
-  public Map<String, String> scheduleTasks(@ApiParam(value = "Task type") @QueryParam("taskType") String taskType,
+  public Map<String, List<String>> scheduleTasks(@ApiParam(value = "Task type") @QueryParam("taskType") String taskType,

Review Comment:
   Updated to comma separated string



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1531205626


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,85 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> presentTaskConfig =
+            pinotMinionInstanceToTaskConfigs.computeIfAbsent(minionInstanceTag, k -> new ArrayList<>());
+        taskGenerator.generateTasks(List.of(tableConfig), presentTaskConfig);
+        pinotMinionInstanceToTaskConfigs.put(minionInstanceTag, presentTaskConfig);
+        long successRunTimestamp = System.currentTimeMillis();
+        _taskManagerStatusCache.saveTaskGeneratorInfo(tableName, taskType,
             taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addSuccessRunTs(successRunTimestamp));
         // before the first task schedule, the follow two gauge metrics will be empty
         // TODO: find a better way to report task generation information
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.TIME_MS_SINCE_LAST_SUCCESSFUL_MINION_TASK_GENERATION,
             () -> System.currentTimeMillis() - successRunTimestamp);
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.LAST_MINION_TASK_GENERATION_ENCOUNTERS_ERROR, 0L);
-      }
-    } catch (Exception e) {
-      StringWriter errors = new StringWriter();
-      try (PrintWriter pw = new PrintWriter(errors)) {
-        e.printStackTrace(pw);
-      }
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
-            taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addErrorRunMessage(successRunTimestamp,
+      } catch (Exception e) {
+        StringWriter errors = new StringWriter();
+        try (PrintWriter pw = new PrintWriter(errors)) {
+          e.printStackTrace(pw);
+        }
+        long failureRunTimestamp = System.currentTimeMillis();
+        _taskManagerStatusCache.saveTaskGeneratorInfo(tableName, taskType,
+            taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addErrorRunMessage(failureRunTimestamp,
                 errors.toString()));
         // before the first task schedule, the follow gauge metric will be empty
         // TODO: find a better way to report task generation information
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.LAST_MINION_TASK_GENERATION_ENCOUNTERS_ERROR, 1L);
+        LOGGER.error("Failed to generate tasks for task type {} for table {}", taskType, tableName, e);
       }
-      throw e;
     }
     if (!isLeader) {
       taskGenerator.nonLeaderCleanUp();
     }
-    String taskType = taskGenerator.getTaskType();
-    int numTasks = pinotTaskConfigs.size();
-    if (numTasks > 0) {
-      LOGGER.info("Submitting {} tasks for task type: {} with task configs: {}", numTasks, taskType, pinotTaskConfigs);
-      _controllerMetrics.addMeteredTableValue(taskType, ControllerMeter.NUMBER_TASKS_SUBMITTED, numTasks);
-      return _helixTaskResourceManager.submitTask(pinotTaskConfigs, taskGenerator.getTaskTimeoutMs(),
-          taskGenerator.getNumConcurrentTasksPerInstance(), taskGenerator.getMaxAttemptsPerTask());
-    }
-    LOGGER.info("No task to schedule for task type: {}", taskType);
-    return null;
+    int numErrorTasksScheduled = 0;
+    List<String> submittedTaskNames = new ArrayList<>();
+    for (String minionInstanceTag : pinotMinionInstanceToTaskConfigs.keySet()) {
+      List<PinotTaskConfig> pinotTaskConfigs = pinotMinionInstanceToTaskConfigs.get(minionInstanceTag);
+      int numTasks = pinotTaskConfigs.size();
+      try {
+        if (numTasks > 0) {
+          // This might lead to lot of logs, maybe sum it up and move outside the loop
+          LOGGER.info("Submitting {} tasks for task type: {} to minionInstance: {} with task configs: {}", numTasks,
+              taskType, minionInstanceTag, pinotTaskConfigs);
+          String submittedTaskName = _helixTaskResourceManager.submitTask(pinotTaskConfigs, minionInstanceTag,

Review Comment:
   Hey @snleee yess for adhoc jobs we have this support already using `minionInstanceTag`. In this i am extending the support to scheduled periodic jobs too.



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1540440738


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -615,12 +615,15 @@ public Map<String, String> scheduleTasks(@ApiParam(value = "Task type") @QueryPa
     tableName = DatabaseUtils.translateTableName(tableName, headers);

Review Comment:
   Would it also make sense to extend this API to take in a  `minionInstanceTag`? 
   I understand that `tasks/execute` API already provides that capability. 
   
   
   We plan to use this feature as follows: 
   * Say all minion periodic tasks are usually executed in the minion_untagged hosts.  
   * If our minion purge tasks are running behind for a table, we plan to deploy new set of hosts with a special tag e.g. `minion_emergency`, and use the schedule/execute API for one-off scheduling of these jobs on these hosts. 
   
   
   



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1535292589


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/BaseTaskGenerator.java:
##########
@@ -120,4 +123,23 @@ public List<PinotTaskConfig> generateTasks(TableConfig tableConfig, Map<String,
       throws Exception {
     throw new UnknownTaskTypeException("Adhoc task generation is not supported for task type - " + this.getTaskType());
   }
+
+  @Override
+  public void generateTasks(List<TableConfig> tableConfigs, List<PinotTaskConfig> pinotTaskConfigs)
+      throws Exception {
+    pinotTaskConfigs.addAll(generateTasks(tableConfigs));
+  }
+
+  @Override
+  public String getMinionInstanceTag(TableConfig tableConfig) {
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig != null) {
+      Map<String, String> configs = tableTaskConfig.getConfigsForTaskType(getTaskType());
+      if (!configs.isEmpty()) {
+        return configs.getOrDefault(PinotTaskManager.MINION_INSTANCE_TAG_CONFIG,

Review Comment:
   Actually we do in PinotHelixResourceManager class - https://github.com/apache/pinot/blob/af0e5b82baf196f1bcea40d9737e3d7926cf9958/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java#L1781-L1785
   
   But I think we should refactor to have it in the TableConfigUtils class. Tracking it here: #12698 



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2008285510

   > To get more clarity -
   @Jackie-Jiang , @snleee, @tibrewalpratik17 - Why should minionInstanceTag be at taskTypeConfigsMap instead of being a table level config ? Do we want isolation at the table level or task level?
   @swaminathanmanish +1 on your point. I think that this feature naturally fits better to the tenant config?
   
   Yes agreed to have a `minionTenant` concept as a table level config. Based on a discussion here - https://github.com/apache/pinot/issues/11240#issuecomment-1662819235 we would also like to give the flexibility to override it as task-level too just in case there are lot of tasks for a table and we want to isolate those too or group similar tasks of different tables in dedicated minion-nodes (may be different SKUs based on tasks). 
   I raised this patch to quickly give the option of isolating tasks first. Post this, can work on a patch to introduce `minionTenant` concept in Pinot. Anyways, since this task-level config will have higher precedence over table's tenant config, this change will be backward compatible even then. What do you all think about this?
   
   Also, will address comments shortly and update the patch.
   


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1508329145


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,88 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> pinotTaskConfig = taskGenerator.generateTasks(List.of(tableConfig));

Review Comment:
   This changes the task generation logic to processing one table at a time, which means we cannot apply constraints across tables (e.g. limit total number of tasks for a given task type). One way to solve this is to add an API `void generateTasks(TableConfig tableConfig, List<PinotTaskConfig> taskConfigs)`, where we also pass in the already generated task configs. When new task config is generated, we directly append it to this list.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -667,13 +667,12 @@ public Map<String, Object> getCronSchedulerJobDetails(
   @Produces(MediaType.APPLICATION_JSON)
   @Authenticate(AccessType.UPDATE)
   @ApiOperation("Schedule tasks and return a map from task type to task name scheduled")
-  public Map<String, String> scheduleTasks(@ApiParam(value = "Task type") @QueryParam("taskType") String taskType,
+  public Map<String, List<String>> scheduleTasks(@ApiParam(value = "Task type") @QueryParam("taskType") String taskType,

Review Comment:
   This is backward incompatible. I guess we can return a comma separated string instead so that the response is still compatible



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,88 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> pinotTaskConfig = taskGenerator.generateTasks(List.of(tableConfig));
+        List<PinotTaskConfig> presentTaskConfig = pinotMinionInstanceToTaskConfigs.get(minionInstanceTag);
+        if (presentTaskConfig == null || presentTaskConfig.isEmpty()) {
+          presentTaskConfig = new ArrayList<>();
+        }
+        presentTaskConfig.addAll(pinotTaskConfig);
+        pinotMinionInstanceToTaskConfigs.put(minionInstanceTag, presentTaskConfig);

Review Comment:
   ```suggestion
           List<PinotTaskConfig> presentTaskConfig = pinotMinionInstanceToTaskConfigs.computeIfAbsent(minionInstanceTag, k -> new ArrayList<>());
           presentTaskConfig.addAll(pinotTaskConfig);
   ```



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1546746797


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -531,78 +532,98 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
 
   /**
    * Helper method to schedule task with the given task generator for the given tables that have the task enabled.
-   * Returns the task name, or {@code null} if no task is scheduled.
+   * Returns the list of task names, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> minionInstanceTagToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> presentTaskConfig =
+            minionInstanceTagToTaskConfigs.computeIfAbsent(minionInstanceTag, k -> new ArrayList<>());
+        taskGenerator.generateTasks(List.of(tableConfig), presentTaskConfig);
+        minionInstanceTagToTaskConfigs.put(minionInstanceTag, presentTaskConfig);

Review Comment:
   We are appending to the list we get from "computeIfAbsent" and putting it back.



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "vvivekiyer (via GitHub)" <gi...@apache.org>.
vvivekiyer merged PR #12459:
URL: https://github.com/apache/pinot/pull/12459


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "snleee (via GitHub)" <gi...@apache.org>.
snleee commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1529520945


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,85 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();
+    String taskType = taskGenerator.getTaskType();
+    for (TableConfig tableConfig : enabledTableConfigs) {
+      String tableName = tableConfig.getTableName();
+      try {
+        String minionInstanceTag = taskGenerator.getMinionInstanceTag(tableConfig);
+        List<PinotTaskConfig> presentTaskConfig =
+            pinotMinionInstanceToTaskConfigs.computeIfAbsent(minionInstanceTag, k -> new ArrayList<>());
+        taskGenerator.generateTasks(List.of(tableConfig), presentTaskConfig);
+        pinotMinionInstanceToTaskConfigs.put(minionInstanceTag, presentTaskConfig);
+        long successRunTimestamp = System.currentTimeMillis();
+        _taskManagerStatusCache.saveTaskGeneratorInfo(tableName, taskType,
             taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addSuccessRunTs(successRunTimestamp));
         // before the first task schedule, the follow two gauge metrics will be empty
         // TODO: find a better way to report task generation information
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.TIME_MS_SINCE_LAST_SUCCESSFUL_MINION_TASK_GENERATION,
             () -> System.currentTimeMillis() - successRunTimestamp);
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.LAST_MINION_TASK_GENERATION_ENCOUNTERS_ERROR, 0L);
-      }
-    } catch (Exception e) {
-      StringWriter errors = new StringWriter();
-      try (PrintWriter pw = new PrintWriter(errors)) {
-        e.printStackTrace(pw);
-      }
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
-            taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addErrorRunMessage(successRunTimestamp,
+      } catch (Exception e) {
+        StringWriter errors = new StringWriter();
+        try (PrintWriter pw = new PrintWriter(errors)) {
+          e.printStackTrace(pw);
+        }
+        long failureRunTimestamp = System.currentTimeMillis();
+        _taskManagerStatusCache.saveTaskGeneratorInfo(tableName, taskType,
+            taskGeneratorMostRecentRunInfo -> taskGeneratorMostRecentRunInfo.addErrorRunMessage(failureRunTimestamp,
                 errors.toString()));
         // before the first task schedule, the follow gauge metric will be empty
         // TODO: find a better way to report task generation information
-        _controllerMetrics.setOrUpdateTableGauge(tableConfig.getTableName(), taskGenerator.getTaskType(),
+        _controllerMetrics.setOrUpdateTableGauge(tableName, taskType,
             ControllerGauge.LAST_MINION_TASK_GENERATION_ENCOUNTERS_ERROR, 1L);
+        LOGGER.error("Failed to generate tasks for task type {} for table {}", taskType, tableName, e);
       }
-      throw e;
     }
     if (!isLeader) {
       taskGenerator.nonLeaderCleanUp();
     }
-    String taskType = taskGenerator.getTaskType();
-    int numTasks = pinotTaskConfigs.size();
-    if (numTasks > 0) {
-      LOGGER.info("Submitting {} tasks for task type: {} with task configs: {}", numTasks, taskType, pinotTaskConfigs);
-      _controllerMetrics.addMeteredTableValue(taskType, ControllerMeter.NUMBER_TASKS_SUBMITTED, numTasks);
-      return _helixTaskResourceManager.submitTask(pinotTaskConfigs, taskGenerator.getTaskTimeoutMs(),
-          taskGenerator.getNumConcurrentTasksPerInstance(), taskGenerator.getMaxAttemptsPerTask());
-    }
-    LOGGER.info("No task to schedule for task type: {}", taskType);
-    return null;
+    int numErrorTasksScheduled = 0;
+    List<String> submittedTaskNames = new ArrayList<>();
+    for (String minionInstanceTag : pinotMinionInstanceToTaskConfigs.keySet()) {
+      List<PinotTaskConfig> pinotTaskConfigs = pinotMinionInstanceToTaskConfigs.get(minionInstanceTag);
+      int numTasks = pinotTaskConfigs.size();
+      try {
+        if (numTasks > 0) {
+          // This might lead to lot of logs, maybe sum it up and move outside the loop
+          LOGGER.info("Submitting {} tasks for task type: {} to minionInstance: {} with task configs: {}", numTasks,
+              taskType, minionInstanceTag, pinotTaskConfigs);
+          String submittedTaskName = _helixTaskResourceManager.submitTask(pinotTaskConfigs, minionInstanceTag,

Review Comment:
   We already support submitting task with `minionInstanceTag`?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,85 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.

Review Comment:
   Let's update the documentation here as we are changing the returning data type



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotTaskManager.java:
##########
@@ -534,66 +535,85 @@ private synchronized Map<String, String> scheduleTasks(List<String> tableNamesWi
    * Returns the task name, or {@code null} if no task is scheduled.
    */
   @Nullable
-  private String scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
+  private List<String> scheduleTask(PinotTaskGenerator taskGenerator, List<TableConfig> enabledTableConfigs,
       boolean isLeader) {
     LOGGER.info("Trying to schedule task type: {}, isLeader: {}", taskGenerator.getTaskType(), isLeader);
-    List<PinotTaskConfig> pinotTaskConfigs;
-    try {
-      /* TODO taskGenerator may skip generating tasks for some of the tables being passed to it.
-        In that case, we should not be storing success timestamps for those table. Same with exceptions that should
-        only be associated with the table for which it was raised and not every eligible table. We can have the
-        generateTasks() return a list of TaskGeneratorMostRecentRunInfo for each table
-       */
-      pinotTaskConfigs = taskGenerator.generateTasks(enabledTableConfigs);
-      long successRunTimestamp = System.currentTimeMillis();
-      for (TableConfig tableConfig : enabledTableConfigs) {
-        _taskManagerStatusCache.saveTaskGeneratorInfo(tableConfig.getTableName(), taskGenerator.getTaskType(),
+    Map<String, List<PinotTaskConfig>> pinotMinionInstanceToTaskConfigs = new HashMap<>();

Review Comment:
   Let's rename this to `minionInstanceTagToTaskConfigs` since pinotMinionInstance and instance tag are different concept.



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "swaminathanmanish (via GitHub)" <gi...@apache.org>.
swaminathanmanish commented on code in PR #12459:
URL: https://github.com/apache/pinot/pull/12459#discussion_r1534912490


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/generator/BaseTaskGenerator.java:
##########
@@ -120,4 +123,23 @@ public List<PinotTaskConfig> generateTasks(TableConfig tableConfig, Map<String,
       throws Exception {
     throw new UnknownTaskTypeException("Adhoc task generation is not supported for task type - " + this.getTaskType());
   }
+
+  @Override
+  public void generateTasks(List<TableConfig> tableConfigs, List<PinotTaskConfig> pinotTaskConfigs)
+      throws Exception {
+    pinotTaskConfigs.addAll(generateTasks(tableConfigs));
+  }
+
+  @Override
+  public String getMinionInstanceTag(TableConfig tableConfig) {
+    TableTaskConfig tableTaskConfig = tableConfig.getTaskConfig();
+    if (tableTaskConfig != null) {
+      Map<String, String> configs = tableTaskConfig.getConfigsForTaskType(getTaskType());
+      if (!configs.isEmpty()) {
+        return configs.getOrDefault(PinotTaskManager.MINION_INSTANCE_TAG_CONFIG,

Review Comment:
   I see. It would be ideal to enforce some validation especially given that user has to enter this for every taskConfig.  Im curious if we have any validation for table tenant.
   Can we keep track of this work? 
    



-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2034023227

   hey @snleee @vvivekiyer can one of you help me merge this? 


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-1985444616

   @snleee can you please review this too? 


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2013922617

   hey @snleee @swaminathanmanish addressed comments. Please review! 
   


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "ankitsultana (via GitHub)" <gi...@apache.org>.
ankitsultana commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2004960090

   @snleee @Jackie-Jiang : we'll be working on getting this merged this week, so would be great if you folks can share your feedback.
   
   @tibrewalpratik17 : can you also share what's the test plan for this PR in the description?


-- 
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-unsubscribe@pinot.apache.org

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


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


Re: [PR] Add minionInstanceTag config to minion-tasks for resource isolation [pinot]

Posted by "tibrewalpratik17 (via GitHub)" <gi...@apache.org>.
tibrewalpratik17 commented on PR #12459:
URL: https://github.com/apache/pinot/pull/12459#issuecomment-2014693940

   > Thanks a lot @tibrewalpratik17 . Would you be able to create a github issue to track this effort?
   LGTM. Could you please create github issues to track the additional work?
   
   Tracking all tasks in #12698 


-- 
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-unsubscribe@pinot.apache.org

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


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