You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/06/21 20:48:06 UTC

[GitHub] [pinot] zhtaoxiang opened a new pull request, #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

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

   Add /tasks/{taskType}/{tableNameWithType}/debug API so that we can get a task debug info for the given table. This makes debugging a specific table's problem easier.


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


[GitHub] [pinot] zhtaoxiang commented on pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
zhtaoxiang commented on PR #8949:
URL: https://github.com/apache/pinot/pull/8949#issuecomment-1162578160

   Sample response when verbosity = 1
   
   `curl -X GET "http://localhost:18998/tasks/FileIngestionTask/fileIngestionTaskTestTable_AdhocExecute_OFFLINE/debug?verbosity=1" -H "accept: application/json"`
   
   ```
   {
     "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_1a36671a-c451-41b1-94a0-86cfdf6dd67e": {
       "taskState": "COMPLETED",
       "subtaskCount": {
         "total": 2,
         "completed": 2,
         "running": 0,
         "waiting": 0,
         "error": 0,
         "unknown": 0
       },
       "startTime": "2022-06-21 19:54:35 PDT",
       "executionStartTime": "2022-06-21 19:54:35 PDT",
       "subtaskInfos": [
         {
           "taskId": "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_1a36671a-c451-41b1-94a0-86cfdf6dd67e_1",
           "state": "COMPLETED",
           "startTime": "2022-06-21 19:54:35 PDT",
           "finishTime": "2022-06-21 19:54:39 PDT",
           "participant": "Minion_127.0.0.1_9514",
           "info": "Succeeded",
           "taskConfig": {
             "taskType": "FileIngestionTask",
             "configs": {
               "recordReader.configClassName": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig",
               "maxNumRecordsPerSegment": "1000",
               "inputDirURI": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/",
               "segmentNamePrefix": "fileIngestionTaskTestTable_AdhocExecute",
               "tableMaxNumTasks": "2",
               "inputFormat": "csv",
               "sequenceId": "FileIngestionTask_1655866466031_1",
               "segmentNamePostfix": "FileIngestionTask_1655866466031_1",
               "tableName": "fileIngestionTaskTestTable_AdhocExecute_OFFLINE",
               "recordReader.className": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReader",
               "checkpointTTLSec": "0",
               "input.data.file.uri": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00003.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00004.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00005.csv",
               "taskMaxNumFiles": "3",
               "uploadURL": "http://localhost:18998/segments",
               "push.controllerUri": "http://localhost:18998"
             }
           }
         },
         {
           "taskId": "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_1a36671a-c451-41b1-94a0-86cfdf6dd67e_0",
           "state": "COMPLETED",
           "startTime": "2022-06-21 19:54:35 PDT",
           "finishTime": "2022-06-21 19:54:38 PDT",
           "participant": "Minion_127.0.0.1_9514",
           "info": "Succeeded",
           "taskConfig": {
             "taskType": "FileIngestionTask",
             "configs": {
               "recordReader.configClassName": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig",
               "maxNumRecordsPerSegment": "1000",
               "inputDirURI": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/",
               "segmentNamePrefix": "fileIngestionTaskTestTable_AdhocExecute",
               "tableMaxNumTasks": "2",
               "inputFormat": "csv",
               "sequenceId": "FileIngestionTask_1655866466031_0",
               "segmentNamePostfix": "FileIngestionTask_1655866466031_0",
               "tableName": "fileIngestionTaskTestTable_AdhocExecute_OFFLINE",
               "recordReader.className": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReader",
               "checkpointTTLSec": "0",
               "input.data.file.uri": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00000.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00001.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00002.csv",
               "taskMaxNumFiles": "3",
               "uploadURL": "http://localhost:18998/segments",
               "push.controllerUri": "http://localhost:18998"
             }
           }
         }
       ]
     },
     "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_2c9e0820-1a0b-4bec-a17d-1d06403727f5": {
       "taskState": "COMPLETED",
       "subtaskCount": {
         "total": 2,
         "completed": 2,
         "running": 0,
         "waiting": 0,
         "error": 0,
         "unknown": 0
       },
       "startTime": "2022-06-21 19:55:03 PDT",
       "executionStartTime": "2022-06-21 19:55:03 PDT",
       "subtaskInfos": [
         {
           "taskId": "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_2c9e0820-1a0b-4bec-a17d-1d06403727f5_1",
           "state": "COMPLETED",
           "startTime": "2022-06-21 19:55:03 PDT",
           "finishTime": "2022-06-21 19:55:03 PDT",
           "participant": "Minion_127.0.0.1_9514",
           "info": "Succeeded",
           "taskConfig": {
             "taskType": "FileIngestionTask",
             "configs": {
               "recordReader.configClassName": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig",
               "maxNumRecordsPerSegment": "1000",
               "inputDirURI": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/",
               "segmentNamePrefix": "fileIngestionTaskTestTable_AdhocExecute",
               "tableMaxNumTasks": "2",
               "inputFormat": "csv",
               "sequenceId": "FileIngestionTask_1655866492891_1",
               "segmentNamePostfix": "FileIngestionTask_1655866492891_1",
               "tableName": "fileIngestionTaskTestTable_AdhocExecute_OFFLINE",
               "recordReader.className": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReader",
               "checkpointTTLSec": "0",
               "input.data.file.uri": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_Dir1/emptyFile_00002.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_Dir1/emptyFile_00003.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_Dir1/emptyFile_00004.csv",
               "taskMaxNumFiles": "3",
               "uploadURL": "http://localhost:18998/segments",
               "push.controllerUri": "http://localhost:18998"
             }
           }
         },
         {
           "taskId": "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_2c9e0820-1a0b-4bec-a17d-1d06403727f5_0",
           "state": "COMPLETED",
           "startTime": "2022-06-21 19:55:03 PDT",
           "finishTime": "2022-06-21 19:55:04 PDT",
           "participant": "Minion_127.0.0.1_9514",
           "info": "Succeeded",
           "taskConfig": {
             "taskType": "FileIngestionTask",
             "configs": {
               "recordReader.configClassName": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig",
               "maxNumRecordsPerSegment": "1000",
               "inputDirURI": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/",
               "segmentNamePrefix": "fileIngestionTaskTestTable_AdhocExecute",
               "tableMaxNumTasks": "2",
               "inputFormat": "csv",
               "sequenceId": "FileIngestionTask_1655866492891_0",
               "segmentNamePostfix": "FileIngestionTask_1655866492891_0",
               "tableName": "fileIngestionTaskTestTable_AdhocExecute_OFFLINE",
               "recordReader.className": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReader",
               "checkpointTTLSec": "0",
               "input.data.file.uri": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00012.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_Dir1/emptyFile_00000.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_Dir1/emptyFile_00001.csv",
               "taskMaxNumFiles": "3",
               "uploadURL": "http://localhost:18998/segments",
               "push.controllerUri": "http://localhost:18998"
             }
           }
         }
       ]
     },
     "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_b7294c35-45aa-4450-ae5a-82eee4a4ef49": {
       "taskState": "COMPLETED",
       "subtaskCount": {
         "total": 2,
         "completed": 2,
         "running": 0,
         "waiting": 0,
         "error": 0,
         "unknown": 0
       },
       "startTime": "2022-06-21 19:54:50 PDT",
       "executionStartTime": "2022-06-21 19:54:50 PDT",
       "subtaskInfos": [
         {
           "taskId": "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_b7294c35-45aa-4450-ae5a-82eee4a4ef49_0",
           "state": "COMPLETED",
           "startTime": "2022-06-21 19:54:50 PDT",
           "finishTime": "2022-06-21 19:54:51 PDT",
           "participant": "Minion_127.0.0.1_9514",
           "info": "Succeeded",
           "taskConfig": {
             "taskType": "FileIngestionTask",
             "configs": {
               "recordReader.configClassName": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig",
               "maxNumRecordsPerSegment": "1000",
               "inputDirURI": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/",
               "segmentNamePrefix": "fileIngestionTaskTestTable_AdhocExecute",
               "tableMaxNumTasks": "2",
               "inputFormat": "csv",
               "sequenceId": "FileIngestionTask_1655866480047_0",
               "segmentNamePostfix": "FileIngestionTask_1655866480047_0",
               "tableName": "fileIngestionTaskTestTable_AdhocExecute_OFFLINE",
               "recordReader.className": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReader",
               "checkpointTTLSec": "0",
               "input.data.file.uri": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00006.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00007.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00008.csv",
               "taskMaxNumFiles": "3",
               "uploadURL": "http://localhost:18998/segments",
               "push.controllerUri": "http://localhost:18998"
             }
           }
         },
         {
           "taskId": "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_b7294c35-45aa-4450-ae5a-82eee4a4ef49_1",
           "state": "COMPLETED",
           "startTime": "2022-06-21 19:54:50 PDT",
           "finishTime": "2022-06-21 19:54:52 PDT",
           "participant": "Minion_127.0.0.1_9514",
           "info": "Succeeded",
           "taskConfig": {
             "taskType": "FileIngestionTask",
             "configs": {
               "recordReader.configClassName": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReaderConfig",
               "maxNumRecordsPerSegment": "1000",
               "inputDirURI": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/",
               "segmentNamePrefix": "fileIngestionTaskTestTable_AdhocExecute",
               "tableMaxNumTasks": "2",
               "inputFormat": "csv",
               "sequenceId": "FileIngestionTask_1655866480047_1",
               "segmentNamePostfix": "FileIngestionTask_1655866480047_1",
               "tableName": "fileIngestionTaskTestTable_AdhocExecute_OFFLINE",
               "recordReader.className": "org.apache.pinot.plugin.inputformat.csv.CSVRecordReader",
               "checkpointTTLSec": "0",
               "input.data.file.uri": "file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00009.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00010.csv,file:/var/folders/kd/4c8chyjn6kdcy7bmv69x6bc40000gn/T/FileIngestionTaskIntegrationTest/fileIngestionTaskTestTable_AdhocExecute/tempFile_00011.csv",
               "taskMaxNumFiles": "3",
               "uploadURL": "http://localhost:18998/segments",
               "push.controllerUri": "http://localhost:18998"
             }
           }
         }
       ]
     }
   }
   ```
   


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


[GitHub] [pinot] npawar commented on a diff in pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
npawar commented on code in PR #8949:
URL: https://github.com/apache/pinot/pull/8949#discussion_r903159292


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotHelixTaskResourceManager.java:
##########
@@ -531,6 +531,48 @@ public synchronized Map<String, TaskDebugInfo> getTasksDebugInfo(String taskType
     return taskDebugInfos;
   }
 
+  /**
+   * Given a taskType and a tableNameWithType, helper method to debug all the HelixJobs for
+   * the taskType and tableNameWithType. For each of the HelixJobs, collects status of
+   * the (sub)tasks in the taskbatch.
+   *
+   * @param taskType Pinot taskType / Helix JobQueue
+   * @param tableNameWithType Table name with type to filter on
+   * @param verbosity By default, does not show details for completed tasks.
+   *                  If verbosity > 0, shows details for all tasks.
+   * @return Map of Pinot Task Name to TaskDebugInfo. TaskDebugInfo contains details for subtasks.
+   */
+  public synchronized Map<String, TaskDebugInfo> getTasksDebugInfoByTable(
+      String taskType, String tableNameWithType, int verbosity) {
+    Map<String, TaskDebugInfo> taskDebugInfos = new TreeMap<>();
+    WorkflowContext workflowContext = _taskDriver.getWorkflowContext(getHelixJobQueueName(taskType));
+    if (workflowContext == null) {
+      return taskDebugInfos;
+    }
+
+    Map<String, TaskState> helixJobStates = workflowContext.getJobStates();
+    for (String helixJobName : helixJobStates.keySet()) {
+      String pinotTaskName = getPinotTaskName(helixJobName);
+
+      // Iterate through all task configs associated with this task name
+      for (PinotTaskConfig taskConfig : getTaskConfigs(pinotTaskName)) {
+        Map<String, String> pinotConfigs = taskConfig.getConfigs();
+
+        // Filter task configs that matches this table name
+        if (pinotConfigs != null) {
+          String tableNameConfig = pinotConfigs.get(TABLE_NAME);
+          if (tableNameConfig != null && tableNameConfig.equals(tableNameWithType)) {
+            // Found a match. Add task debug info to the result
+            // TODO: we may want to filter out debug info that does not belong to the given table.

Review Comment:
   didn't follow this TODO. why would we get info from `getTaskDebugInfo` that's not for this table?



##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -215,6 +215,18 @@ public Map<String, PinotHelixTaskResourceManager.TaskDebugInfo> getTasksDebugInf
     return _pinotHelixTaskResourceManager.getTasksDebugInfo(taskType, verbosity);
   }
 
+  @GET
+  @Path("/tasks/{taskType}/{tableNameWithType}/debug")
+  @ApiOperation("Fetch information for the given task type and table")
+  public Map<String, PinotHelixTaskResourceManager.TaskDebugInfo> getTasksDebugInfo(
+      @ApiParam(value = "Task type", required = true) @PathParam("taskType") String taskType,
+      @ApiParam(value = "Table name with type", required = true) @PathParam("tableNameWithType")
+          String tableNameWithType,
+      @ApiParam(value = "verbosity (By default, prints for running and error tasks. Value of >0 prints for all tasks)")
+      @DefaultValue("0") @QueryParam("verbosity") int verbosity) {
+    return _pinotHelixTaskResourceManager.getTasksDebugInfoByTable(taskType, tableNameWithType, verbosity);

Review Comment:
   I tried this out with RealtimeQuickstartWithMinion, and saw it return 3 COMPLETED tasks, for both 0 and 1 verbosity. Only difference was level of detail. Is this `By default, prints for running and error tasks. Value of >0 prints for all tasks)` incorrect or does the impl need some change?



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


[GitHub] [pinot] npawar commented on pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
npawar commented on PR #8949:
URL: https://github.com/apache/pinot/pull/8949#issuecomment-1162473612

   > ## What Does This PR Do
   > Add /tasks/{taskType}/{tableNameWithType}/debug API so that we can get a task debug info for the given table. This makes debugging a specific table's problem easier.
   > 
   > ## Test
   > Test by setting up a Pinot cluster locally.
   
   Mind adding a sample response here?


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


[GitHub] [pinot] npawar merged pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
npawar merged PR #8949:
URL: https://github.com/apache/pinot/pull/8949


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


[GitHub] [pinot] zhtaoxiang commented on a diff in pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
zhtaoxiang commented on code in PR #8949:
URL: https://github.com/apache/pinot/pull/8949#discussion_r903227158


##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTaskRestletResource.java:
##########
@@ -215,6 +215,18 @@ public Map<String, PinotHelixTaskResourceManager.TaskDebugInfo> getTasksDebugInf
     return _pinotHelixTaskResourceManager.getTasksDebugInfo(taskType, verbosity);
   }
 
+  @GET
+  @Path("/tasks/{taskType}/{tableNameWithType}/debug")
+  @ApiOperation("Fetch information for the given task type and table")
+  public Map<String, PinotHelixTaskResourceManager.TaskDebugInfo> getTasksDebugInfo(
+      @ApiParam(value = "Task type", required = true) @PathParam("taskType") String taskType,
+      @ApiParam(value = "Table name with type", required = true) @PathParam("tableNameWithType")
+          String tableNameWithType,
+      @ApiParam(value = "verbosity (By default, prints for running and error tasks. Value of >0 prints for all tasks)")
+      @DefaultValue("0") @QueryParam("verbosity") int verbosity) {
+    return _pinotHelixTaskResourceManager.getTasksDebugInfoByTable(taskType, tableNameWithType, verbosity);

Review Comment:
   simply copied and pasted the existing docs 😄  . fixed this one and existing ones.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/minion/PinotHelixTaskResourceManager.java:
##########
@@ -531,6 +531,48 @@ public synchronized Map<String, TaskDebugInfo> getTasksDebugInfo(String taskType
     return taskDebugInfos;
   }
 
+  /**
+   * Given a taskType and a tableNameWithType, helper method to debug all the HelixJobs for
+   * the taskType and tableNameWithType. For each of the HelixJobs, collects status of
+   * the (sub)tasks in the taskbatch.
+   *
+   * @param taskType Pinot taskType / Helix JobQueue
+   * @param tableNameWithType Table name with type to filter on
+   * @param verbosity By default, does not show details for completed tasks.
+   *                  If verbosity > 0, shows details for all tasks.
+   * @return Map of Pinot Task Name to TaskDebugInfo. TaskDebugInfo contains details for subtasks.
+   */
+  public synchronized Map<String, TaskDebugInfo> getTasksDebugInfoByTable(
+      String taskType, String tableNameWithType, int verbosity) {
+    Map<String, TaskDebugInfo> taskDebugInfos = new TreeMap<>();
+    WorkflowContext workflowContext = _taskDriver.getWorkflowContext(getHelixJobQueueName(taskType));
+    if (workflowContext == null) {
+      return taskDebugInfos;
+    }
+
+    Map<String, TaskState> helixJobStates = workflowContext.getJobStates();
+    for (String helixJobName : helixJobStates.keySet()) {
+      String pinotTaskName = getPinotTaskName(helixJobName);
+
+      // Iterate through all task configs associated with this task name
+      for (PinotTaskConfig taskConfig : getTaskConfigs(pinotTaskName)) {
+        Map<String, String> pinotConfigs = taskConfig.getConfigs();
+
+        // Filter task configs that matches this table name
+        if (pinotConfigs != null) {
+          String tableNameConfig = pinotConfigs.get(TABLE_NAME);
+          if (tableNameConfig != null && tableNameConfig.equals(tableNameWithType)) {
+            // Found a match. Add task debug info to the result
+            // TODO: we may want to filter out debug info that does not belong to the given table.

Review Comment:
   updated the TODO.



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


[GitHub] [pinot] codecov-commenter commented on pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #8949:
URL: https://github.com/apache/pinot/pull/8949#issuecomment-1162385106

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8949?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#8949](https://codecov.io/gh/apache/pinot/pull/8949?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3efa10d) into [master](https://codecov.io/gh/apache/pinot/commit/2644f48e7c896efcff4d5411ff81ddc09e13c3ac?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2644f48) will **decrease** coverage by `6.72%`.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #8949      +/-   ##
   ============================================
   - Coverage     69.69%   62.96%   -6.73%     
   - Complexity     4715     4870     +155     
   ============================================
     Files          1813     1770      -43     
     Lines         94503    92695    -1808     
     Branches      14117    13941     -176     
   ============================================
   - Hits          65860    58363    -7497     
   - Misses        24039    30100    +6061     
   + Partials       4604     4232     -372     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `66.26% <ø> (-0.04%)` | :arrow_down: |
   | unittests2 | `14.97% <0.00%> (+0.12%)` | :arrow_up: |
   
   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=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/8949?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...roller/api/resources/PinotTaskRestletResource.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90VGFza1Jlc3RsZXRSZXNvdXJjZS5qYXZh) | `0.00% <0.00%> (-4.06%)` | :arrow_down: |
   | [...lix/core/minion/PinotHelixTaskResourceManager.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9QaW5vdEhlbGl4VGFza1Jlc291cmNlTWFuYWdlci5qYXZh) | `2.42% <0.00%> (-37.52%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/routing/RoutingTable.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yb3V0aW5nL1JvdXRpbmdUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/common/config/NettyConfig.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL05ldHR5Q29uZmlnLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...apache/pinot/common/helix/ExtraInstanceConfig.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vaGVsaXgvRXh0cmFJbnN0YW5jZUNvbmZpZy5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [421 more](https://codecov.io/gh/apache/pinot/pull/8949/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8949?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8949?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [2644f48...3efa10d](https://codecov.io/gh/apache/pinot/pull/8949?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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


[GitHub] [pinot] zhtaoxiang commented on pull request #8949: add /tasks/{taskType}/{tableNameWithType}/debug API

Posted by GitBox <gi...@apache.org>.
zhtaoxiang commented on PR #8949:
URL: https://github.com/apache/pinot/pull/8949#issuecomment-1162577449

   `curl -X GET "http://localhost:18998/tasks/FileIngestionTask/fileIngestionTaskTestTable_AdhocExecute_OFFLINE/debug?verbosity=0" -H "accept: application/json"`
   
   ```
   {
     "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_1a36671a-c451-41b1-94a0-86cfdf6dd67e": {
       "taskState": "COMPLETED",
       "subtaskCount": {
         "total": 2,
         "completed": 2,
         "running": 0,
         "waiting": 0,
         "error": 0,
         "unknown": 0
       },
       "startTime": "2022-06-21 19:54:35 PDT",
       "executionStartTime": "2022-06-21 19:54:35 PDT"
     },
     "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_2c9e0820-1a0b-4bec-a17d-1d06403727f5": {
       "taskState": "COMPLETED",
       "subtaskCount": {
         "total": 2,
         "completed": 2,
         "running": 0,
         "waiting": 0,
         "error": 0,
         "unknown": 0
       },
       "startTime": "2022-06-21 19:55:03 PDT",
       "executionStartTime": "2022-06-21 19:55:03 PDT"
     },
     "Task_FileIngestionTask_fileIngestionTaskTestTable_AdhocExecute_OFFLINE_b7294c35-45aa-4450-ae5a-82eee4a4ef49": {
       "taskState": "COMPLETED",
       "subtaskCount": {
         "total": 2,
         "completed": 2,
         "running": 0,
         "waiting": 0,
         "error": 0,
         "unknown": 0
       },
       "startTime": "2022-06-21 19:54:50 PDT",
       "executionStartTime": "2022-06-21 19:54:50 PDT"
     }
   }
   ```
   


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