From 2de4769ae4d3bdd8bd6bf152fc0d13d50399dec1 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Tue, 27 Jun 2023 17:19:15 -0700 Subject: [PATCH 01/14] Tasks API refactor --- docs/api-reference/tasks-api.md | 1381 ++++++++++++++++++++++++++++++- website/static/css/custom.css | 12 + 2 files changed, 1358 insertions(+), 35 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index f8c1ca2e358c..036232a6b9d0 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -30,72 +30,1383 @@ This document describes the API endpoints for task retrieval, submission, and de Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`. -`GET /druid/indexer/v1/tasks` +### Get an array of tasks -Retrieve list of tasks. Accepts query string parameters `state`, `datasource`, `createdTimeInterval`, `max`, and `type`. +#### URL +GET `/druid/indexer/v1/tasks` -|Query Parameter |Description | -|---|---| -|`state`|filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| -| `datasource`| return tasks filtered by Druid datasource.| -| `createdTimeInterval`| return tasks created within the specified interval. | -| `max`| maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| -| `type`| filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| +This endpoint retrieves an array of all task objects currently running or executed in the current Druid cluster. It provides information about each task such as its task id, task status, associated data source, and other metadata. It supports a set of optional query parameters to filter results. +#### Query parameters +|Parameter|Type|Description| +|---|---|---| +|`state`|String|Filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| +| `datasource`|String| Return tasks filtered by Druid datasource.| +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| -`GET /druid/indexer/v1/completeTasks` +#### Responses -Retrieve list of complete tasks. Equivalent to `/druid/indexer/v1/tasks?state=complete`. + -`GET /druid/indexer/v1/runningTasks` + +
+*Successfully retrieved list of tasks* + +
+*Invalid `state` query parameter value* + +
+*Invalid query parameter* + -Retrieve list of running tasks. Equivalent to `/druid/indexer/v1/tasks?state=running`. +#### Sample request -`GET /druid/indexer/v1/waitingTasks` +The following example shows how to retrieve a list of tasks filtered with the following query parameters: +* State: `complete` +* Datasource: `wikipedia_api` +* Time interval: between `2015-09-12` and `2015-09-13` +* Max entries returned: `10` +* Task type: `query_worker` -Retrieve list of waiting tasks. Equivalent to `/druid/indexer/v1/tasks?state=waiting`. + -`GET /druid/indexer/v1/pendingTasks` + +```shell +curl "{domain}/druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12T00%3A00%3A00Z%2F2015-09-13T23%3A59%3A59Z&max=10&type=query_worker" +``` + +```HTTP +GET /druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12T00%3A00%3A00Z%2F2015-09-13T23%3A59%3A59Z&max=10&type=query_worker HTTP/1.1 +Host: {domain} +``` -Retrieve list of pending tasks. Equivalent to `/druid/indexer/v1/tasks?state=pending`. + -`GET /druid/indexer/v1/task/{taskId}` +#### Sample response -Retrieve the 'payload' of a task. +
+ Toggle to show sample response +
[
+    {
+        "id": "query-223549f8-b993-4483-b028-1b0d54713cad-worker0_0",
+        "groupId": "query-223549f8-b993-4483-b028-1b0d54713cad",
+        "type": "query_worker",
+        "createdTime": "2023-06-22T22:11:37.012Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "SUCCESS",
+        "status": "SUCCESS",
+        "runnerStatusCode": "NONE",
+        "duration": 17897,
+        "location": {
+            "host": "localhost",
+            "port": 8101,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    },
+    {
+        "id": "query-fa82fa40-4c8c-4777-b832-cabbee5f519f-worker0_0",
+        "groupId": "query-fa82fa40-4c8c-4777-b832-cabbee5f519f",
+        "type": "query_worker",
+        "createdTime": "2023-06-20T22:51:21.302Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "SUCCESS",
+        "status": "SUCCESS",
+        "runnerStatusCode": "NONE",
+        "duration": 16911,
+        "location": {
+            "host": "localhost",
+            "port": 8101,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    },
+    {
+        "id": "query-5419da7a-b270-492f-90e6-920ecfba766a-worker0_0",
+        "groupId": "query-5419da7a-b270-492f-90e6-920ecfba766a",
+        "type": "query_worker",
+        "createdTime": "2023-06-20T22:45:53.909Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "SUCCESS",
+        "status": "SUCCESS",
+        "runnerStatusCode": "NONE",
+        "duration": 17030,
+        "location": {
+            "host": "localhost",
+            "port": 8101,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    }
+]
+
-`GET /druid/indexer/v1/task/{taskId}/status` +### Get an array of complete tasks -Retrieve the status of a task. +#### URL +GET `/druid/indexer/v1/completeTasks` -`GET /druid/indexer/v1/task/{taskId}/segments` +This endpoint retrieves an array of completed task objects in the current Druid cluster. This is functionally equivalent to `/druid/indexer/v1/tasks?state=complete`. It supports a set of optional query parameters to filter results. + +#### Query parameters +|Parameter|Type|Description| +|---|---|---| +| `datasource`|String| Return tasks filtered by Druid datasource.| +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| + +#### Responses + + + + +
+*Successfully retrieved list of complete tasks* + +
+*Request sent to incorrect service* + + + +#### Sample request + + + + +```shell +curl "{domain}/druid/indexer/v1/completeTasks" +``` + +```HTTP +GET /druid/indexer/v1/completeTasks HTTP/1.1 +Host: {domain} +``` + + +#### Sample response + +
+ Toggle to show sample response +
[
+    {
+        "id": "query-223549f8-b993-4483-b028-1b0d54713cad-worker0_0",
+        "groupId": "query-223549f8-b993-4483-b028-1b0d54713cad",
+        "type": "query_worker",
+        "createdTime": "2023-06-22T22:11:37.012Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "SUCCESS",
+        "status": "SUCCESS",
+        "runnerStatusCode": "NONE",
+        "duration": 17897,
+        "location": {
+            "host": "localhost",
+            "port": 8101,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    },
+    {
+        "id": "query-223549f8-b993-4483-b028-1b0d54713cad",
+        "groupId": "query-223549f8-b993-4483-b028-1b0d54713cad",
+        "type": "query_controller",
+        "createdTime": "2023-06-22T22:11:28.367Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "SUCCESS",
+        "status": "SUCCESS",
+        "runnerStatusCode": "NONE",
+        "duration": 30317,
+        "location": {
+            "host": "localhost",
+            "port": 8100,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    }
+]
+
+ +### Get an array of running tasks + +#### URL +GET `/druid/indexer/v1/runningTasks` + +This endpoint retrieves an array of running task objects in the current Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=running`. It supports a set of optional query parameters to filter results. + +#### Query parameters +|Parameter|Type|Description| +|---|---|---| +| `datasource`|String| Return tasks filtered by Druid datasource.| +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| + +#### Responses + + + + +
+*Successfully retrieved list of running tasks* + + + +#### Sample request + + + + +```shell +curl "{domain}/druid/indexer/v1/runningTasks" +``` + +```HTTP +GET /druid/indexer/v1/runningTasks HTTP/1.1 +Host: {domain} +``` + + +#### Sample response + +
+ Toggle to show sample response +
[
+    {
+        "id": "query-32663269-ead9-405a-8eb6-0817a952ef47",
+        "groupId": "query-32663269-ead9-405a-8eb6-0817a952ef47",
+        "type": "query_controller",
+        "createdTime": "2023-06-22T22:54:43.170Z",
+        "queueInsertionTime": "2023-06-22T22:54:43.170Z",
+        "statusCode": "RUNNING",
+        "status": "RUNNING",
+        "runnerStatusCode": "RUNNING",
+        "duration": -1,
+        "location": {
+            "host": "localhost",
+            "port": 8100,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    }
+]
+
+ +### Get an array of waiting tasks + +#### URL +GET `/druid/indexer/v1/waitingTasks` + +This endpoint retrieves an array of waiting task objects in the current Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=waiting`. It supports a set of optional query parameters to filter results. + +#### Query parameters +|Parameter|Type|Description| +|---|---|---| +| `datasource`|String| Return tasks filtered by Druid datasource.| +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| + +#### Responses + + + + +
+*Successfully retrieved list of waiting tasks* + + + +#### Sample request + + + + +```shell +curl "{domain}/druid/indexer/v1/waitingTasks" +``` + +```HTTP +GET /druid/indexer/v1/waitingTasks HTTP/1.1 +Host: {domain} +``` + + +#### Sample response + +
+ Toggle to show sample response +
[
+    {
+        "id": "index_parallel_wikipedia_auto_biahcbmf_2023-06-26T21:08:05.216Z",
+        "groupId": "index_parallel_wikipedia_auto_biahcbmf_2023-06-26T21:08:05.216Z",
+        "type": "index_parallel",
+        "createdTime": "2023-06-26T21:08:05.217Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "RUNNING",
+        "status": "RUNNING",
+        "runnerStatusCode": "WAITING",
+        "duration": -1,
+        "location": {
+            "host": null,
+            "port": -1,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_auto",
+        "errorMsg": null
+    },
+    {
+        "id": "index_parallel_wikipedia_auto_afggfiec_2023-06-26T21:08:05.546Z",
+        "groupId": "index_parallel_wikipedia_auto_afggfiec_2023-06-26T21:08:05.546Z",
+        "type": "index_parallel",
+        "createdTime": "2023-06-26T21:08:05.548Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "RUNNING",
+        "status": "RUNNING",
+        "runnerStatusCode": "WAITING",
+        "duration": -1,
+        "location": {
+            "host": null,
+            "port": -1,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_auto",
+        "errorMsg": null
+    },
+    {
+        "id": "index_parallel_wikipedia_auto_jmmddihf_2023-06-26T21:08:06.644Z",
+        "groupId": "index_parallel_wikipedia_auto_jmmddihf_2023-06-26T21:08:06.644Z",
+        "type": "index_parallel",
+        "createdTime": "2023-06-26T21:08:06.671Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "RUNNING",
+        "status": "RUNNING",
+        "runnerStatusCode": "WAITING",
+        "duration": -1,
+        "location": {
+            "host": null,
+            "port": -1,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_auto",
+        "errorMsg": null
+    }
+]
+
+ +### Get an array of pending tasks + +#### URL + +GET `/druid/indexer/v1/pendingTasks` + +This endpoint retrieves an array of pending task objects in the current Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=pending`. It supports a set of optional query parameters to filter results. + +#### Query parameters +|Parameter|Type|Description| +|---|---|---| +| `datasource`|String| Return tasks filtered by Druid datasource.| +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| + +#### Responses + + + + +
+*Successfully retrieved list of pending tasks* + + + +#### Sample request + + + + +```shell +curl "{domain}/druid/indexer/v1/pendingTasks" +``` + +```HTTP +GET /druid/indexer/v1/pendingTasks HTTP/1.1 +Host: {domain} +``` + + +#### Sample response + +
+ Toggle to show sample response +
[
+    {
+        "id": "query-7b37c315-50a0-4b68-aaa8-b1ef1f060e67",
+        "groupId": "query-7b37c315-50a0-4b68-aaa8-b1ef1f060e67",
+        "type": "query_controller",
+        "createdTime": "2023-06-23T19:53:06.037Z",
+        "queueInsertionTime": "2023-06-23T19:53:06.037Z",
+        "statusCode": "RUNNING",
+        "status": "RUNNING",
+        "runnerStatusCode": "PENDING",
+        "duration": -1,
+        "location": {
+            "host": null,
+            "port": -1,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    },
+    {
+        "id": "query-544f0c41-f81d-4504-b98b-f9ab8b36ef36",
+        "groupId": "query-544f0c41-f81d-4504-b98b-f9ab8b36ef36",
+        "type": "query_controller",
+        "createdTime": "2023-06-23T19:53:06.616Z",
+        "queueInsertionTime": "2023-06-23T19:53:06.616Z",
+        "statusCode": "RUNNING",
+        "status": "RUNNING",
+        "runnerStatusCode": "PENDING",
+        "duration": -1,
+        "location": {
+            "host": null,
+            "port": -1,
+            "tlsPort": -1
+        },
+        "dataSource": "wikipedia_api",
+        "errorMsg": null
+    }
+]
+
+ +### Get task payload + +#### URL +GET `/druid/indexer/v1/task/{taskId}` + +Retrieves the payload of a task given the task id. It returns a JSON object with the task id and payload which includes task configuration details and relevant specifications associated with the execution of the task. + +#### Query parameters + +No query parameters. + +#### Responses + + + +
+*Successfully retrieved payload of task* + +
+*Cannot find task with id* + + + +#### Sample request + +The following examples shows how to retrieve the task payload of a task with the specified id `query-32663269-ead9-405a-8eb6-0817a952ef47`. + + + + +```shell +curl "{domain}/druid/indexer/v1/task/query-32663269-ead9-405a-8eb6-0817a952ef47" +``` + +```HTTP +GET /druid/indexer/v1/task/query-32663269-ead9-405a-8eb6-0817a952ef47 HTTP/1.1 +Host: {domain} +``` + + + +#### Sample response + +
+ Toggle to show sample response +
{
+    "task": "query-32663269-ead9-405a-8eb6-0817a952ef47",
+    "payload": {
+        "type": "query_controller",
+        "id": "query-32663269-ead9-405a-8eb6-0817a952ef47",
+        "spec": {
+            "query": {
+                "queryType": "scan",
+                "dataSource": {
+                    "type": "external",
+                    "inputSource": {
+                        "type": "http",
+                        "uris": [
+                            "https://druid.apache.org/data/wikipedia.json.gz"
+                        ]
+                    },
+                    "inputFormat": {
+                        "type": "json",
+                        "keepNullColumns": false,
+                        "assumeNewlineDelimited": false,
+                        "useJsonNodeReader": false
+                    },
+                    "signature": [
+                        {
+                            "name": "added",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "channel",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "cityName",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "comment",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "commentLength",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "countryIsoCode",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "countryName",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "deleted",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "delta",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "deltaBucket",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "diffUrl",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "flags",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "isAnonymous",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "isMinor",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "isNew",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "isRobot",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "isUnpatrolled",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "metroCode",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "namespace",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "page",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "regionIsoCode",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "regionName",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "timestamp",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "user",
+                            "type": "STRING"
+                        }
+                    ]
+                },
+                "intervals": {
+                    "type": "intervals",
+                    "intervals": [
+                        "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
+                    ]
+                },
+                "virtualColumns": [
+                    {
+                        "type": "expression",
+                        "name": "v0",
+                        "expression": "timestamp_parse(\"timestamp\",null,'UTC')",
+                        "outputType": "LONG"
+                    }
+                ],
+                "resultFormat": "compactedList",
+                "columns": [
+                    "added",
+                    "channel",
+                    "cityName",
+                    "comment",
+                    "commentLength",
+                    "countryIsoCode",
+                    "countryName",
+                    "deleted",
+                    "delta",
+                    "deltaBucket",
+                    "diffUrl",
+                    "flags",
+                    "isAnonymous",
+                    "isMinor",
+                    "isNew",
+                    "isRobot",
+                    "isUnpatrolled",
+                    "metroCode",
+                    "namespace",
+                    "page",
+                    "regionIsoCode",
+                    "regionName",
+                    "timestamp",
+                    "user",
+                    "v0"
+                ],
+                "legacy": false,
+                "context": {
+                    "finalize": true,
+                    "maxNumTasks": 3,
+                    "maxParseExceptions": 0,
+                    "queryId": "32663269-ead9-405a-8eb6-0817a952ef47",
+                    "scanSignature": "[{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"commentLength\",\"type\":\"LONG\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"deltaBucket\",\"type\":\"STRING\"},{\"name\":\"diffUrl\",\"type\":\"STRING\"},{\"name\":\"flags\",\"type\":\"STRING\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"STRING\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"LONG\"}]",
+                    "sqlInsertSegmentGranularity": "\"DAY\"",
+                    "sqlQueryId": "32663269-ead9-405a-8eb6-0817a952ef47"
+                },
+                "granularity": {
+                    "type": "all"
+                }
+            },
+            "columnMappings": [
+                {
+                    "queryColumn": "v0",
+                    "outputColumn": "__time"
+                },
+                {
+                    "queryColumn": "added",
+                    "outputColumn": "added"
+                },
+                {
+                    "queryColumn": "channel",
+                    "outputColumn": "channel"
+                },
+                {
+                    "queryColumn": "cityName",
+                    "outputColumn": "cityName"
+                },
+                {
+                    "queryColumn": "comment",
+                    "outputColumn": "comment"
+                },
+                {
+                    "queryColumn": "commentLength",
+                    "outputColumn": "commentLength"
+                },
+                {
+                    "queryColumn": "countryIsoCode",
+                    "outputColumn": "countryIsoCode"
+                },
+                {
+                    "queryColumn": "countryName",
+                    "outputColumn": "countryName"
+                },
+                {
+                    "queryColumn": "deleted",
+                    "outputColumn": "deleted"
+                },
+                {
+                    "queryColumn": "delta",
+                    "outputColumn": "delta"
+                },
+                {
+                    "queryColumn": "deltaBucket",
+                    "outputColumn": "deltaBucket"
+                },
+                {
+                    "queryColumn": "diffUrl",
+                    "outputColumn": "diffUrl"
+                },
+                {
+                    "queryColumn": "flags",
+                    "outputColumn": "flags"
+                },
+                {
+                    "queryColumn": "isAnonymous",
+                    "outputColumn": "isAnonymous"
+                },
+                {
+                    "queryColumn": "isMinor",
+                    "outputColumn": "isMinor"
+                },
+                {
+                    "queryColumn": "isNew",
+                    "outputColumn": "isNew"
+                },
+                {
+                    "queryColumn": "isRobot",
+                    "outputColumn": "isRobot"
+                },
+                {
+                    "queryColumn": "isUnpatrolled",
+                    "outputColumn": "isUnpatrolled"
+                },
+                {
+                    "queryColumn": "metroCode",
+                    "outputColumn": "metroCode"
+                },
+                {
+                    "queryColumn": "namespace",
+                    "outputColumn": "namespace"
+                },
+                {
+                    "queryColumn": "page",
+                    "outputColumn": "page"
+                },
+                {
+                    "queryColumn": "regionIsoCode",
+                    "outputColumn": "regionIsoCode"
+                },
+                {
+                    "queryColumn": "regionName",
+                    "outputColumn": "regionName"
+                },
+                {
+                    "queryColumn": "timestamp",
+                    "outputColumn": "timestamp"
+                },
+                {
+                    "queryColumn": "user",
+                    "outputColumn": "user"
+                }
+            ],
+            "destination": {
+                "type": "dataSource",
+                "dataSource": "wikipedia_api",
+                "segmentGranularity": "DAY"
+            },
+            "assignmentStrategy": "max",
+            "tuningConfig": {
+                "maxNumWorkers": 2,
+                "maxRowsInMemory": 100000,
+                "rowsPerSegment": 3000000
+            }
+        },
+        "sqlQuery": "\nINSERT INTO wikipedia_api \nSELECT \n  TIME_PARSE(\"timestamp\") AS __time,\n  * \nFROM TABLE(EXTERN(\n  '{\"type\": \"http\", \"uris\": [\"https://druid.apache.org/data/wikipedia.json.gz\"]}', \n  '{\"type\": \"json\"}', \n  '[{\"name\": \"added\", \"type\": \"long\"}, {\"name\": \"channel\", \"type\": \"string\"}, {\"name\": \"cityName\", \"type\": \"string\"}, {\"name\": \"comment\", \"type\": \"string\"}, {\"name\": \"commentLength\", \"type\": \"long\"}, {\"name\": \"countryIsoCode\", \"type\": \"string\"}, {\"name\": \"countryName\", \"type\": \"string\"}, {\"name\": \"deleted\", \"type\": \"long\"}, {\"name\": \"delta\", \"type\": \"long\"}, {\"name\": \"deltaBucket\", \"type\": \"string\"}, {\"name\": \"diffUrl\", \"type\": \"string\"}, {\"name\": \"flags\", \"type\": \"string\"}, {\"name\": \"isAnonymous\", \"type\": \"string\"}, {\"name\": \"isMinor\", \"type\": \"string\"}, {\"name\": \"isNew\", \"type\": \"string\"}, {\"name\": \"isRobot\", \"type\": \"string\"}, {\"name\": \"isUnpatrolled\", \"type\": \"string\"}, {\"name\": \"metroCode\", \"type\": \"string\"}, {\"name\": \"namespace\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"regionIsoCode\", \"type\": \"string\"}, {\"name\": \"regionName\", \"type\": \"string\"}, {\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n  ))\nPARTITIONED BY DAY\n",
+        "sqlQueryContext": {
+            "sqlQueryId": "32663269-ead9-405a-8eb6-0817a952ef47",
+            "sqlInsertSegmentGranularity": "\"DAY\"",
+            "maxNumTasks": 3,
+            "queryId": "32663269-ead9-405a-8eb6-0817a952ef47"
+        },
+        "sqlResultsContext": {
+            "timeZone": "UTC",
+            "serializeComplexValues": true,
+            "stringifyArrays": true
+        },
+        "sqlTypeNames": [
+            "TIMESTAMP",
+            "BIGINT",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "BIGINT",
+            "VARCHAR",
+            "VARCHAR",
+            "BIGINT",
+            "BIGINT",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR"
+        ],
+        "context": {
+            "forceTimeChunkLock": true,
+            "useLineageBasedSegmentAllocation": true
+        },
+        "groupId": "query-32663269-ead9-405a-8eb6-0817a952ef47",
+        "dataSource": "wikipedia_api",
+        "resource": {
+            "availabilityGroup": "query-32663269-ead9-405a-8eb6-0817a952ef47",
+            "requiredCapacity": 1
+        }
+    }
+}
+
+ +### Get task status + +#### URL +GET `/druid/indexer/v1/task/{taskId}/status` + +Retrieves the status of a taks given the task id. It returns a JSON object with the task's current status (`running`, `complete`, `waiting`, `pending`), task type, datasource, and other relevant metadata. + +#### Query parameters + +No query parameters. + +#### Responses + + + +
+*Successfully retrieved task status* + +
+*Cannot find task with id* + + +#### Sample request + +The following examples shows how to retrieve the status of a task with the specified id `query-223549f8-b993-4483-b028-1b0d54713cad`. + + + + +```shell +curl "{domain}/druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status" +``` + +```HTTP +GET /druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status HTTP/1.1 +Host: {domain} +``` + + + +#### Sample response +
+ Toggle to show sample response +
{'task': 'query-223549f8-b993-4483-b028-1b0d54713cad',
+ 'status': {'id': 'query-223549f8-b993-4483-b028-1b0d54713cad',
+  'groupId': 'query-223549f8-b993-4483-b028-1b0d54713cad',
+  'type': 'query_controller',
+  'createdTime': '2023-06-22T22:11:28.367Z',
+  'queueInsertionTime': '1970-01-01T00:00:00.000Z',
+  'statusCode': 'RUNNING',
+  'status': 'RUNNING',
+  'runnerStatusCode': 'RUNNING',
+  'duration': -1,
+  'location': {'host': 'localhost', 'port': 8100, 'tlsPort': -1},
+  'dataSource': 'wikipedia_api',
+  'errorMsg': None}
+}
+
+ +### Get task segments + +#### URL + +GET `/druid/indexer/v1/task/{taskId}/segments` > This API is deprecated and will be removed in future releases. -Retrieve information about the segments of a task. +Retrieves information about segments generated by the task given the task id. To hit this endpoint, make sure to enable the audit log config on the Overlord with `druid.indexer.auditLog.enabled = true`. + +In addition to enabling audit log config, enable the cleanup of audit logs on the Coordinator. For reference, see [Audit records](../operations/clean-metadata-store.md#audit-records). + +#### Query parameters -`GET /druid/indexer/v1/task/{taskId}/reports` +No query parameters. -Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. Only works for completed tasks. +#### Responses + -`POST /druid/indexer/v1/task` + +
+*Successfully retrieved task segments* + -Endpoint for submitting tasks and supervisor specs to the Overlord. Returns the taskId of the submitted task. +#### Sample request -`POST /druid/indexer/v1/task/{taskId}/shutdown` +The following examples shows how to retrieve the task segment of the task with the specified id `query-52a8aafe-7265-4427-89fe-dc51275cc470`. -Shuts down a task. + -`POST /druid/indexer/v1/datasources/{dataSource}/shutdownAllTasks` + +```shell +curl "{domain}/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports" +``` + +```HTTP +GET /druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports HTTP/1.1 +Host: {domain} +``` + -Shuts down all tasks for a dataSource. +#### Sample response -`POST /druid/indexer/v1/taskStatus` +A successful request returns a `200 OK` response and an array of the task segments. -Retrieve list of task status objects for list of task id strings in request body. +### Get task completion report -`DELETE /druid/indexer/v1/pendingSegments/{dataSource}` +#### URL -Manually clean up pending segments table in metadata storage for `datasource`. Returns a JSON object response with +GET `/druid/indexer/v1/task/{taskId}/reports` + +Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. It returns a JSON object with information about the number of rows ingested, and any parse exceptions that occurred. It is available for both completed tasks and running tasks. + +#### Query parameters + +No query parameters. + +#### Responses + + + + +
+*Successfully retrieved task report* + + +#### Sample request + +The following examples shows how to retrieve the completion report of a task with the specified id `query-52a8aafe-7265-4427-89fe-dc51275cc470`. + + + + +```shell +curl "{domain}/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports" +``` + +```HTTP +GET /druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports HTTP/1.1 +Host: {domain} +``` + + +#### Sample response +
+ Toggle to show sample response +
{
+    "ingestionStatsAndErrors": {
+        "type": "ingestionStatsAndErrors",
+        "taskId": "query-52a8aafe-7265-4427-89fe-dc51275cc470",
+        "payload": {
+            "ingestionState": "COMPLETED",
+            "unparseableEvents": {},
+            "rowStats": {
+                "determinePartitions": {
+                    "processed": 0,
+                    "processedBytes": 0,
+                    "processedWithError": 0,
+                    "thrownAway": 0,
+                    "unparseable": 0
+                },
+                "buildSegments": {
+                    "processed": 39244,
+                    "processedBytes": 17106256,
+                    "processedWithError": 0,
+                    "thrownAway": 0,
+                    "unparseable": 0
+                }
+            },
+            "errorMsg": null,
+            "segmentAvailabilityConfirmed": false,
+            "segmentAvailabilityWaitTimeMs": 0
+        }
+    }
+}
+
+ +### Submit a task + +#### URL + +POST `/druid/indexer/v1/task` + +Endpoint for submitting tasks and supervisor specs to the Overlord. It returns the task id of the submitted task. + +#### Query parameters + +No query parameters. + +#### Responses + + + +
+*Successfully submitted task* + +
+*Missing information in query* + +
+*Incorrect request body media type* + +
+*Unexpected token or characters in request body* + + +#### Sample request + +The following request is an example of submitting a task to create a datasource named "wikipedia auto". + + + + +```shell +curl "{domain}/druid/indexer/v1/task" \ +--header "Content-Type: application/json" \ +--data "{ + \"type\" : \"index_parallel\", + \"spec\" : { + \"dataSchema\" : { + \"dataSource\" : \"wikipedia_auto\", + \"timestampSpec\": { + \"column\": \"time\", + \"format\": \"iso\" + }, + \"dimensionsSpec\" : { + \"useSchemaDiscovery\": true + }, + \"metricsSpec\" : [], + \"granularitySpec\" : { + \"type\" : \"uniform\", + \"segmentGranularity\" : \"day\", + \"queryGranularity\" : \"none\", + \"intervals\" : [\"2015-09-12/2015-09-13\"], + \"rollup\" : false + } + }, + \"ioConfig\" : { + \"type\" : \"index_parallel\", + \"inputSource\" : { + \"type\" : \"local\", + \"baseDir\" : \"quickstart/tutorial/\", + \"filter\" : \"wikiticker-2015-09-12-sampled.json.gz\" + }, + \"inputFormat\" : { + \"type\" : \"json\" + }, + \"appendToExisting\" : false + }, + \"tuningConfig\" : { + \"type\" : \"index_parallel\", + \"maxRowsPerSegment\" : 5000000, + \"maxRowsInMemory\" : 25000 + } + } +}" +``` + +```HTTP +POST /druid/indexer/v1/task HTTP/1.1 +Host: {domain} +Content-Type: application/json +Content-Length: 952 + +{ + "type" : "index_parallel", + "spec" : { + "dataSchema" : { + "dataSource" : "wikipedia_auto", + "timestampSpec": { + "column": "time", + "format": "iso" + }, + "dimensionsSpec" : { + "useSchemaDiscovery": true + }, + "metricsSpec" : [], + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "day", + "queryGranularity" : "none", + "intervals" : ["2015-09-12/2015-09-13"], + "rollup" : false + } + }, + "ioConfig" : { + "type" : "index_parallel", + "inputSource" : { + "type" : "local", + "baseDir" : "quickstart/tutorial/", + "filter" : "wikiticker-2015-09-12-sampled.json.gz" + }, + "inputFormat" : { + "type" : "json" + }, + "appendToExisting" : false + }, + "tuningConfig" : { + "type" : "index_parallel", + "maxRowsPerSegment" : 5000000, + "maxRowsInMemory" : 25000 + } + } +} +``` + + +#### Sample response +
+ Toggle to show sample response +
{
+    "task": "index_parallel_wikipedia_odofhkle_2023-06-23T21:07:28.226Z"
+}
+
+ +### Shut down a task + +#### URL +POST `/druid/indexer/v1/task/{taskId}/shutdown` + +Shuts down a task if it not already complete. Returns a JSON object with the id of the task that was shutdown successfully. + +#### Query parameters + +No query parameters. + +#### Responses + + + +
+*Successfully shut down task* + +
+*Cannot find task with id* + + +#### Sample request + +The following request is an example of shutting down task of id `query-52as 8aafe-7265-4427-89fe-dc51275cc470`. + + + + +```shell +curl --request POST "{domain}/druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown" +``` + +```HTTP +POST /druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown HTTP/1.1 +Host: (domain) +``` + + + +#### Sample response +
+ Toggle to show sample response +
{'task': 'query-577a83dd-a14e-4380-bd01-c942b781236b'}
+
+ +### Shut down all tasks for a datasource + +#### URL +POST `/druid/indexer/v1/datasources/{dataSource}/shutdownAllTasks` + +Shuts down all tasks for a specified datasource. If successful, it returns a JSON object with the name of the datasource whose tasks are shut down. + +#### Query parameters + +No query parameters. + +#### Responses + + + + +
+*Successfully shut down tasks* + +
+*Error or datasource does not have a running task* + + +#### Sample request + +The following request is an example of shutting down all tasks for datasource `wikipedia_auto`. + + + + +```shell +curl --request POST "{domain}/druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks" +``` + +```HTTP +POST /druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks HTTP/1.1 +Host: {domain} +``` + + + +#### Sample response +
+ Toggle to show sample response +
{
+    "dataSource": "wikipedia_api"
+}
+
+ + +### Retrieve status objects for tasks + +#### URL +POST `/druid/indexer/v1/taskStatus` + +Retrieve list of task status objects for list of task id strings in request body. It returns a set of JSON objects with the status, duration, location of each task, and any error messages if applicable. + +#### Query parameters + +No query parameters. + +#### Responses + + + + +
+*Successfully retrieved status objects* + +
+*Missing request body or incorrect request body type* + + +#### Sample request + +The following request is an example of retrieving status objects for task id `index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z` and `index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z` . + + + + +```shell +curl "{domain}/druid/indexer/v1/taskStatus" \ +--header "Content-Type: application/json" \ +--data "[\"index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z\", \"index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z\"]" +``` + +```HTTP +POST /druid/indexer/v1/taskStatus HTTP/1.1 +Host: {domain} +Content-Type: application/json +Content-Length: 134 + +["index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z", "index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z"] +``` + + + +#### Sample response +
+ Toggle to show sample response +
{
+    "index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z": {
+        "id": "index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z",
+        "status": "SUCCESS",
+        "duration": 10630,
+        "errorMsg": null,
+        "location": {
+            "host": "localhost",
+            "port": 8100,
+            "tlsPort": -1
+        }
+    },
+    "index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z": {
+        "id": "index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z",
+        "status": "SUCCESS",
+        "duration": 11012,
+        "errorMsg": null,
+        "location": {
+            "host": "localhost",
+            "port": 8100,
+            "tlsPort": -1
+        }
+    }
+}
+
+ +### Clean up pending segments for a data source. + +#### URL + +DELETE `/druid/indexer/v1/pendingSegments/{dataSource}` + +Manually clean up pending segments table in metadata storage for `datasource`. It returns a JSON object response with `numDeleted` and count of rows deleted from the pending segments table. This API is used by the `druid.coordinator.kill.pendingSegments.on` [coordinator setting](../configuration/index.md#coordinator-operation) which automates this operation to perform periodically. + +#### Query parameters + +No query parameters. + +#### Responses + + + +
+*Successfully deleted pending segments* + + + +#### Sample request + +The following request is an example of cleaning up pending segments for the `wikipedia_api` datasource. + + + + +```shell +curl --request DELETE "{domain}/druid/indexer/v1/pendingSegments/wikipedia_api" +``` + +```HTTP +DELETE /druid/indexer/v1/pendingSegments/wikipedia_api HTTP/1.1 +Host: {domain} +``` + + + +#### Sample response +
+ Toggle to show sample response +
{
+    "numDeleted": 2
+}
+
\ No newline at end of file diff --git a/website/static/css/custom.css b/website/static/css/custom.css index c73a53081425..c625c6826253 100644 --- a/website/static/css/custom.css +++ b/website/static/css/custom.css @@ -99,3 +99,15 @@ article iframe { margin-right: auto; max-width: 100%; } +.getAPI { + color: #0073e6; + font-weight: bold; +} +.postAPI { + color: #00bf7d; + font-weight: bold; +} +.deleteAPI { + color: #f49200; + font-weight: bold; +} \ No newline at end of file From b1956ce59cbd0a774d98636349cafaa152c2677f Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Wed, 28 Jun 2023 09:08:21 -0700 Subject: [PATCH 02/14] typo --- docs/api-reference/tasks-api.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 036232a6b9d0..8ce7031f0973 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -880,7 +880,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/task/{taskId}/status` -Retrieves the status of a taks given the task id. It returns a JSON object with the task's current status (`running`, `complete`, `waiting`, `pending`), task type, datasource, and other relevant metadata. +Retrieves the status of a task given the task id. It returns a JSON object with the task's current status (`running`, `complete`, `waiting`, `pending`), task type, datasource, and other relevant metadata. #### Query parameters From 2178ffd488fdf1b8316fa76b6be03b273ff17906 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Wed, 28 Jun 2023 09:31:44 -0700 Subject: [PATCH 03/14] categorization --- docs/api-reference/tasks-api.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 8ce7031f0973..fbd10cc2015b 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -25,11 +25,11 @@ sidebar_label: Tasks This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. -## Tasks - Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`. +## Task information and retrieval + ### Get an array of tasks #### URL @@ -1051,6 +1051,8 @@ Host: {domain} } +## Task operations + ### Submit a task #### URL @@ -1284,6 +1286,7 @@ Host: {domain} } +## Task management ### Retrieve status objects for tasks From e3403dd73bd38f58ed3bfd8c5c1897ac005cd33c Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Wed, 28 Jun 2023 10:51:45 -0700 Subject: [PATCH 04/14] update code block and change toggle to click --- docs/api-reference/tasks-api.md | 159 ++++++++++++++++++++++---------- 1 file changed, 108 insertions(+), 51 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index fbd10cc2015b..7a20a18c8fd3 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -87,8 +87,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
[
+  Click to show sample response
+  
+  ```json
+  [
     {
         "id": "query-223549f8-b993-4483-b028-1b0d54713cad-worker0_0",
         "groupId": "query-223549f8-b993-4483-b028-1b0d54713cad",
@@ -143,7 +145,9 @@ Host: {domain}
         "dataSource": "wikipedia_api",
         "errorMsg": null
     }
-]
+ ] + ``` +
### Get an array of complete tasks @@ -192,8 +196,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
[
+  Click to show sample response
+  
+  ```json
+  [
     {
         "id": "query-223549f8-b993-4483-b028-1b0d54713cad-worker0_0",
         "groupId": "query-223549f8-b993-4483-b028-1b0d54713cad",
@@ -230,7 +236,9 @@ Host: {domain}
         "dataSource": "wikipedia_api",
         "errorMsg": null
     }
-]
+ ] + ``` +
### Get an array of running tasks @@ -276,8 +284,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
[
+  Click to show sample response
+  
+  ```json
+  [
     {
         "id": "query-32663269-ead9-405a-8eb6-0817a952ef47",
         "groupId": "query-32663269-ead9-405a-8eb6-0817a952ef47",
@@ -296,7 +306,9 @@ Host: {domain}
         "dataSource": "wikipedia_api",
         "errorMsg": null
     }
-]
+ ] + ``` +
### Get an array of waiting tasks @@ -342,8 +354,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
[
+  Click to show sample response
+  
+  ```json
+  [
     {
         "id": "index_parallel_wikipedia_auto_biahcbmf_2023-06-26T21:08:05.216Z",
         "groupId": "index_parallel_wikipedia_auto_biahcbmf_2023-06-26T21:08:05.216Z",
@@ -398,7 +412,9 @@ Host: {domain}
         "dataSource": "wikipedia_auto",
         "errorMsg": null
     }
-]
+ ] + ``` +
### Get an array of pending tasks @@ -445,8 +461,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
[
+  Click to show sample response
+  
+  ```json
+  [
     {
         "id": "query-7b37c315-50a0-4b68-aaa8-b1ef1f060e67",
         "groupId": "query-7b37c315-50a0-4b68-aaa8-b1ef1f060e67",
@@ -483,7 +501,9 @@ Host: {domain}
         "dataSource": "wikipedia_api",
         "errorMsg": null
     }
-]
+ ] + ``` +
### Get task payload @@ -530,8 +550,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
{
+  Click to show sample response
+  
+  ```json
+  {
     "task": "query-32663269-ead9-405a-8eb6-0817a952ef47",
     "payload": {
         "type": "query_controller",
@@ -872,7 +894,9 @@ Host: {domain}
             "requiredCapacity": 1
         }
     }
-}
+ } + ``` +
### Get task status @@ -917,21 +941,28 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
{'task': 'query-223549f8-b993-4483-b028-1b0d54713cad',
- 'status': {'id': 'query-223549f8-b993-4483-b028-1b0d54713cad',
-  'groupId': 'query-223549f8-b993-4483-b028-1b0d54713cad',
-  'type': 'query_controller',
-  'createdTime': '2023-06-22T22:11:28.367Z',
-  'queueInsertionTime': '1970-01-01T00:00:00.000Z',
-  'statusCode': 'RUNNING',
-  'status': 'RUNNING',
-  'runnerStatusCode': 'RUNNING',
-  'duration': -1,
-  'location': {'host': 'localhost', 'port': 8100, 'tlsPort': -1},
-  'dataSource': 'wikipedia_api',
-  'errorMsg': None}
-}
+ Click to show sample response + + ```json + { + 'task': 'query-223549f8-b993-4483-b028-1b0d54713cad', + 'status': { + 'id': 'query-223549f8-b993-4483-b028-1b0d54713cad', + 'groupId': 'query-223549f8-b993-4483-b028-1b0d54713cad', + 'type': 'query_controller', + 'createdTime': '2023-06-22T22:11:28.367Z', + 'queueInsertionTime': '1970-01-01T00:00:00.000Z', + 'statusCode': 'RUNNING', + 'status': 'RUNNING', + 'runnerStatusCode': 'RUNNING', + 'duration': -1, + 'location': {'host': 'localhost', 'port': 8100, 'tlsPort': -1}, + 'dataSource': 'wikipedia_api', + 'errorMsg': None + } + } + ``` +
### Get task segments @@ -1019,8 +1050,10 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
{
+  Click to show sample response
+  
+  ```json
+  {
     "ingestionStatsAndErrors": {
         "type": "ingestionStatsAndErrors",
         "taskId": "query-52a8aafe-7265-4427-89fe-dc51275cc470",
@@ -1048,7 +1081,9 @@ Host: {domain}
             "segmentAvailabilityWaitTimeMs": 0
         }
     }
-}
+ } + ``` +
## Task operations @@ -1185,10 +1220,14 @@ Content-Length: 952 #### Sample response
- Toggle to show sample response -
{
-    "task": "index_parallel_wikipedia_odofhkle_2023-06-23T21:07:28.226Z"
-}
+ Click to show sample response + + ```json + { + "task": "index_parallel_wikipedia_odofhkle_2023-06-23T21:07:28.226Z" + } + ``` +
### Shut down a task @@ -1233,8 +1272,14 @@ Host: (domain) #### Sample response
- Toggle to show sample response -
{'task': 'query-577a83dd-a14e-4380-bd01-c942b781236b'}
+ Click to show sample response + + ```json + { + 'task': 'query-577a83dd-a14e-4380-bd01-c942b781236b' + } + ``` +
### Shut down all tasks for a datasource @@ -1280,10 +1325,14 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
{
+  Click to show sample response
+  
+  ```json
+  {
     "dataSource": "wikipedia_api"
-}
+ } + ``` +
## Task management @@ -1337,8 +1386,10 @@ Content-Length: 134 #### Sample response
- Toggle to show sample response -
{
+  Click to show sample response
+  
+  ```json
+  {
     "index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z": {
         "id": "index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z",
         "status": "SUCCESS",
@@ -1361,7 +1412,9 @@ Content-Length: 134
             "tlsPort": -1
         }
     }
-}
+ } + ``` +
### Clean up pending segments for a data source. @@ -1408,8 +1461,12 @@ Host: {domain} #### Sample response
- Toggle to show sample response -
{
+  Click to show sample response
+  
+  ```json
+  {
     "numDeleted": 2
-}
+ } + ``` +
\ No newline at end of file From 8f4dcb830dd2d403bd91dd8532439537856aecb2 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Thu, 29 Jun 2023 11:53:22 -0700 Subject: [PATCH 05/14] feedback --- docs/api-reference/tasks-api.md | 148 +++++++++++++++++--------------- 1 file changed, 77 insertions(+), 71 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 7a20a18c8fd3..5052b16bd010 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -25,8 +25,8 @@ sidebar_label: Tasks This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. -Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` -as in `2016-06-27_2016-06-28`. +For query parameters that take an interval, provide ISO 8601 strings delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`. +In this document, `{domain}` is a placeholder for the server address of deployment. ## Task information and retrieval @@ -35,15 +35,18 @@ as in `2016-06-27_2016-06-28`. #### URL GET `/druid/indexer/v1/tasks` -This endpoint retrieves an array of all task objects currently running or executed in the current Druid cluster. It provides information about each task such as its task id, task status, associated data source, and other metadata. It supports a set of optional query parameters to filter results. +Retrieves an array of all tasks in the Druid cluster. Returns a task object with information including its ID, status, associated datasource, and other metadata. #### Query parameters + +The endpoint supports a set of optional query parameters to filter results. + |Parameter|Type|Description| |---|---|---| |`state`|String|Filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| | `datasource`|String| Return tasks filtered by Druid datasource.| | `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | -| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| #### Responses @@ -61,6 +64,8 @@ This endpoint retrieves an array of all task objects currently running or execut *Invalid query parameter* +--- + #### Sample request The following example shows how to retrieve a list of tasks filtered with the following query parameters: @@ -155,14 +160,17 @@ Host: {domain} #### URL GET `/druid/indexer/v1/completeTasks` -This endpoint retrieves an array of completed task objects in the current Druid cluster. This is functionally equivalent to `/druid/indexer/v1/tasks?state=complete`. It supports a set of optional query parameters to filter results. +Retrieves an array of completed tasks in the Druid cluster. This is functionally equivalent to `/druid/indexer/v1/tasks?state=complete`. #### Query parameters + +The endpoint supports a set of optional query parameters to filter results. + |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| | `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | -| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| #### Responses @@ -175,9 +183,10 @@ This endpoint retrieves an array of completed task objects in the current Druid
*Request sent to incorrect service* - +--- + #### Sample request @@ -246,14 +255,17 @@ Host: {domain} #### URL GET `/druid/indexer/v1/runningTasks` -This endpoint retrieves an array of running task objects in the current Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=running`. It supports a set of optional query parameters to filter results. +Retrieves an array of running task objects in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=running`. #### Query parameters + +The endpoint supports a set of optional query parameters to filter results. + |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| | `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | -| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| #### Responses @@ -266,6 +278,8 @@ This endpoint retrieves an array of running task objects in the current Druid cl +--- + #### Sample request @@ -281,6 +295,7 @@ Host: {domain} ``` + #### Sample response
@@ -316,14 +331,17 @@ Host: {domain} #### URL GET `/druid/indexer/v1/waitingTasks` -This endpoint retrieves an array of waiting task objects in the current Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=waiting`. It supports a set of optional query parameters to filter results. +Retrieves an array of waiting tasks in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=waiting`. #### Query parameters + +The endpoint supports a set of optional query parameters to filter results. + |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| | `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | -| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| #### Responses @@ -336,6 +354,8 @@ This endpoint retrieves an array of waiting task objects in the current Druid cl +--- + #### Sample request @@ -423,14 +443,17 @@ Host: {domain} GET `/druid/indexer/v1/pendingTasks` -This endpoint retrieves an array of pending task objects in the current Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=pending`. It supports a set of optional query parameters to filter results. +Retrieves an array of pending tasks in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=pending`. #### Query parameters + +The endpoint supports a set of optional query parameters to filter results. + |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| | `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | -| `max`|Integer|Maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.| +| `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| #### Responses @@ -440,9 +463,10 @@ This endpoint retrieves an array of pending task objects in the current Druid cl
*Successfully retrieved list of pending tasks* - +--- + #### Sample request @@ -511,11 +535,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/task/{taskId}` -Retrieves the payload of a task given the task id. It returns a JSON object with the task id and payload which includes task configuration details and relevant specifications associated with the execution of the task. - -#### Query parameters - -No query parameters. +Retrieves the payload of a task given the task ID. It returns a JSON object with the task ID and payload that includes task configuration details and relevant specifications associated with the execution of the task. #### Responses @@ -525,13 +545,15 @@ No query parameters. *Successfully retrieved payload of task*
-*Cannot find task with id* +*Cannot find task with ID* +--- + #### Sample request -The following examples shows how to retrieve the task payload of a task with the specified id `query-32663269-ead9-405a-8eb6-0817a952ef47`. +The following examples shows how to retrieve the task payload of a task with the specified ID `query-32663269-ead9-405a-8eb6-0817a952ef47`. @@ -904,11 +926,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/task/{taskId}/status` -Retrieves the status of a task given the task id. It returns a JSON object with the task's current status (`running`, `complete`, `waiting`, `pending`), task type, datasource, and other relevant metadata. - -#### Query parameters - -No query parameters. +Retrieves the status of a task given the task ID. It returns a JSON object with the task's current state, task type, datasource, and other relevant metadata. #### Responses @@ -918,12 +936,14 @@ No query parameters. *Successfully retrieved task status*
-*Cannot find task with id* +*Cannot find task with ID* +--- + #### Sample request -The following examples shows how to retrieve the status of a task with the specified id `query-223549f8-b993-4483-b028-1b0d54713cad`. +The following examples shows how to retrieve the status of a task with the specified ID `query-223549f8-b993-4483-b028-1b0d54713cad`. @@ -973,13 +993,9 @@ Host: {domain} > This API is deprecated and will be removed in future releases. -Retrieves information about segments generated by the task given the task id. To hit this endpoint, make sure to enable the audit log config on the Overlord with `druid.indexer.auditLog.enabled = true`. - -In addition to enabling audit log config, enable the cleanup of audit logs on the Coordinator. For reference, see [Audit records](../operations/clean-metadata-store.md#audit-records). +Retrieves information about segments generated by the task given the task ID. To hit this endpoint, make sure to enable the audit log config on the Overlord with `druid.indexer.auditLog.enabled = true`. -#### Query parameters - -No query parameters. +In addition to enabling audit logs, configure a cleanup strategy to prevent overloading the metadata store with old audit logs and cause performance issues. You can enable automated cleanup of audit logs on the Coordinator with `druid.coordinator.kill.audit.on` or export the audit logs to external storage. For reference, see [Audit records](../operations/clean-metadata-store.md#audit-records). #### Responses @@ -989,9 +1005,11 @@ No query parameters. *Successfully retrieved task segments* +--- + #### Sample request -The following examples shows how to retrieve the task segment of the task with the specified id `query-52a8aafe-7265-4427-89fe-dc51275cc470`. +The following examples shows how to retrieve the task segment of the task with the specified ID `query-52a8aafe-7265-4427-89fe-dc51275cc470`. @@ -1016,11 +1034,7 @@ A successful request returns a `200 OK` response and an array of the task segmen GET `/druid/indexer/v1/task/{taskId}/reports` -Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. It returns a JSON object with information about the number of rows ingested, and any parse exceptions that occurred. It is available for both completed tasks and running tasks. - -#### Query parameters - -No query parameters. +Retrieves a [task completion report](../ingestion/tasks.md#task-reports) for a task. It returns a JSON object with information about the number of rows ingested, and any parse exceptions that occurred. It is available for both completed tasks and running tasks. #### Responses @@ -1031,9 +1045,11 @@ No query parameters. *Successfully retrieved task report* +--- + #### Sample request -The following examples shows how to retrieve the completion report of a task with the specified id `query-52a8aafe-7265-4427-89fe-dc51275cc470`. +The following examples shows how to retrieve the completion report of a task with the specified ID `query-52a8aafe-7265-4427-89fe-dc51275cc470`. @@ -1094,11 +1110,7 @@ Host: {domain} POST `/druid/indexer/v1/task` -Endpoint for submitting tasks and supervisor specs to the Overlord. It returns the task id of the submitted task. - -#### Query parameters - -No query parameters. +Endpoint for submitting tasks and supervisor specs to the Overlord. It returns the task ID of the submitted task. #### Responses @@ -1117,9 +1129,11 @@ No query parameters. *Unexpected token or characters in request body* +--- + #### Sample request -The following request is an example of submitting a task to create a datasource named "wikipedia auto". +The following request is an example of submitting a task to create a datasource named `"wikipedia auto"`. @@ -1235,11 +1249,7 @@ Content-Length: 952 #### URL POST `/druid/indexer/v1/task/{taskId}/shutdown` -Shuts down a task if it not already complete. Returns a JSON object with the id of the task that was shutdown successfully. - -#### Query parameters - -No query parameters. +Shuts down a task if it not already complete. Returns a JSON object with the ID of the task that was shut down successfully. #### Responses @@ -1249,12 +1259,14 @@ No query parameters. *Successfully shut down task*
-*Cannot find task with id* +*Cannot find task with ID* +--- + #### Sample request -The following request is an example of shutting down task of id `query-52as 8aafe-7265-4427-89fe-dc51275cc470`. +The following request shows how to shut down a task with the ID `query-52as 8aafe-7265-4427-89fe-dc51275cc470`. @@ -1289,10 +1301,6 @@ Host: (domain) Shuts down all tasks for a specified datasource. If successful, it returns a JSON object with the name of the datasource whose tasks are shut down. -#### Query parameters - -No query parameters. - #### Responses @@ -1305,6 +1313,8 @@ No query parameters. *Error or datasource does not have a running task* +--- + #### Sample request The following request is an example of shutting down all tasks for datasource `wikipedia_auto`. @@ -1342,11 +1352,7 @@ Host: {domain} #### URL POST `/druid/indexer/v1/taskStatus` -Retrieve list of task status objects for list of task id strings in request body. It returns a set of JSON objects with the status, duration, location of each task, and any error messages if applicable. - -#### Query parameters - -No query parameters. +Retrieves list of task status objects for list of task ID strings in request body. It returns a set of JSON objects with the status, duration, location of each task, and any error messages if applicable. #### Responses @@ -1360,9 +1366,11 @@ No query parameters. *Missing request body or incorrect request body type* +--- + #### Sample request -The following request is an example of retrieving status objects for task id `index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z` and `index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z` . +The following request is an example of retrieving status objects for task ID `index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z` and `index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z` . @@ -1417,21 +1425,17 @@ Content-Length: 134
-### Clean up pending segments for a data source. +### Clean up pending segments for a datasource. #### URL DELETE `/druid/indexer/v1/pendingSegments/{dataSource}` Manually clean up pending segments table in metadata storage for `datasource`. It returns a JSON object response with -`numDeleted` and count of rows deleted from the pending segments table. This API is used by the -`druid.coordinator.kill.pendingSegments.on` [coordinator setting](../configuration/index.md#coordinator-operation) +`numDeleted` for the number of rows deleted from the pending segments table. This API is used by the +`druid.coordinator.kill.pendingSegments.on` [Coordinator setting](../configuration/index.md#coordinator-operation) which automates this operation to perform periodically. -#### Query parameters - -No query parameters. - #### Responses @@ -1441,6 +1445,8 @@ No query parameters. +--- + #### Sample request The following request is an example of cleaning up pending segments for the `wikipedia_api` datasource. From d898961063911c4d31fd312088659b389773c603 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Fri, 30 Jun 2023 14:04:58 -0700 Subject: [PATCH 06/14] revisions --- docs/api-reference/tasks-api.md | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 5052b16bd010..ecfafb2328db 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -25,8 +25,9 @@ sidebar_label: Tasks This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. +In this document, `{domain}` is a placeholder for the server address of deployment. For example, on the quickstart configuration, replace `{domain}` with `http://localhost:8888`. + For query parameters that take an interval, provide ISO 8601 strings delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`. -In this document, `{domain}` is a placeholder for the server address of deployment. ## Task information and retrieval @@ -35,7 +36,7 @@ In this document, `{domain}` is a placeholder for the server address of deployme #### URL GET `/druid/indexer/v1/tasks` -Retrieves an array of all tasks in the Druid cluster. Returns a task object with information including its ID, status, associated datasource, and other metadata. +Retrieves an array of all tasks in the Druid cluster. Each task object includes information on its ID, status, associated datasource, and other metadata. #### Query parameters @@ -995,7 +996,7 @@ Host: {domain} Retrieves information about segments generated by the task given the task ID. To hit this endpoint, make sure to enable the audit log config on the Overlord with `druid.indexer.auditLog.enabled = true`. -In addition to enabling audit logs, configure a cleanup strategy to prevent overloading the metadata store with old audit logs and cause performance issues. You can enable automated cleanup of audit logs on the Coordinator with `druid.coordinator.kill.audit.on` or export the audit logs to external storage. For reference, see [Audit records](../operations/clean-metadata-store.md#audit-records). +In addition to enabling audit logs, configure a cleanup strategy to prevent overloading the metadata store with old audit logs which may cause performance issues. To enable automated cleanup of audit logs on the Coordinator, set `druid.coordinator.kill.audit.on`. You may also manually export the audit logs to external storage. For more information, see [Audit records](../operations/clean-metadata-store.md#audit-records). #### Responses @@ -1034,7 +1035,7 @@ A successful request returns a `200 OK` response and an array of the task segmen GET `/druid/indexer/v1/task/{taskId}/reports` -Retrieves a [task completion report](../ingestion/tasks.md#task-reports) for a task. It returns a JSON object with information about the number of rows ingested, and any parse exceptions that occurred. It is available for both completed tasks and running tasks. +Retrieves a [task completion report](../ingestion/tasks.md#task-reports) for a task. It returns a JSON object with information about the number of rows ingested, and any parse exceptions that Druid raised. #### Responses @@ -1110,7 +1111,7 @@ Host: {domain} POST `/druid/indexer/v1/task` -Endpoint for submitting tasks and supervisor specs to the Overlord. It returns the task ID of the submitted task. +Submits a task or supervisor spec to the Overlord. It returns the task ID of the submitted task. #### Responses @@ -1352,7 +1353,7 @@ Host: {domain} #### URL POST `/druid/indexer/v1/taskStatus` -Retrieves list of task status objects for list of task ID strings in request body. It returns a set of JSON objects with the status, duration, location of each task, and any error messages if applicable. +Retrieves list of task status objects for list of task ID strings in request body. It returns a set of JSON objects with the status, duration, location of each task, and any error messages. #### Responses @@ -1425,7 +1426,7 @@ Content-Length: 134 -### Clean up pending segments for a datasource. +### Clean up pending segments for a datasource #### URL From 3809216e5bd707e9cdd737d3a993532cb85828e1 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Mon, 3 Jul 2023 15:24:29 -0700 Subject: [PATCH 07/14] Add undocumented task endpoint --- docs/api-reference/tasks-api.md | 54 +++++++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index ecfafb2328db..0e9f1661ae8a 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -1029,6 +1029,60 @@ Host: {domain} A successful request returns a `200 OK` response and an array of the task segments. +### Get task log + +#### URL + +GET `/druid/indexer/v1/task/{taskId}/log` + +Retrieves the event log associated with a task. It returns a list of logged events during the lifecycle of the task. The endpoint is useful for providing information about the execution of the task, including any errors or warnings raised. + +#### Query parameters +* `offset` + * Type: Int + * Exclude the first passed in number of entries from the response. + +#### Responses + + + + +
+*Successfully retrieved task report* + + +--- + +#### Sample request + +The following examples shows how to retrieve the task log of a task with the specified ID `index_kafka_social_media_0e905aa31037879_nommnaeg`. + + + + +```shell +curl "{domain}/druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log" +``` + +```HTTP +GET /druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log HTTP/1.1 +Host: {domain} +``` + + +#### Sample response + +
+ Click to show sample response + + ```json + 2023-07-03T21:32:06,409 WARN [task-runner-0-priority-0] org.apache.druid.indexing.kafka.IncrementalPublishingKafkaIndexTaskRunner - OffsetOutOfRangeException with message [Fetch position FetchPosition{offset=230985, offsetEpoch=Optional.empty, currentLeader=LeaderAndEpoch{leader=Optional[localhost:9094 (id: 1 rack: null)], epoch=0}} is out of range for partition social_media-0] + 2023-07-03T21:32:06,409 WARN [task-runner-0-priority-0] org.apache.druid.indexing.kafka.IncrementalPublishingKafkaIndexTaskRunner - Retrying in 30000ms + 2023-07-03T21:32:36,437 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.Fetcher - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Fetch position FetchPosition{offset=230985, offsetEpoch=Optional.empty, currentLeader=LeaderAndEpoch{leader=Optional[localhost:9094 (id: 1 rack: null)], epoch=0}} is out of range for partition social_media-0, raising error to the application since no reset policy is configured + ``` + +
+ ### Get task completion report #### URL From fdbcf59079e8c87360e6b8baf9a6594d9b3123a7 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Mon, 3 Jul 2023 15:31:14 -0700 Subject: [PATCH 08/14] Update task log example --- docs/api-reference/tasks-api.md | 64 ++++++++++++++++++++++++++++++--- 1 file changed, 60 insertions(+), 4 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 0e9f1661ae8a..4367d9ea7983 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -1038,7 +1038,7 @@ A successful request returns a `200 OK` response and an array of the task segmen Retrieves the event log associated with a task. It returns a list of logged events during the lifecycle of the task. The endpoint is useful for providing information about the execution of the task, including any errors or warnings raised. #### Query parameters -* `offset` +* `offset` (optional) * Type: Int * Exclude the first passed in number of entries from the response. @@ -1076,9 +1076,65 @@ Host: {domain} Click to show sample response ```json - 2023-07-03T21:32:06,409 WARN [task-runner-0-priority-0] org.apache.druid.indexing.kafka.IncrementalPublishingKafkaIndexTaskRunner - OffsetOutOfRangeException with message [Fetch position FetchPosition{offset=230985, offsetEpoch=Optional.empty, currentLeader=LeaderAndEpoch{leader=Optional[localhost:9094 (id: 1 rack: null)], epoch=0}} is out of range for partition social_media-0] - 2023-07-03T21:32:06,409 WARN [task-runner-0-priority-0] org.apache.druid.indexing.kafka.IncrementalPublishingKafkaIndexTaskRunner - Retrying in 30000ms - 2023-07-03T21:32:36,437 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.Fetcher - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Fetch position FetchPosition{offset=230985, offsetEpoch=Optional.empty, currentLeader=LeaderAndEpoch{leader=Optional[localhost:9094 (id: 1 rack: null)], epoch=0}} is out of range for partition social_media-0, raising error to the application since no reset policy is configured + 2023-07-03T22:11:17,891 INFO [qtp1251996697-122] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Sequence[index_kafka_social_media_0e905aa31037879_0] end offsets updated from [{0=9223372036854775807}] to [{0=230985}]. + 2023-07-03T22:11:17,900 INFO [qtp1251996697-122] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Saved sequence metadata to disk: [SequenceMetadata{sequenceId=0, sequenceName='index_kafka_social_media_0e905aa31037879_0', assignments=[0], startOffsets={0=230985}, exclusiveStartPartitions=[], endOffsets={0=230985}, sentinel=false, checkpointed=true}] + 2023-07-03T22:11:17,901 INFO [task-runner-0-priority-0] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Received resume command, resuming ingestion. + 2023-07-03T22:11:17,901 INFO [task-runner-0-priority-0] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Finished reading partition[0], up to[230985]. + 2023-07-03T22:11:17,902 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Resetting generation and member id due to: consumer pro-actively leaving the group + 2023-07-03T22:11:17,902 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Request joining group due to: consumer pro-actively leaving the group + 2023-07-03T22:11:17,902 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.KafkaConsumer - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Unsubscribed all topics or patterns and assigned partitions + 2023-07-03T22:11:17,912 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted rows[0] and (estimated) bytes[0] + 2023-07-03T22:11:17,916 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments: + 2023-07-03T22:11:17,917 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], sinks: [0], total fireHydrants (across sinks): [0], persisted fireHydrants (across sinks): [0] + 2023-07-03T22:11:17,919 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver - Pushing [0] segments in background + 2023-07-03T22:11:17,921 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted rows[0] and (estimated) bytes[0] + 2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments: + 2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], sinks: [0], total fireHydrants (across sinks): [0], persisted fireHydrants (across sinks): [0] + 2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Preparing to push (stats): processed rows: [0], sinks: [0], fireHydrants (across sinks): [0] + 2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Push complete... + 2023-07-03T22:11:17,929 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.indexing.seekablestream.SequenceMetadata - With empty segment set, start offsets [SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}] and end offsets [SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}] are the same, skipping metadata commit. + 2023-07-03T22:11:17,930 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver - Published [0] segments with commit metadata [{nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}] + 2023-07-03T22:11:17,930 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Published 0 segments for sequence [index_kafka_social_media_0e905aa31037879_0] with metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}]. + 2023-07-03T22:11:17,931 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Saved sequence metadata to disk: [] + 2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Handoff complete for segments: + 2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Resetting generation and member id due to: consumer pro-actively leaving the group + 2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Request joining group due to: consumer pro-actively leaving the group + 2023-07-03T22:11:17,933 INFO [task-runner-0-priority-0] org.apache.kafka.common.metrics.Metrics - Metrics scheduler closed + 2023-07-03T22:11:17,933 INFO [task-runner-0-priority-0] org.apache.kafka.common.metrics.Metrics - Closing reporter org.apache.kafka.common.metrics.JmxReporter + 2023-07-03T22:11:17,933 INFO [task-runner-0-priority-0] org.apache.kafka.common.metrics.Metrics - Metrics reporters closed + 2023-07-03T22:11:17,935 INFO [task-runner-0-priority-0] org.apache.kafka.common.utils.AppInfoParser - App info kafka.consumer for consumer-kafka-supervisor-dcanhmig-1 unregistered + 2023-07-03T22:11:17,936 INFO [task-runner-0-priority-0] org.apache.druid.curator.announcement.Announcer - Unannouncing [/druid/internal-discovery/PEON/localhost:8100] + 2023-07-03T22:11:17,972 INFO [task-runner-0-priority-0] org.apache.druid.curator.discovery.CuratorDruidNodeAnnouncer - Unannounced self [{"druidNode":{"service":"druid/middleManager","host":"localhost","bindOnHost":false,"plaintextPort":8100,"port":-1,"tlsPort":-1,"enablePlaintextPort":true,"enableTlsPort":false},"nodeType":"peon","services":{"dataNodeService":{"type":"dataNodeService","tier":"_default_tier","maxSize":0,"type":"indexer-executor","serverType":"indexer-executor","priority":0},"lookupNodeService":{"type":"lookupNodeService","lookupTier":"__default"}}}]. + 2023-07-03T22:11:17,972 INFO [task-runner-0-priority-0] org.apache.druid.curator.announcement.Announcer - Unannouncing [/druid/announcements/localhost:8100] + 2023-07-03T22:11:17,996 INFO [task-runner-0-priority-0] org.apache.druid.indexing.worker.executor.ExecutorLifecycle - Task completed with status: { + "id" : "index_kafka_social_media_0e905aa31037879_nommnaeg", + "status" : "SUCCESS", + "duration" : 3601130, + "errorMsg" : null, + "location" : { + "host" : null, + "port" : -1, + "tlsPort" : -1 + } + } + 2023-07-03T22:11:17,998 INFO [main] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [ANNOUNCEMENTS] + 2023-07-03T22:11:18,005 INFO [main] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [SERVER] + 2023-07-03T22:11:18,009 INFO [main] org.eclipse.jetty.server.AbstractConnector - Stopped ServerConnector@6491006{HTTP/1.1, (http/1.1)}{0.0.0.0:8100} + 2023-07-03T22:11:18,009 INFO [main] org.eclipse.jetty.server.session - node0 Stopped scavenging + 2023-07-03T22:11:18,012 INFO [main] org.eclipse.jetty.server.handler.ContextHandler - Stopped o.e.j.s.ServletContextHandler@742aa00a{/,null,STOPPED} + 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [NORMAL] + 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.server.coordination.ZkCoordinator - Stopping ZkCoordinator for [DruidServerMetadata{name='localhost:8100', hostAndPort='localhost:8100', hostAndTlsPort='null', maxSize=0, tier='_default_tier', type=indexer-executor, priority=0}] + 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.server.coordination.SegmentLoadDropHandler - Stopping... + 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.server.coordination.SegmentLoadDropHandler - Stopped. + 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.indexing.overlord.SingleTaskBackgroundRunner - Starting graceful shutdown of task[index_kafka_social_media_0e905aa31037879_nommnaeg]. + 2023-07-03T22:11:18,014 INFO [main] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Stopping forcefully (status: [PUBLISHING]) + 2023-07-03T22:11:18,019 INFO [LookupExtractorFactoryContainerProvider-MainThread] org.apache.druid.query.lookup.LookupReferencesManager - Lookup Management loop exited. Lookup notices are not handled anymore. + 2023-07-03T22:11:18,020 INFO [main] org.apache.druid.query.lookup.LookupReferencesManager - Closed lookup [name]. + 2023-07-03T22:11:18,020 INFO [Curator-Framework-0] org.apache.curator.framework.imps.CuratorFrameworkImpl - backgroundOperationsLoop exiting + 2023-07-03T22:11:18,147 INFO [main] org.apache.zookeeper.ZooKeeper - Session: 0x1000097ceaf0007 closed + 2023-07-03T22:11:18,147 INFO [main-EventThread] org.apache.zookeeper.ClientCnxn - EventThread shut down for session: 0x1000097ceaf0007 + 2023-07-03T22:11:18,151 INFO [main] org.apache.druid.java.util.common.lifecycle.Lifecycle - Stopping lifecycle [module] stage [INIT] + Finished peon task ``` From e01ec3dba76e7f331c7458c21b10f5cfa9303281 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Mon, 3 Jul 2023 15:37:28 -0700 Subject: [PATCH 09/14] typo... --- docs/api-reference/tasks-api.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 4367d9ea7983..a0feff9c21b7 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -1048,7 +1048,7 @@ Retrieves the event log associated with a task. It returns a list of logged even
-*Successfully retrieved task report* +*Successfully retrieved task log* --- From 766e748688e322488ba6d00cd1df7d967d450101 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Fri, 7 Jul 2023 12:46:49 -0700 Subject: [PATCH 10/14] Incorporate feedback, update intro -> shorten payload response --- docs/api-reference/tasks-api.md | 459 +++++++++----------------------- 1 file changed, 133 insertions(+), 326 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index a0feff9c21b7..856e5a9d9cc5 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -23,12 +23,10 @@ sidebar_label: Tasks ~ under the License. --> -This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. +This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete data-related operations such as ingestion and compaction. In this document, `{domain}` is a placeholder for the server address of deployment. For example, on the quickstart configuration, replace `{domain}` with `http://localhost:8888`. -For query parameters that take an interval, provide ISO 8601 strings delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`. - ## Task information and retrieval ### Get an array of tasks @@ -36,7 +34,7 @@ For query parameters that take an interval, provide ISO 8601 strings delimited b #### URL GET `/druid/indexer/v1/tasks` -Retrieves an array of all tasks in the Druid cluster. Each task object includes information on its ID, status, associated datasource, and other metadata. +Retrieves an array of all tasks in the Druid cluster. Each task object includes information on its ID, status, associated datasource, and other metadata. #### Query parameters @@ -46,7 +44,7 @@ The endpoint supports a set of optional query parameters to filter results. |---|---|---| |`state`|String|Filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| | `datasource`|String| Return tasks filtered by Druid datasource.| -| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. The interval string should be delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`.| | `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| @@ -80,11 +78,11 @@ The following example shows how to retrieve a list of tasks filtered with the fo ```shell -curl "{domain}/druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12T00%3A00%3A00Z%2F2015-09-13T23%3A59%3A59Z&max=10&type=query_worker" +curl "{domain}/druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker" ``` ```HTTP -GET /druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12T00%3A00%3A00Z%2F2015-09-13T23%3A59%3A59Z&max=10&type=query_worker HTTP/1.1 +GET /druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker HTTP/1.1 Host: {domain} ``` @@ -170,7 +168,7 @@ The endpoint supports a set of optional query parameters to filter results. |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| -| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. The interval string should be delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`.| | `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| @@ -265,7 +263,7 @@ The endpoint supports a set of optional query parameters to filter results. |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| -| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. The interval string should be delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`.| | `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| @@ -341,7 +339,7 @@ The endpoint supports a set of optional query parameters to filter results. |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| -| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. The interval string should be delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`.| | `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| @@ -453,7 +451,7 @@ The endpoint supports a set of optional query parameters to filter results. |Parameter|Type|Description| |---|---|---| | `datasource`|String| Return tasks filtered by Druid datasource.| -| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. | +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. The interval string should be delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`.| | `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| @@ -554,17 +552,17 @@ Retrieves the payload of a task given the task ID. It returns a JSON object with #### Sample request -The following examples shows how to retrieve the task payload of a task with the specified ID `query-32663269-ead9-405a-8eb6-0817a952ef47`. +The following examples shows how to retrieve the task payload of a task with the specified ID `index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z`. ```shell -curl "{domain}/druid/indexer/v1/task/query-32663269-ead9-405a-8eb6-0817a952ef47" +curl "{domain}/druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z" ``` ```HTTP -GET /druid/indexer/v1/task/query-32663269-ead9-405a-8eb6-0817a952ef47 HTTP/1.1 +GET /druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z HTTP/1.1 Host: {domain} ``` @@ -577,347 +575,152 @@ Host: {domain} ```json { - "task": "query-32663269-ead9-405a-8eb6-0817a952ef47", + "task": "index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z", "payload": { - "type": "query_controller", - "id": "query-32663269-ead9-405a-8eb6-0817a952ef47", + "type": "index_parallel", + "id": "index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z", + "groupId": "index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z", + "resource": { + "availabilityGroup": "index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z", + "requiredCapacity": 1 + }, "spec": { - "query": { - "queryType": "scan", - "dataSource": { - "type": "external", - "inputSource": { - "type": "http", - "uris": [ - "https://druid.apache.org/data/wikipedia.json.gz" - ] - }, - "inputFormat": { - "type": "json", - "keepNullColumns": false, - "assumeNewlineDelimited": false, - "useJsonNodeReader": false - }, - "signature": [ - { - "name": "added", - "type": "LONG" - }, - { - "name": "channel", - "type": "STRING" - }, + "dataSchema": { + "dataSource": "wikipedia_short", + "timestampSpec": { + "column": "time", + "format": "iso", + "missingValue": null + }, + "dimensionsSpec": { + "dimensions": [ { + "type": "string", "name": "cityName", - "type": "STRING" - }, - { - "name": "comment", - "type": "STRING" - }, - { - "name": "commentLength", - "type": "LONG" - }, - { - "name": "countryIsoCode", - "type": "STRING" + "multiValueHandling": "SORTED_ARRAY", + "createBitmapIndex": true }, { + "type": "string", "name": "countryName", - "type": "STRING" - }, - { - "name": "deleted", - "type": "LONG" - }, - { - "name": "delta", - "type": "LONG" - }, - { - "name": "deltaBucket", - "type": "STRING" - }, - { - "name": "diffUrl", - "type": "STRING" - }, - { - "name": "flags", - "type": "STRING" - }, - { - "name": "isAnonymous", - "type": "STRING" - }, - { - "name": "isMinor", - "type": "STRING" - }, - { - "name": "isNew", - "type": "STRING" - }, - { - "name": "isRobot", - "type": "STRING" - }, - { - "name": "isUnpatrolled", - "type": "STRING" - }, - { - "name": "metroCode", - "type": "STRING" - }, - { - "name": "namespace", - "type": "STRING" - }, - { - "name": "page", - "type": "STRING" - }, - { - "name": "regionIsoCode", - "type": "STRING" + "multiValueHandling": "SORTED_ARRAY", + "createBitmapIndex": true }, { + "type": "string", "name": "regionName", - "type": "STRING" - }, - { - "name": "timestamp", - "type": "STRING" - }, - { - "name": "user", - "type": "STRING" + "multiValueHandling": "SORTED_ARRAY", + "createBitmapIndex": true } - ] + ], + "dimensionExclusions": [ + "__time", + "time" + ], + "includeAllDimensions": false, + "useSchemaDiscovery": false }, - "intervals": { - "type": "intervals", + "metricsSpec": [], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "DAY", + "queryGranularity": { + "type": "none" + }, + "rollup": false, "intervals": [ - "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z" + "2015-09-12T00:00:00.000Z/2015-09-13T00:00:00.000Z" ] }, - "virtualColumns": [ - { - "type": "expression", - "name": "v0", - "expression": "timestamp_parse(\"timestamp\",null,'UTC')", - "outputType": "LONG" - } - ], - "resultFormat": "compactedList", - "columns": [ - "added", - "channel", - "cityName", - "comment", - "commentLength", - "countryIsoCode", - "countryName", - "deleted", - "delta", - "deltaBucket", - "diffUrl", - "flags", - "isAnonymous", - "isMinor", - "isNew", - "isRobot", - "isUnpatrolled", - "metroCode", - "namespace", - "page", - "regionIsoCode", - "regionName", - "timestamp", - "user", - "v0" - ], - "legacy": false, - "context": { - "finalize": true, - "maxNumTasks": 3, - "maxParseExceptions": 0, - "queryId": "32663269-ead9-405a-8eb6-0817a952ef47", - "scanSignature": "[{\"name\":\"added\",\"type\":\"LONG\"},{\"name\":\"channel\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"comment\",\"type\":\"STRING\"},{\"name\":\"commentLength\",\"type\":\"LONG\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"deleted\",\"type\":\"LONG\"},{\"name\":\"delta\",\"type\":\"LONG\"},{\"name\":\"deltaBucket\",\"type\":\"STRING\"},{\"name\":\"diffUrl\",\"type\":\"STRING\"},{\"name\":\"flags\",\"type\":\"STRING\"},{\"name\":\"isAnonymous\",\"type\":\"STRING\"},{\"name\":\"isMinor\",\"type\":\"STRING\"},{\"name\":\"isNew\",\"type\":\"STRING\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"isUnpatrolled\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"STRING\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"page\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"user\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"LONG\"}]", - "sqlInsertSegmentGranularity": "\"DAY\"", - "sqlQueryId": "32663269-ead9-405a-8eb6-0817a952ef47" - }, - "granularity": { - "type": "all" + "transformSpec": { + "filter": null, + "transforms": [] } }, - "columnMappings": [ - { - "queryColumn": "v0", - "outputColumn": "__time" - }, - { - "queryColumn": "added", - "outputColumn": "added" - }, - { - "queryColumn": "channel", - "outputColumn": "channel" - }, - { - "queryColumn": "cityName", - "outputColumn": "cityName" - }, - { - "queryColumn": "comment", - "outputColumn": "comment" - }, - { - "queryColumn": "commentLength", - "outputColumn": "commentLength" - }, - { - "queryColumn": "countryIsoCode", - "outputColumn": "countryIsoCode" - }, - { - "queryColumn": "countryName", - "outputColumn": "countryName" - }, - { - "queryColumn": "deleted", - "outputColumn": "deleted" - }, - { - "queryColumn": "delta", - "outputColumn": "delta" - }, - { - "queryColumn": "deltaBucket", - "outputColumn": "deltaBucket" - }, - { - "queryColumn": "diffUrl", - "outputColumn": "diffUrl" - }, - { - "queryColumn": "flags", - "outputColumn": "flags" - }, - { - "queryColumn": "isAnonymous", - "outputColumn": "isAnonymous" - }, - { - "queryColumn": "isMinor", - "outputColumn": "isMinor" - }, - { - "queryColumn": "isNew", - "outputColumn": "isNew" + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "baseDir": "quickstart/tutorial", + "filter": "wikiticker-2015-09-12-sampled.json.gz" }, - { - "queryColumn": "isRobot", - "outputColumn": "isRobot" + "inputFormat": { + "type": "json", + "keepNullColumns": false, + "assumeNewlineDelimited": false, + "useJsonNodeReader": false }, - { - "queryColumn": "isUnpatrolled", - "outputColumn": "isUnpatrolled" - }, - { - "queryColumn": "metroCode", - "outputColumn": "metroCode" - }, - { - "queryColumn": "namespace", - "outputColumn": "namespace" - }, - { - "queryColumn": "page", - "outputColumn": "page" + "appendToExisting": false, + "dropExisting": false + }, + "tuningConfig": { + "type": "index_parallel", + "maxRowsPerSegment": 5000000, + "appendableIndexSpec": { + "type": "onheap", + "preserveExistingMetrics": false }, - { - "queryColumn": "regionIsoCode", - "outputColumn": "regionIsoCode" + "maxRowsInMemory": 25000, + "maxBytesInMemory": 0, + "skipBytesInMemoryOverheadCheck": false, + "maxTotalRows": null, + "numShards": null, + "splitHintSpec": null, + "partitionsSpec": { + "type": "dynamic", + "maxRowsPerSegment": 5000000, + "maxTotalRows": null }, - { - "queryColumn": "regionName", - "outputColumn": "regionName" + "indexSpec": { + "bitmap": { + "type": "roaring" + }, + "dimensionCompression": "lz4", + "stringDictionaryEncoding": { + "type": "utf8" + }, + "metricCompression": "lz4", + "longEncoding": "longs" }, - { - "queryColumn": "timestamp", - "outputColumn": "timestamp" + "indexSpecForIntermediatePersists": { + "bitmap": { + "type": "roaring" + }, + "dimensionCompression": "lz4", + "stringDictionaryEncoding": { + "type": "utf8" + }, + "metricCompression": "lz4", + "longEncoding": "longs" }, - { - "queryColumn": "user", - "outputColumn": "user" - } - ], - "destination": { - "type": "dataSource", - "dataSource": "wikipedia_api", - "segmentGranularity": "DAY" - }, - "assignmentStrategy": "max", - "tuningConfig": { - "maxNumWorkers": 2, - "maxRowsInMemory": 100000, - "rowsPerSegment": 3000000 + "maxPendingPersists": 0, + "forceGuaranteedRollup": false, + "reportParseExceptions": false, + "pushTimeout": 0, + "segmentWriteOutMediumFactory": null, + "maxNumConcurrentSubTasks": 1, + "maxRetry": 3, + "taskStatusCheckPeriodMs": 1000, + "chatHandlerTimeout": "PT10S", + "chatHandlerNumRetries": 5, + "maxNumSegmentsToMerge": 100, + "totalNumMergeTasks": 10, + "logParseExceptions": false, + "maxParseExceptions": 2147483647, + "maxSavedParseExceptions": 0, + "maxColumnsToMerge": -1, + "awaitSegmentAvailabilityTimeoutMillis": 0, + "maxAllowedLockCount": -1, + "partitionDimensions": [] } }, - "sqlQuery": "\nINSERT INTO wikipedia_api \nSELECT \n TIME_PARSE(\"timestamp\") AS __time,\n * \nFROM TABLE(EXTERN(\n '{\"type\": \"http\", \"uris\": [\"https://druid.apache.org/data/wikipedia.json.gz\"]}', \n '{\"type\": \"json\"}', \n '[{\"name\": \"added\", \"type\": \"long\"}, {\"name\": \"channel\", \"type\": \"string\"}, {\"name\": \"cityName\", \"type\": \"string\"}, {\"name\": \"comment\", \"type\": \"string\"}, {\"name\": \"commentLength\", \"type\": \"long\"}, {\"name\": \"countryIsoCode\", \"type\": \"string\"}, {\"name\": \"countryName\", \"type\": \"string\"}, {\"name\": \"deleted\", \"type\": \"long\"}, {\"name\": \"delta\", \"type\": \"long\"}, {\"name\": \"deltaBucket\", \"type\": \"string\"}, {\"name\": \"diffUrl\", \"type\": \"string\"}, {\"name\": \"flags\", \"type\": \"string\"}, {\"name\": \"isAnonymous\", \"type\": \"string\"}, {\"name\": \"isMinor\", \"type\": \"string\"}, {\"name\": \"isNew\", \"type\": \"string\"}, {\"name\": \"isRobot\", \"type\": \"string\"}, {\"name\": \"isUnpatrolled\", \"type\": \"string\"}, {\"name\": \"metroCode\", \"type\": \"string\"}, {\"name\": \"namespace\", \"type\": \"string\"}, {\"name\": \"page\", \"type\": \"string\"}, {\"name\": \"regionIsoCode\", \"type\": \"string\"}, {\"name\": \"regionName\", \"type\": \"string\"}, {\"name\": \"timestamp\", \"type\": \"string\"}, {\"name\": \"user\", \"type\": \"string\"}]'\n ))\nPARTITIONED BY DAY\n", - "sqlQueryContext": { - "sqlQueryId": "32663269-ead9-405a-8eb6-0817a952ef47", - "sqlInsertSegmentGranularity": "\"DAY\"", - "maxNumTasks": 3, - "queryId": "32663269-ead9-405a-8eb6-0817a952ef47" - }, - "sqlResultsContext": { - "timeZone": "UTC", - "serializeComplexValues": true, - "stringifyArrays": true - }, - "sqlTypeNames": [ - "TIMESTAMP", - "BIGINT", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "BIGINT", - "VARCHAR", - "VARCHAR", - "BIGINT", - "BIGINT", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR", - "VARCHAR" - ], "context": { "forceTimeChunkLock": true, "useLineageBasedSegmentAllocation": true }, - "groupId": "query-32663269-ead9-405a-8eb6-0817a952ef47", - "dataSource": "wikipedia_api", - "resource": { - "availabilityGroup": "query-32663269-ead9-405a-8eb6-0817a952ef47", - "requiredCapacity": 1 - } + "dataSource": "wikipedia_short" } - } +} ``` @@ -1037,6 +840,8 @@ A successful request returns a `200 OK` response and an array of the task segmen Retrieves the event log associated with a task. It returns a list of logged events during the lifecycle of the task. The endpoint is useful for providing information about the execution of the task, including any errors or warnings raised. +Task logs are automatically retrieved from the MiddleManager/Indexer or in long-term storage. For reference, see [Task logs](../ingestion/tasks.md#task-logs). + #### Query parameters * `offset` (optional) * Type: Int @@ -1221,7 +1026,9 @@ Host: {domain} POST `/druid/indexer/v1/task` -Submits a task or supervisor spec to the Overlord. It returns the task ID of the submitted task. +Submits a JSON-based ingestion spec or supervisor spec to the Overlord. It returns the task ID of the submitted task. For information on creating an ingestion spec, refer to the [ingestion spec reference](../ingestion/ingestion-spec.md). + +Note that for most batch ingestion use cases, you should prefer to use [SQL-ingestion API](./sql-ingestion-api.md). #### Responses From 428e98695b9f7a7d656fae7e9de7fe98d4d4d44c Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Fri, 7 Jul 2023 15:23:58 -0700 Subject: [PATCH 11/14] Link out to task table --- docs/api-reference/tasks-api.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 856e5a9d9cc5..9e3c16f691a0 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -23,7 +23,7 @@ sidebar_label: Tasks ~ under the License. --> -This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete data-related operations such as ingestion and compaction. +This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete operations such as ingestion and compaction. In this document, `{domain}` is a placeholder for the server address of deployment. For example, on the quickstart configuration, replace `{domain}` with `http://localhost:8888`. @@ -34,7 +34,7 @@ In this document, `{domain}` is a placeholder for the server address of deployme #### URL GET `/druid/indexer/v1/tasks` -Retrieves an array of all tasks in the Druid cluster. Each task object includes information on its ID, status, associated datasource, and other metadata. +Retrieves an array of all tasks in the Druid cluster. Each task object includes information on its ID, status, associated datasource, and other metadata. For definitions of the response properties, see the [Tasks table](../querying/sql-metadata-tables.md#tasks-table). #### Query parameters @@ -159,7 +159,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/completeTasks` -Retrieves an array of completed tasks in the Druid cluster. This is functionally equivalent to `/druid/indexer/v1/tasks?state=complete`. +Retrieves an array of completed tasks in the Druid cluster. This is functionally equivalent to `/druid/indexer/v1/tasks?state=complete`. For definitions of the response properties, see the [Tasks table](../querying/sql-metadata-tables.md#tasks-table). #### Query parameters @@ -254,7 +254,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/runningTasks` -Retrieves an array of running task objects in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=running`. +Retrieves an array of running task objects in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=running`. For definitions of the response properties, see the [Tasks table](../querying/sql-metadata-tables.md#tasks-table). #### Query parameters @@ -330,7 +330,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/waitingTasks` -Retrieves an array of waiting tasks in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=waiting`. +Retrieves an array of waiting tasks in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=waiting`. For definitions of the response properties, see the [Tasks table](../querying/sql-metadata-tables.md#tasks-table). #### Query parameters @@ -442,7 +442,7 @@ Host: {domain} GET `/druid/indexer/v1/pendingTasks` -Retrieves an array of pending tasks in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=pending`. +Retrieves an array of pending tasks in the Druid cluster. It is functionally equivalent to `/druid/indexer/v1/tasks?state=pending`. For definitions of the response properties, see the [Tasks table](../querying/sql-metadata-tables.md#tasks-table). #### Query parameters @@ -730,7 +730,7 @@ Host: {domain} #### URL GET `/druid/indexer/v1/task/{taskId}/status` -Retrieves the status of a task given the task ID. It returns a JSON object with the task's current state, task type, datasource, and other relevant metadata. +Retrieves the status of a task given the task ID. It returns a JSON object with the task's status code, runner status, task type, datasource, and other relevant metadata. #### Responses @@ -840,7 +840,7 @@ A successful request returns a `200 OK` response and an array of the task segmen Retrieves the event log associated with a task. It returns a list of logged events during the lifecycle of the task. The endpoint is useful for providing information about the execution of the task, including any errors or warnings raised. -Task logs are automatically retrieved from the MiddleManager/Indexer or in long-term storage. For reference, see [Task logs](../ingestion/tasks.md#task-logs). +Task logs are automatically retrieved from the Middle Manager/Indexer or in long-term storage. For reference, see [Task logs](../ingestion/tasks.md#task-logs). #### Query parameters * `offset` (optional) From 6bfbeb9a110ecbaa7608fcfbc086d7a527526ea1 Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Fri, 7 Jul 2023 15:26:02 -0700 Subject: [PATCH 12/14] Actually link out --- docs/api-reference/tasks-api.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 9e3c16f691a0..654d1dd01680 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -23,7 +23,7 @@ sidebar_label: Tasks ~ under the License. --> -This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete operations such as ingestion and compaction. +This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete operations such as ingestion, querying, and compaction. In this document, `{domain}` is a placeholder for the server address of deployment. For example, on the quickstart configuration, replace `{domain}` with `http://localhost:8888`. From 578dec2487a026c8647e338954b43a7446c012bd Mon Sep 17 00:00:00 2001 From: demo-kratia <56242907+demo-kratia@users.noreply.github.com> Date: Tue, 11 Jul 2023 10:21:22 -0700 Subject: [PATCH 13/14] Update {domain} to specific service port --- docs/api-reference/tasks-api.md | 64 ++++++++++++++++----------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 654d1dd01680..90924cce2d1c 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -25,7 +25,7 @@ sidebar_label: Tasks This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete operations such as ingestion, querying, and compaction. -In this document, `{domain}` is a placeholder for the server address of deployment. For example, on the quickstart configuration, replace `{domain}` with `http://localhost:8888`. +In this document, `http://:` is a placeholder for the server address of deployment and the service port. For example, on the quickstart configuration, replace `http://:` with `http://localhost:8888`. ## Task information and retrieval @@ -78,12 +78,12 @@ The following example shows how to retrieve a list of tasks filtered with the fo ```shell -curl "{domain}/druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker" +curl "http://:/druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker" ``` ```HTTP GET /druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -192,12 +192,12 @@ The endpoint supports a set of optional query parameters to filter results. ```shell -curl "{domain}/druid/indexer/v1/completeTasks" +curl "http://:/druid/indexer/v1/completeTasks" ``` ```HTTP GET /druid/indexer/v1/completeTasks HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -285,12 +285,12 @@ The endpoint supports a set of optional query parameters to filter results. ```shell -curl "{domain}/druid/indexer/v1/runningTasks" +curl "http://:/druid/indexer/v1/runningTasks" ``` ```HTTP GET /druid/indexer/v1/runningTasks HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -361,12 +361,12 @@ The endpoint supports a set of optional query parameters to filter results. ```shell -curl "{domain}/druid/indexer/v1/waitingTasks" +curl "http://:/druid/indexer/v1/waitingTasks" ``` ```HTTP GET /druid/indexer/v1/waitingTasks HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -472,12 +472,12 @@ The endpoint supports a set of optional query parameters to filter results. ```shell -curl "{domain}/druid/indexer/v1/pendingTasks" +curl "http://:/druid/indexer/v1/pendingTasks" ``` ```HTTP GET /druid/indexer/v1/pendingTasks HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -558,12 +558,12 @@ The following examples shows how to retrieve the task payload of a task with the ```shell -curl "{domain}/druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z" +curl "http://:/druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z" ``` ```HTTP GET /druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -753,12 +753,12 @@ The following examples shows how to retrieve the status of a task with the speci ```shell -curl "{domain}/druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status" +curl "http://:/druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status" ``` ```HTTP GET /druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -819,12 +819,12 @@ The following examples shows how to retrieve the task segment of the task with t ```shell -curl "{domain}/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports" +curl "http://:/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports" ``` ```HTTP GET /druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -866,12 +866,12 @@ The following examples shows how to retrieve the task log of a task with the spe ```shell -curl "{domain}/druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log" +curl "http://:/druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log" ``` ```HTTP GET /druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -971,12 +971,12 @@ The following examples shows how to retrieve the completion report of a task wit ```shell -curl "{domain}/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports" +curl "http://:/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports" ``` ```HTTP GET /druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -1057,7 +1057,7 @@ The following request is an example of submitting a task to create a datasource ```shell -curl "{domain}/druid/indexer/v1/task" \ +curl "http://:/druid/indexer/v1/task" \ --header "Content-Type: application/json" \ --data "{ \"type\" : \"index_parallel\", @@ -1103,7 +1103,7 @@ curl "{domain}/druid/indexer/v1/task" \ ```HTTP POST /druid/indexer/v1/task HTTP/1.1 -Host: {domain} +Host: http://: Content-Type: application/json Content-Length: 952 @@ -1177,7 +1177,7 @@ Shuts down a task if it not already complete. Returns a JSON object with the ID *Successfully shut down task*
-*Cannot find task with ID* +*Cannot find task with ID or task is no longer running* --- @@ -1190,12 +1190,12 @@ The following request shows how to shut down a task with the ID `query-52as 8aaf ```shell -curl --request POST "{domain}/druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown" +curl --request POST "http://:/druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown" ``` ```HTTP POST /druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown HTTP/1.1 -Host: (domain) +Host: http://: ``` @@ -1241,12 +1241,12 @@ The following request is an example of shutting down all tasks for datasource `w ```shell -curl --request POST "{domain}/druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks" +curl --request POST "http://:/druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks" ``` ```HTTP POST /druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks HTTP/1.1 -Host: {domain} +Host: http://: ``` @@ -1294,14 +1294,14 @@ The following request is an example of retrieving status objects for task ID `in ```shell -curl "{domain}/druid/indexer/v1/taskStatus" \ +curl "http://:/druid/indexer/v1/taskStatus" \ --header "Content-Type: application/json" \ --data "[\"index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z\", \"index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z\"]" ``` ```HTTP POST /druid/indexer/v1/taskStatus HTTP/1.1 -Host: {domain} +Host: http://: Content-Type: application/json Content-Length: 134 @@ -1373,12 +1373,12 @@ The following request is an example of cleaning up pending segments for the `wik ```shell -curl --request DELETE "{domain}/druid/indexer/v1/pendingSegments/wikipedia_api" +curl --request DELETE "http://:/druid/indexer/v1/pendingSegments/wikipedia_api" ``` ```HTTP DELETE /druid/indexer/v1/pendingSegments/wikipedia_api HTTP/1.1 -Host: {domain} +Host: http://: ``` From b57376cd038324127404a3f02196353615e605ea Mon Sep 17 00:00:00 2001 From: Charles Smith Date: Tue, 11 Jul 2023 11:34:02 -0700 Subject: [PATCH 14/14] Apply suggestions from code review minor style updates/clarifications --- docs/api-reference/tasks-api.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md index 90924cce2d1c..1d745abfd29e 100644 --- a/docs/api-reference/tasks-api.md +++ b/docs/api-reference/tasks-api.md @@ -44,7 +44,7 @@ The endpoint supports a set of optional query parameters to filter results. |---|---|---| |`state`|String|Filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.| | `datasource`|String| Return tasks filtered by Druid datasource.| -| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. The interval string should be delimited by `_` instead of `/`. For example, `2023-06-27_2023-06-28`.| +| `createdTimeInterval`|String (ISO-8601)| Return tasks created within the specified interval. Use `_` as the delimiter for the interval string. Do not use `/`. For example, `2023-06-27_2023-06-28`.| | `max`|Integer|Maximum number of `complete` tasks to return. Only applies when `state` is set to `complete`.| | `type`|String|Filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.| @@ -1028,7 +1028,7 @@ Host: http://: Submits a JSON-based ingestion spec or supervisor spec to the Overlord. It returns the task ID of the submitted task. For information on creating an ingestion spec, refer to the [ingestion spec reference](../ingestion/ingestion-spec.md). -Note that for most batch ingestion use cases, you should prefer to use [SQL-ingestion API](./sql-ingestion-api.md). +Note that for most batch ingestion use cases, you should use the [SQL-ingestion API](./sql-ingestion-api.md) instead of JSON-based batch ingestion. #### Responses