You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by ab...@apache.org on 2022/09/06 17:38:01 UTC

[druid] branch 24.0.0 updated: msq: add multi-stage-query docs (#12983)

This is an automated email from the ASF dual-hosted git repository.

abhishek pushed a commit to branch 24.0.0
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/24.0.0 by this push:
     new a769458b12 msq: add multi-stage-query docs (#12983)
a769458b12 is described below

commit a769458b1279f07b607d572a344f96ea8a31c9b9
Author: 317brian <53...@users.noreply.github.com>
AuthorDate: Tue Sep 6 10:36:09 2022 -0700

    msq: add multi-stage-query docs (#12983)
    
    * msq: add multi-stage-query docs
    
    * add screenshots
    
    add back theta sketches tutoria
    
    change filename
    
    fix filename
    
    fix link
    
    fix headings
    
    * fixes
    
    * fixes
    
    * fix spelling issues and update spell file
    
    * address feedback from karan
    
    * add missing guardrail to known issues
    
    * update blurb
    
    * fix typo
    
    * remove durable storage info
    
    * update titles
    
    * Restore en.json
    
    * Update query view
    
    * address comments from vad
    
    * Update docs/multi-stage-query/msq-known-issues.md
    
    finish sentence
    
    * add apache license to docs
    
    * add apache license to docs
    
    Co-authored-by: Katya Macedo <ka...@imply.io>
    Co-authored-by: Charles Smith <te...@gmail.com>
---
 .../msq-ui-download-query-results.png              |  Bin 0 -> 62538 bytes
 .../multi-stage-query/tutorial-msq-convert.png     |  Bin 0 -> 80379 bytes
 docs/assets/multi-stage-query/ui-annotated.png     |  Bin 0 -> 227968 bytes
 docs/assets/multi-stage-query/ui-empty.png         |  Bin 0 -> 77306 bytes
 docs/assets/tutorial-quickstart-02.png             |  Bin 0 -> 115511 bytes
 docs/assets/tutorial-quickstart-03.png             |  Bin 0 -> 174445 bytes
 docs/assets/tutorial-quickstart-04.png             |  Bin 0 -> 144088 bytes
 docs/assets/tutorial-quickstart-05.png             |  Bin 0 -> 84116 bytes
 docs/development/extensions.md                     |    1 +
 docs/multi-stage-query/index.md                    |  342 ++++
 docs/multi-stage-query/msq-api.md                  | 1652 ++++++++++++++++++++
 docs/multi-stage-query/msq-concepts.md             |  168 ++
 docs/multi-stage-query/msq-example-queries.md      |  503 ++++++
 docs/multi-stage-query/msq-known-issues.md         |  118 ++
 docs/multi-stage-query/msq-reference.md            |  169 ++
 docs/multi-stage-query/msq-security.md             |   43 +
 .../msq-tutorial-connect-external-data.md          |  145 ++
 .../msq-tutorial-convert-ingest-spec.md            |  169 ++
 docs/operations/druid-console.md                   |   54 +-
 docs/operations/security-user-auth.md              |   40 +-
 docs/tutorials/docker.md                           |    2 +-
 docs/tutorials/index.md                            |  284 ++--
 docs/tutorials/tutorial-batch-native.md            |  155 ++
 docs/tutorials/tutorial-msq-connect-extern.md      |   44 +
 docs/tutorials/tutorial-msq-convert-json.md        |   44 +
 website/.spelling                                  |  100 ++
 website/sidebars.json                              |   25 +-
 27 files changed, 3866 insertions(+), 192 deletions(-)

diff --git a/docs/assets/multi-stage-query/msq-ui-download-query-results.png b/docs/assets/multi-stage-query/msq-ui-download-query-results.png
new file mode 100644
index 0000000000..e428cb2dfd
Binary files /dev/null and b/docs/assets/multi-stage-query/msq-ui-download-query-results.png differ
diff --git a/docs/assets/multi-stage-query/tutorial-msq-convert.png b/docs/assets/multi-stage-query/tutorial-msq-convert.png
new file mode 100644
index 0000000000..f16941af67
Binary files /dev/null and b/docs/assets/multi-stage-query/tutorial-msq-convert.png differ
diff --git a/docs/assets/multi-stage-query/ui-annotated.png b/docs/assets/multi-stage-query/ui-annotated.png
new file mode 100644
index 0000000000..5a98c00d19
Binary files /dev/null and b/docs/assets/multi-stage-query/ui-annotated.png differ
diff --git a/docs/assets/multi-stage-query/ui-empty.png b/docs/assets/multi-stage-query/ui-empty.png
new file mode 100644
index 0000000000..7c30d5a671
Binary files /dev/null and b/docs/assets/multi-stage-query/ui-empty.png differ
diff --git a/docs/assets/tutorial-quickstart-02.png b/docs/assets/tutorial-quickstart-02.png
new file mode 100644
index 0000000000..5edec67c3f
Binary files /dev/null and b/docs/assets/tutorial-quickstart-02.png differ
diff --git a/docs/assets/tutorial-quickstart-03.png b/docs/assets/tutorial-quickstart-03.png
new file mode 100644
index 0000000000..cff8fecfc1
Binary files /dev/null and b/docs/assets/tutorial-quickstart-03.png differ
diff --git a/docs/assets/tutorial-quickstart-04.png b/docs/assets/tutorial-quickstart-04.png
new file mode 100644
index 0000000000..431d8b97c8
Binary files /dev/null and b/docs/assets/tutorial-quickstart-04.png differ
diff --git a/docs/assets/tutorial-quickstart-05.png b/docs/assets/tutorial-quickstart-05.png
new file mode 100644
index 0000000000..6178289fdc
Binary files /dev/null and b/docs/assets/tutorial-quickstart-05.png differ
diff --git a/docs/development/extensions.md b/docs/development/extensions.md
index 49f61c8d74..7a8175fa55 100644
--- a/docs/development/extensions.md
+++ b/docs/development/extensions.md
@@ -50,6 +50,7 @@ Core extensions are maintained by Druid committers.
 |druid-kerberos|Kerberos authentication for druid processes.|[link](../development/extensions-core/druid-kerberos.md)|
 |druid-lookups-cached-global|A module for [lookups](../querying/lookups.md) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.md)|
 |druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.md)|
+|druid-multi-stage-query| Support for the multi-stage query architecture for Apache Druid and the multi-stage query task engine.|[link](../multi-stage-query/index.md)|
 |druid-orc-extensions|Support for data in Apache ORC data format.|[link](../development/extensions-core/orc.md)|
 |druid-parquet-extensions|Support for data in Apache Parquet data format. Requires druid-avro-extensions to be loaded.|[link](../development/extensions-core/parquet.md)|
 |druid-protobuf-extensions| Support for data in Protobuf data format.|[link](../development/extensions-core/protobuf.md)|
diff --git a/docs/multi-stage-query/index.md b/docs/multi-stage-query/index.md
new file mode 100644
index 0000000000..b82408607f
--- /dev/null
+++ b/docs/multi-stage-query/index.md
@@ -0,0 +1,342 @@
+---
+id: index
+title: SQL-based ingestion overview and syntax
+sidebar_label: Overview and syntax
+description: Introduces multi-stage query architecture and its task engine
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+SQL-based ingestion for Apache Druid uses a distributed multi-stage query architecture, which includes a query engine called the multi-stage query task engine (MSQ task engine). The MSQ task engine extends Druid's query capabilities, so you can write queries that reference [external data](#read-external-data) as well as perform ingestion with SQL [INSERT](#insert-data) and [REPLACE](#replace-data). Essentially, you can perform SQL-based ingestion instead of using JSON ingestion specs tha [...]
+
+The MSQ task engine excels at executing queries that can get bottlenecked at the Broker when using Druid's native SQL engine. When you submit queries, the MSQ task engine splits them into stages and automatically exchanges data between stages. Each stage is parallelized to run across multiple data servers at once, simplifying performance.
+
+
+## MSQ task engine features
+
+In its current state, the MSQ task engine enables you to do the following:
+
+- Read external data at query time using EXTERN.
+- Execute batch ingestion jobs by writing SQL queries using INSERT and REPLACE. You no longer need to generate a JSON-based ingestion spec.
+- Transform and rewrite existing tables using SQL.
+- Perform multi-dimension range partitioning reliably, which leads to more evenly distributed segment sizes and better performance.
+
+The MSQ task engine has additional features that can be used as part of a proof of concept or demo, but don't use or rely on the following features for any meaningful use cases, especially production ones:
+
+- Execute heavy-weight queries and return large numbers of rows.
+- Execute queries that exchange large amounts of data between servers, like exact count distinct of high-cardinality fields.
+
+
+## Load the extension
+
+For new clusters that use 24.0 or later, the multi-stage query extension is loaded by default. If you want to add the extension to an existing cluster, add the extension `druid-multi-stage-query` to `druid.extensions.loadlist` in your `common.runtime.properties` file.
+
+For more information about how to load an extension, see [Loading extensions](../development/extensions.md#loading-extensions).
+
+To use EXTERN, you need READ permission on the resource named "EXTERNAL" of the resource type "EXTERNAL". If you encounter a 403 error when trying to use EXTERN, verify that you have the correct permissions.
+
+## MSQ task engine query syntax
+
+You can submit queries to the MSQ task engine through the **Query** view in the Druid console or through the API. The Druid console is a good place to start because you can preview a query before you run it. You can also experiment with many of the [context parameters](./msq-reference.md#context-parameters) through the UI. Once you're comfortable with submitting queries through the Druid console, [explore using the API to submit a query](./msq-api.md#submit-a-query).
+
+If you encounter an issue after you submit a query, you can learn more about what an error means from the [limits](./msq-concepts.md#limits) and [errors](./msq-concepts.md#error-codes). 
+
+Queries for the MSQ task engine involve three primary functions:
+
+- EXTERN to query external data
+- INSERT INTO ... SELECT to insert data, such as data from an external source
+- REPLACE to replace existing datasources, partially or fully, with query results
+
+For information about the syntax for queries, see [SQL syntax](./msq-reference.md#sql-syntax).
+
+### Read external data
+
+Query tasks can access external data through the EXTERN function. When using EXTERN, keep in mind that  large files do not get split across different worker tasks. If you have fewer input files than worker tasks, you can increase query parallelism by splitting up your input files such that you have at least one input file per worker task.
+
+You can use the EXTERN function anywhere a table is expected in the following form: `TABLE(EXTERN(...))`. You can use external data with SELECT, INSERT, and REPLACE queries.
+
+The following query reads external data:
+
+```sql
+SELECT
+  *
+FROM TABLE(
+  EXTERN(
+    '{"type": "http", "uris": ["https://static.imply.io/data/wikipedia.json.gz"]}',
+    '{"type": "json"}',
+    '[{"name": "timestamp", "type": "string"}, {"name": "page", "type": "string"}, {"name": "user", "type": "string"}]'
+  )
+)
+LIMIT 100
+``` 
+
+For more information about the syntax, see [EXTERN](./msq-reference.md#extern).
+
+### Insert data
+
+With the MSQ task engine, Druid can use the results of a query task to create a new datasource or to append to an existing datasource. Syntactically, there is no difference between the two. These operations use the INSERT INTO ... SELECT syntax.
+
+All SELECT capabilities are available for INSERT queries. However, the MSQ task engine does not include all the existing SQL query features of Druid. See [Known issues](./msq-known-issues.md) for a list of capabilities that aren't available.
+
+The following example query inserts data from an external source into a table named `w000` and partitions it by day:
+
+```sql
+INSERT INTO w000
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  "page",
+  "user"
+FROM TABLE(
+  EXTERN(
+    '{"type": "http", "uris": ["https://static.imply.io/data/wikipedia.json.gz"]}',
+    '{"type": "json"}',
+    '[{"name": "timestamp", "type": "string"}, {"name": "page", "type": "string"}, {"name": "user", "type": "string"}]'
+  )
+)
+PARTITIONED BY DAY
+```
+
+For more information about the syntax, see [INSERT](./msq-reference.md#insert).
+
+### Replace data 
+
+The syntax for REPLACE is similar to INSERT. All SELECT functionality is available for REPLACE queries.
+Note that the MSQ task engine does not yet implement all native Druid query features.
+For details, see [Known issues](./msq-known-issues.md).
+
+When working with REPLACE queries, keep the following in mind:
+
+- The intervals generated as a result of the OVERWRITE WHERE query must align with the granularity specified in the PARTITIONED BY clause.
+- OVERWRITE WHERE queries only support the `__time` column.
+
+For more information about the syntax, see [REPLACE](./msq-reference.md#replace).
+
+The following examples show how to replace data in a table.
+
+#### REPLACE all data
+
+You can replace all the data in a table by using REPLACE INTO ... OVERWRITE ALL SELECT:
+
+```sql
+REPLACE INTO w000
+OVERWRITE ALL
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  "page",
+  "user"
+FROM TABLE(
+  EXTERN(
+    '{"type": "http", "uris": ["https://static.imply.io/data/wikipedia.json.gz"]}',
+    '{"type": "json"}',
+    '[{"name": "timestamp", "type": "string"}, {"name": "page", "type": "string"}, {"name": "user", "type": "string"}]'
+  )
+)
+PARTITIONED BY DAY
+```
+
+#### REPLACE some data
+
+You can replace some of the data in a table by using REPLACE INTO ... OVERWRITE WHERE ... SELECT:
+
+```sql
+REPLACE INTO w000
+OVERWRITE WHERE __time >= TIMESTAMP '2019-08-25' AND __time < TIMESTAMP '2019-08-28'
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  "page",
+  "user"
+FROM TABLE(
+  EXTERN(
+    '{"type": "http", "uris": ["https://static.imply.io/data/wikipedia.json.gz"]}',
+    '{"type": "json"}',
+    '[{"name": "timestamp", "type": "string"}, {"name": "page", "type": "string"}, {"name": "user", "type": "string"}]'
+  )
+)
+PARTITIONED BY DAY
+```
+
+## Adjust query behavior
+
+In addition to the basic functions, you can further modify your query behavior to control how your queries run or what your results look like. You can control how your queries behave by changing the following:
+
+### Primary timestamp
+
+Druid tables always include a primary timestamp named `__time`, so your ingestion query should generally include a column named `__time`. 
+
+The following formats are supported for `__time` in the source data:
+- ISO 8601 with 'T' separator, such as "2000-01-01T01:02:03.456"
+- Milliseconds since Unix epoch (00:00:00 UTC on January 1, 1970)
+
+The `__time` column is used for time-based partitioning, such as `PARTITIONED BY DAY`.
+
+If you use `PARTITIONED BY ALL` or `PARTITIONED BY ALL TIME`, time-based
+partitioning is disabled. In these cases, your ingestion query doesn't need
+to include a `__time` column. However, Druid still creates a `__time` column 
+in your Druid table and sets all timestamps to 1970-01-01 00:00:00.
+
+For more information, see [Primary timestamp](../ingestion/data-model.md#primary-timestamp).
+
+### PARTITIONED BY
+
+INSERT and REPLACE queries require the PARTITIONED BY clause, which determines how time-based partitioning is done. In Druid, data is split into segments, one or more per time chunk defined by the PARTITIONED BY granularity. A good general rule is to adjust the granularity so that each segment contains about five million rows. Choose a granularity based on your ingestion rate. For example, if you ingest a million rows per day, PARTITION BY DAY is good. If you ingest a million rows an hou [...]
+
+Using the clause provides the following benefits:
+
+- Better query performance due to time-based segment pruning, which removes segments from
+   consideration when they do not contain any data for a query's time filter.
+- More efficient data management, as data can be rewritten for each time partition individually
+   rather than the whole table.
+
+You can use the following arguments for PARTITIONED BY:
+
+- Time unit: `HOUR`, `DAY`, `MONTH`, or `YEAR`. Equivalent to `FLOOR(__time TO TimeUnit)`.
+- `TIME_FLOOR(__time, 'granularity_string')`, where granularity_string is an ISO 8601 period like
+  'PT1H'. The first argument must be `__time`.
+- `FLOOR(__time TO TimeUnit)`, where `TimeUnit` is any unit supported by the [FLOOR function](../querying/sql-scalar.md#date-and-time-functions). The first
+  argument must be `__time`.
+- `ALL` or `ALL TIME`, which effectively disables time partitioning by placing all data in a single
+  time chunk. To use LIMIT or OFFSET at the outer level of your INSERT or REPLACE query, you must set PARTITIONED BY to ALL or ALL TIME.
+
+You can use the following ISO 8601 periods for `TIME_FLOOR`:
+
+- PT1S
+- PT1M
+- PT5M
+- PT10M
+- PT15M
+- PT30M
+- PT1H
+- PT6H
+- P1D
+- P1W
+- P1M
+- P3M
+- P1Y
+
+
+### CLUSTERED BY
+
+Data is first divided by the PARTITION BY clause. Data can be further split by the CLUSTERED BY clause. For example, suppose you ingest 100 M rows per hour and use `PARTITIONED BY HOUR` as your time partition. You then divide up the data further by adding `CLUSTERED BY hostName`. The result is segments of about 5 million rows, with like `hostNames` grouped within the same segment.
+
+Using CLUSTERED BY has the following benefits:
+
+- Lower storage footprint due to combining similar data into the same segments, which improves
+   compressibility.
+- Better query performance due to dimension-based segment pruning, which removes segments from
+   consideration when they cannot possibly contain data matching a query's filter.
+
+For dimension-based segment pruning to be effective, your queries should meet the following conditions:
+
+- All CLUSTERED BY columns are single-valued string columns
+- Use a REPLACE query for ingestion
+
+Druid still clusters data during ingestion if these conditions aren't met but won't perform dimension-based segment pruning at query time. That means if you use an INSERT query for ingestion or have numeric columns or multi-valued string columns, dimension-based segment pruning doesn't occur at query time.
+
+You can tell if dimension-based segment pruning is possible by using the `sys.segments` table to
+inspect the `shard_spec` for the segments generated by an ingestion query. If they are of type
+`range` or `single`, then dimension-based segment pruning is possible. Otherwise, it is not. The
+shard spec type is also available in the **Segments** view under the **Partitioning**
+column.
+
+You can use the following filters for dimension-based segment pruning:
+
+- Equality to string literals, like `x = 'foo'` or `x IN ('foo', 'bar')`.
+- Comparison to string literals, like `x < 'foo'` or other comparisons involving `<`, `>`, `<=`, or `>=`.
+
+This differs from multi-dimension range based partitioning in classic batch ingestion where both
+string and numeric columns support Broker-level pruning. With SQL-based batch ingestion,
+only string columns support Broker-level pruning.
+
+It is okay to mix time partitioning with secondary partitioning. For example, you can
+combine `PARTITIONED BY HOUR` with `CLUSTERED BY channel` to perform
+time partitioning by hour and secondary partitioning by channel within each hour.
+
+### GROUP BY
+
+A query's GROUP BY clause determines how data is rolled up. The expressions in the GROUP BY clause become
+dimensions, and aggregation functions become metrics.
+
+### Ingest-time aggregations
+
+When performing rollup using aggregations, it is important to use aggregators
+that return nonfinalized state. This allows you to perform further rollups
+at query time. To achieve this, set `finalizeAggregations: false` in your
+ingestion query context.
+
+Check out the [INSERT with rollup example query](./msq-example-queries.md#insert-with-rollup) to see this feature in
+action.
+
+Druid needs information for aggregating measures of different segments to compact. For example, to aggregate `count("col") as example_measure`, Druid needs to sum the value of `example_measure`
+across the segments. This information is stored inside the metadata of the segment. For the SQL-based ingestion, Druid only populates the
+aggregator information of a column in the segment metadata when:
+
+- The INSERT or REPLACE query has an outer GROUP BY clause.
+- The following context parameters are set for the query context: `finalizeAggregations: false` and `groupByEnableMultiValueUnnesting: false`
+
+The following table lists query-time aggregations for SQL-based ingestion:
+
+|Query-time aggregation|Notes|
+|----------------------|-----|
+|SUM|Use unchanged at ingest time.|
+|MIN|Use unchanged at ingest time.|
+|MAX|Use unchanged at ingest time.|
+|AVG|Use SUM and COUNT at ingest time. Switch to quotient of SUM at query time.|
+|COUNT|Use unchanged at ingest time, but switch to SUM at query time.|
+|COUNT(DISTINCT expr)|If approximate, use APPROX_COUNT_DISTINCT at ingest time.<br /><br />If exact, you cannot use an ingest-time aggregation. Instead, `expr` must be stored as-is. Add it to the SELECT and GROUP BY lists.|
+|EARLIEST(expr)<br /><br />(numeric form)|Not supported.|
+|EARLIEST(expr, maxBytes)<br /><br />(string form)|Use unchanged at ingest time.|
+|LATEST(expr)<br /><br />(numeric form)|Not supported.|
+|LATEST(expr, maxBytes)<br /><br />(string form)|Use unchanged at ingest time.|
+|APPROX_COUNT_DISTINCT|Use unchanged at ingest time.|
+|APPROX_COUNT_DISTINCT_BUILTIN|Use unchanged at ingest time.|
+|APPROX_COUNT_DISTINCT_DS_HLL|Use unchanged at ingest time.|
+|APPROX_COUNT_DISTINCT_DS_THETA|Use unchanged at ingest time.|
+|APPROX_QUANTILE|Not supported. Deprecated; use APPROX_QUANTILE_DS instead.|
+|APPROX_QUANTILE_DS|Use DS_QUANTILES_SKETCH at ingest time. Continue using APPROX_QUANTILE_DS at query time.|
+|APPROX_QUANTILE_FIXED_BUCKETS|Not supported.|
+
+### Multi-value dimensions
+
+By default, multi-value dimensions are not ingested as expected when rollup is enabled because the
+GROUP BY operator unnests them instead of leaving them as arrays. This is [standard behavior](../querying/sql-data-types.md#multi-value-strings) for GROUP BY but it is generally not desirable behavior for ingestion.
+
+To address this:
+
+- When using GROUP BY with data from EXTERN, wrap any string type fields from EXTERN that may be
+  multi-valued in `MV_TO_ARRAY`.
+- Set `groupByEnableMultiValueUnnesting: false` in your query context to ensure that all multi-value
+  strings are properly converted to arrays using `MV_TO_ARRAY`. If any strings aren't
+  wrapped in `MV_TO_ARRAY`, the query reports an error that includes the message "Encountered
+  multi-value dimension x that cannot be processed with groupByEnableMultiValueUnnesting set to false."
+
+For an example, see [INSERT with rollup example query](./msq-example-queries.md#insert-with-rollup).
+
+### Context parameters
+
+Context parameters can control things such as how many tasks get launched or what happens if there's a malformed record.
+
+For a full list of context parameters and how they affect a query, see [Context parameters](./msq-reference.md#context-parameters).
+
+## Next steps
+
+* [Understand how the multi-stage query architecture works](./msq-concepts.md) by reading about the concepts behind it and its processes.
+* [Explore the Query view](../operations/druid-console.md) to learn about the UI tools that can help you get started.
\ No newline at end of file
diff --git a/docs/multi-stage-query/msq-api.md b/docs/multi-stage-query/msq-api.md
new file mode 100644
index 0000000000..b9c2f165ac
--- /dev/null
+++ b/docs/multi-stage-query/msq-api.md
@@ -0,0 +1,1652 @@
+---
+id: api
+title: SQL-based ingestion APIs
+sidebar_label: API
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+The **Query** view in the Druid console provides the most stable experience for the multi-stage query task engine (MSQ task engine) and multi-stage query architecture. Use the UI if you do not need a programmatic interface.
+
+When using the API for the MSQ task engine, the action you want to take determines the endpoint you use:
+
+- `/druid/v2/sql/task` endpoint: Submit a query for ingestion.
+- `/druid/indexer/v1/task` endpoint: Interact with a query, including getting its status, getting its details, or canceling it. This page describes a few of the Overlord Task APIs that you can use with the MSQ task engine. For information about Druid APIs, see the [API reference for Druid](../operations/api-reference.md#tasks).
+
+## Submit a query
+
+You submit queries to the MSQ task engine using the `POST /druid/v2/sql/task/` endpoint.
+
+### Request
+
+Currently, the MSQ task engine ignores the provided values of `resultFormat`, `header`,
+`typesHeader`, and `sqlTypesHeader`. SQL SELECT queries write out their results into the task report (in the `multiStageQuery.payload.results.results` key) formatted as if `resultFormat` is an `array`.
+
+For task queries similar to the [example queries](./msq-example-queries.md), you need to escape characters such as quotation marks (") if you use something like `curl`. 
+You don't need to escape characters if you use a method that can parse JSON seamlessly, such as Python.
+The Python example in this topic escapes quotation marks although it's not required.
+
+The following example is the same query that you submit when you complete [Convert a JSON ingestion spec](./msq-tutorial-convert-ingest-spec.md) where you insert data into a table named `wikipedia`. 
+
+<!--DOCUSAURUS_CODE_TABS-->
+
+<!--HTTP-->
+
+```
+POST /druid/v2/sql/task
+```
+
+```json
+{
+  "query": "INSERT INTO wikipedia\nSELECT\n  TIME_PARSE(\"timestamp\") AS __time,\n  *\nFROM TABLE(\n  EXTERN(\n    '{\"type\": \"http\", \"uris\": [\"https://static.imply.io/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\": \"cou [...]
+  "context": {
+      "maxNumTasks": 3
+  }
+}
+```
+
+<!--curl-->
+
+Make sure you replace `username`, `password`, `your-instance`, and `port` with the values for your deployment.
+
+```bash
+curl --location --request POST 'https://<username>:<password>@<your-instance>:<port>/druid/v2/sql/task/' \
+--header 'Content-Type: application/json' \
+--data-raw '{
+    "query": "INSERT INTO wikipedia\nSELECT\n  TIME_PARSE(\"timestamp\") AS __time,\n  *\nFROM TABLE(\n  EXTERN(\n    '\''{\"type\": \"http\", \"uris\": [\"https://static.imply.io/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\"}, [...]
+    "context": {
+        "maxNumTasks": 3
+    }
+```
+
+<!--Python-->
+Make sure you replace `username`, `password`, `your-instance`, and `port` with the values for your deployment.
+
+```python
+import json
+import requests
+
+url = "https://<username>:<password>@<your-instance>:<port>/druid/v2/sql/task/"
+
+payload = json.dumps({
+  "query": "INSERT INTO wikipedia\nSELECT\n  TIME_PARSE(\"timestamp\") AS __time,\n  *\nFROM TABLE(\n  EXTERN(\n    '{\"type\": \"http\", \"uris\": [\"https://static.imply.io/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\": \"cou [...]
+  "context": {
+    "maxNumTasks": 3
+  }
+})
+headers = {
+  'Content-Type': 'application/json'
+}
+
+response = requests.request("POST", url, headers=headers, data=payload)
+
+print(response.text)
+
+```
+
+<!--END_DOCUSAURUS_CODE_TABS-->
+
+
+### Response
+
+```json
+{
+  "taskId": "query-f795a235-4dc7-4fef-abac-3ae3f9686b79",
+  "state": "RUNNING",
+}
+```
+
+**Response fields**
+
+|Field|Description|
+|-----|-----------|
+| taskId | Controller task ID. You can use Druid's standard [task APIs](../operations/api-reference.md#overlord) to interact with this controller task.|
+| state | Initial state for the query, which is "RUNNING".|
+
+
+## Get the payload for a query task
+
+You can retrieve basic information about a query task, such as the SQL query and context parameters that were submitted. 
+
+### Request
+
+<!--DOCUSAURUS_CODE_TABS-->
+
+<!--HTTP-->
+
+```
+GET /druid/indexer/v1/task/<taskId>
+```
+
+<!--curl-->
+
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```bash
+curl --location --request GET 'https://<username>:<password>@<your-instance>:<port>/druid/indexer/v1/task/<taskId>'
+```
+
+<!--Python-->
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```python
+import requests
+
+url = "<username>:<password>@<your-instance>:<port>/druid/indexer/v1/task/<taskId>"
+
+payload={}
+headers = {}
+
+response = requests.request("GET", url, headers=headers, data=payload)
+
+print(response.text)
+
+```
+
+<!--END_DOCUSAURUS_CODE_TABS-->
+
+### Response
+
+<details><summary>Show the response</summary>
+
+```
+{
+    "task": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+    "payload": {
+        "type": "query_controller",
+        "id": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+        "spec": {
+            "query": {
+                "queryType": "scan",
+                "dataSource": {
+                    "type": "external",
+                    "inputSource": {
+                        "type": "http",
+                        "uris": [
+                            "https://static.imply.io/data/kttm/kttm-v2-2019-08-25.json.gz"
+                        ],
+                        "httpAuthenticationUsername": null,
+                        "httpAuthenticationPassword": null
+                    },
+                    "inputFormat": {
+                        "type": "json",
+                        "flattenSpec": null,
+                        "featureSpec": {},
+                        "keepNullColumns": false
+                    },
+                    "signature": [
+                        {
+                            "name": "timestamp",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "agent_category",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "agent_type",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "browser",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "browser_version",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "city",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "continent",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "country",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "version",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "event_type",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "event_subtype",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "loaded_image",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "adblock_list",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "forwarded_for",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "language",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "number",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "os",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "path",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "platform",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "referrer",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "referrer_host",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "region",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "remote_address",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "screen",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "session",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "session_length",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "timezone",
+                            "type": "STRING"
+                        },
+                        {
+                            "name": "timezone_offset",
+                            "type": "LONG"
+                        },
+                        {
+                            "name": "window",
+                            "type": "STRING"
+                        }
+                    ]
+                },
+                "intervals": {
+                    "type": "intervals",
+                    "intervals": [
+                        "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
+                    ]
+                },
+                "resultFormat": "compactedList",
+                "columns": [
+                    "adblock_list",
+                    "agent_category",
+                    "agent_type",
+                    "browser",
+                    "browser_version",
+                    "city",
+                    "continent",
+                    "country",
+                    "event_subtype",
+                    "event_type",
+                    "forwarded_for",
+                    "language",
+                    "loaded_image",
+                    "number",
+                    "os",
+                    "path",
+                    "platform",
+                    "referrer",
+                    "referrer_host",
+                    "region",
+                    "remote_address",
+                    "screen",
+                    "session",
+                    "session_length",
+                    "timestamp",
+                    "timezone",
+                    "timezone_offset",
+                    "version",
+                    "window"
+                ],
+                "legacy": false,
+                "context": {
+                    "finalize": true,
+                    "msqMaxNumTasks": 3,
+                    "msqSignature": "[{\"name\":\"adblock_list\",\"type\":\"STRING\"},{\"name\":\"agent_category\",\"type\":\"STRING\"},{\"name\":\"agent_type\",\"type\":\"STRING\"},{\"name\":\"browser\",\"type\":\"STRING\"},{\"name\":\"browser_version\",\"type\":\"STRING\"},{\"name\":\"city\",\"type\":\"STRING\"},{\"name\":\"continent\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"},{\"name\":\"event_subtype\",\"type\":\"STRING\"},{\"name\":\"event_type\",\"type\":\"STR [...]
+                    "multiStageQuery": true,
+                    "sqlInsertSegmentGranularity": "{\"type\":\"all\"}",
+                    "sqlQueryId": "a6b65442-f77e-44e4-af28-ab3b711a27ac",
+                    "sqlReplaceTimeChunks": "all"
+                },
+                "granularity": {
+                    "type": "all"
+                }
+            },
+            "columnMappings": [
+                {
+                    "queryColumn": "timestamp",
+                    "outputColumn": "timestamp"
+                },
+                {
+                    "queryColumn": "agent_category",
+                    "outputColumn": "agent_category"
+                },
+                {
+                    "queryColumn": "agent_type",
+                    "outputColumn": "agent_type"
+                },
+                {
+                    "queryColumn": "browser",
+                    "outputColumn": "browser"
+                },
+                {
+                    "queryColumn": "browser_version",
+                    "outputColumn": "browser_version"
+                },
+                {
+                    "queryColumn": "city",
+                    "outputColumn": "city"
+                },
+                {
+                    "queryColumn": "continent",
+                    "outputColumn": "continent"
+                },
+                {
+                    "queryColumn": "country",
+                    "outputColumn": "country"
+                },
+                {
+                    "queryColumn": "version",
+                    "outputColumn": "version"
+                },
+                {
+                    "queryColumn": "event_type",
+                    "outputColumn": "event_type"
+                },
+                {
+                    "queryColumn": "event_subtype",
+                    "outputColumn": "event_subtype"
+                },
+                {
+                    "queryColumn": "loaded_image",
+                    "outputColumn": "loaded_image"
+                },
+                {
+                    "queryColumn": "adblock_list",
+                    "outputColumn": "adblock_list"
+                },
+                {
+                    "queryColumn": "forwarded_for",
+                    "outputColumn": "forwarded_for"
+                },
+                {
+                    "queryColumn": "language",
+                    "outputColumn": "language"
+                },
+                {
+                    "queryColumn": "number",
+                    "outputColumn": "number"
+                },
+                {
+                    "queryColumn": "os",
+                    "outputColumn": "os"
+                },
+                {
+                    "queryColumn": "path",
+                    "outputColumn": "path"
+                },
+                {
+                    "queryColumn": "platform",
+                    "outputColumn": "platform"
+                },
+                {
+                    "queryColumn": "referrer",
+                    "outputColumn": "referrer"
+                },
+                {
+                    "queryColumn": "referrer_host",
+                    "outputColumn": "referrer_host"
+                },
+                {
+                    "queryColumn": "region",
+                    "outputColumn": "region"
+                },
+                {
+                    "queryColumn": "remote_address",
+                    "outputColumn": "remote_address"
+                },
+                {
+                    "queryColumn": "screen",
+                    "outputColumn": "screen"
+                },
+                {
+                    "queryColumn": "session",
+                    "outputColumn": "session"
+                },
+                {
+                    "queryColumn": "session_length",
+                    "outputColumn": "session_length"
+                },
+                {
+                    "queryColumn": "timezone",
+                    "outputColumn": "timezone"
+                },
+                {
+                    "queryColumn": "timezone_offset",
+                    "outputColumn": "timezone_offset"
+                },
+                {
+                    "queryColumn": "window",
+                    "outputColumn": "window"
+                }
+            ],
+            "destination": {
+                "type": "dataSource",
+                "dataSource": "kttm_simple",
+                "segmentGranularity": {
+                    "type": "all"
+                },
+                "replaceTimeChunks": [
+                    "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
+                ]
+            },
+            "assignmentStrategy": "max",
+            "tuningConfig": {
+                "type": "index_parallel",
+                "maxRowsPerSegment": 3000000,
+                "appendableIndexSpec": {
+                    "type": "onheap",
+                    "preserveExistingMetrics": false
+                },
+                "maxRowsInMemory": 100000,
+                "maxBytesInMemory": 0,
+                "skipBytesInMemoryOverheadCheck": false,
+                "maxTotalRows": null,
+                "numShards": null,
+                "splitHintSpec": null,
+                "partitionsSpec": {
+                    "type": "dynamic",
+                    "maxRowsPerSegment": 3000000,
+                    "maxTotalRows": null
+                },
+                "indexSpec": {
+                    "bitmap": {
+                        "type": "roaring",
+                        "compressRunOnSerialization": true
+                    },
+                    "dimensionCompression": "lz4",
+                    "metricCompression": "lz4",
+                    "longEncoding": "longs",
+                    "segmentLoader": null
+                },
+                "indexSpecForIntermediatePersists": {
+                    "bitmap": {
+                        "type": "roaring",
+                        "compressRunOnSerialization": true
+                    },
+                    "dimensionCompression": "lz4",
+                    "metricCompression": "lz4",
+                    "longEncoding": "longs",
+                    "segmentLoader": null
+                },
+                "maxPendingPersists": 0,
+                "forceGuaranteedRollup": false,
+                "reportParseExceptions": false,
+                "pushTimeout": 0,
+                "segmentWriteOutMediumFactory": null,
+                "maxNumConcurrentSubTasks": 2,
+                "maxRetry": 1,
+                "taskStatusCheckPeriodMs": 1000,
+                "chatHandlerTimeout": "PT10S",
+                "chatHandlerNumRetries": 5,
+                "maxNumSegmentsToMerge": 100,
+                "totalNumMergeTasks": 10,
+                "logParseExceptions": false,
+                "maxParseExceptions": 2147483647,
+                "maxSavedParseExceptions": 0,
+                "maxColumnsToMerge": -1,
+                "awaitSegmentAvailabilityTimeoutMillis": 0,
+                "maxAllowedLockCount": -1,
+                "partitionDimensions": []
+            }
+        },
+        "sqlQuery": "REPLACE INTO \"kttm_simple\" OVERWRITE ALL\nSELECT *\nFROM TABLE(\n  EXTERN(\n    '{\"type\":\"http\",\"uris\":[\"https://static.imply.io/data/kttm/kttm-v2-2019-08-25.json.gz\"]}',\n    '{\"type\":\"json\"}',\n    '[{\"name\":\"timestamp\",\"type\":\"string\"},{\"name\":\"agent_category\",\"type\":\"string\"},{\"name\":\"agent_type\",\"type\":\"string\"},{\"name\":\"browser\",\"type\":\"string\"},{\"name\":\"browser_version\",\"type\":\"string\"},{\"name\":\"city\",\ [...]
+        "sqlQueryContext": {
+            "parseExceptions": 0,
+            "maxNumTasks": 3,
+            "signature": "[{\"name\":\"adblock_list\",\"type\":\"STRING\"},{\"name\":\"agent_category\",\"type\":\"STRING\"},{\"name\":\"agent_type\",\"type\":\"STRING\"},{\"name\":\"browser\",\"type\":\"STRING\"},{\"name\":\"browser_version\",\"type\":\"STRING\"},{\"name\":\"city\",\"type\":\"STRING\"},{\"name\":\"continent\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"},{\"name\":\"event_subtype\",\"type\":\"STRING\"},{\"name\":\"event_type\",\"type\":\"STRING\"},{\"n [...]
+            "multiStageQuery": true,
+            "sqlInsertSegmentGranularity": "{\"type\":\"all\"}",
+            "sqlQueryId": "a6b65442-f77e-44e4-af28-ab3b711a27ac",
+            "sqlReplaceTimeChunks": "all"
+        },
+        "sqlTypeNames": [
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "BIGINT",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "VARCHAR",
+            "BIGINT",
+            "VARCHAR",
+            "BIGINT",
+            "VARCHAR"
+        ],
+        "context": {
+            "forceTimeChunkLock": true,
+            "useLineageBasedSegmentAllocation": true
+        },
+        "groupId": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+        "dataSource": "kttm_simple",
+        "resource": {
+            "availabilityGroup": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+            "requiredCapacity": 1
+        }
+    }
+}
+```
+
+</details>
+
+## Get the status for a query task
+
+You can retrieve status of a query to see if it is still running, completed successfully, failed, or got canceled. 
+
+### Request
+
+<!--DOCUSAURUS_CODE_TABS-->
+
+<!--HTTP-->
+
+```
+GET /druid/indexer/v1/task/<taskId>
+```
+
+<!--curl-->
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```bash
+curl --location --request GET 'https://<username>:<password>@<hostname>:<port>/druid/indexer/v1/task/<taskId>/status'
+```
+
+<!--Python-->
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```python
+import requests
+
+url = "https://<username>:<password>@<hostname>:<port>/druid/indexer/v1/task/<taskId>/status"
+
+payload={}
+headers = {}
+
+response = requests.request("GET", url, headers=headers, data=payload)
+
+print(response.text)
+```
+
+<!--END_DOCUSAURUS_CODE_TABS-->
+
+### Response
+
+```
+{
+    "task": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+    "status": {
+        "id": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+        "groupId": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+        "type": "query_controller",
+        "createdTime": "2022-07-27T20:09:23.551Z",
+        "queueInsertionTime": "1970-01-01T00:00:00.000Z",
+        "statusCode": "SUCCESS",
+        "status": "SUCCESS",
+        "runnerStatusCode": "WAITING",
+        "duration": 136636,
+        "location": {
+            "host": "ip-10-201-5-81.ec2.internal",
+            "port": -1,
+            "tlsPort": 8100
+        },
+        "dataSource": "kttm_simple",
+        "errorMsg": null
+    }
+}
+```
+
+## Get the report for a query task
+
+A report provides detailed information about a query task, including things like the stages, warnings, and errors.
+
+Keep the following in mind when using the task API to view reports:
+
+- For SELECT queries, the report includes the results. At this time, if you want to view results for SELECT queries, you need to retrieve them as a generic map from the report and extract the results.
+- The task report stores query details for controller tasks.
+- If you encounter `500 Server Error` or `404 Not Found` errors, the task may be in the process of starting up or shutting down.
+
+For an explanation of the fields in a report, see [Report response fields](#report-response-fields).
+
+### Request
+
+<!--DOCUSAURUS_CODE_TABS-->
+
+<!--HTTP-->
+
+```
+GET /druid/indexer/v1/task/<taskId>/report
+```
+
+<!--curl-->
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```bash
+curl --location --request GET 'https://<username>:<password>@<hostname>:<port>/druid/indexer/v1/task/<taskId>/report'
+```
+
+<!--Python-->
+
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```python
+import requests
+
+url = "https://<username>:<password>@<hostname>:<port>/druid/indexer/v1/task/<taskId>/reports"
+
+payload={}
+headers = {}
+
+response = requests.request("GET", url, headers=headers, data=payload)
+
+print(response.text)
+```
+
+
+<!--END_DOCUSAURUS_CODE_TABS-->
+
+### Response
+
+The response shows an example report for a query.
+
+<details><summary>Show the response</summary>
+
+```json
+{
+    "multiStageQuery": {
+        "taskId": "query-a6b65442-f77e-44e4-af28-ab3b711a27ac",
+        "payload": {
+            "status": {
+                "status": "SUCCESS",
+                "startTime": "2022-07-27T20:09:39.915Z",
+                "durationMs": 116516,
+                "warningReports": []
+            },
+            "stages": [
+                {
+                    "stageNumber": 0,
+                    "definition": {
+                        "id": "f224410f-1cad-4ee7-b10d-f10ddf8bb517_0",
+                        "input": [
+                            {
+                                "type": "external",
+                                "inputSource": {
+                                    "type": "http",
+                                    "uris": [
+                                        "https://static.imply.io/data/kttm/kttm-v2-2019-08-25.json.gz"
+                                    ],
+                                    "httpAuthenticationUsername": null,
+                                    "httpAuthenticationPassword": null
+                                },
+                                "inputFormat": {
+                                    "type": "json",
+                                    "flattenSpec": null,
+                                    "featureSpec": {},
+                                    "keepNullColumns": false
+                                },
+                                "signature": [
+                                    {
+                                        "name": "timestamp",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "agent_category",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "agent_type",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "browser",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "browser_version",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "city",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "continent",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "country",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "version",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "event_type",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "event_subtype",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "loaded_image",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "adblock_list",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "forwarded_for",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "language",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "number",
+                                        "type": "LONG"
+                                    },
+                                    {
+                                        "name": "os",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "path",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "platform",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "referrer",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "referrer_host",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "region",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "remote_address",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "screen",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "session",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "session_length",
+                                        "type": "LONG"
+                                    },
+                                    {
+                                        "name": "timezone",
+                                        "type": "STRING"
+                                    },
+                                    {
+                                        "name": "timezone_offset",
+                                        "type": "LONG"
+                                    },
+                                    {
+                                        "name": "window",
+                                        "type": "STRING"
+                                    }
+                                ]
+                            }
+                        ],
+                        "processor": {
+                            "type": "scan",
+                            "query": {
+                                "queryType": "scan",
+                                "dataSource": {
+                                    "type": "inputNumber",
+                                    "inputNumber": 0
+                                },
+                                "intervals": {
+                                    "type": "intervals",
+                                    "intervals": [
+                                        "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
+                                    ]
+                                },
+                                "resultFormat": "compactedList",
+                                "columns": [
+                                    "adblock_list",
+                                    "agent_category",
+                                    "agent_type",
+                                    "browser",
+                                    "browser_version",
+                                    "city",
+                                    "continent",
+                                    "country",
+                                    "event_subtype",
+                                    "event_type",
+                                    "forwarded_for",
+                                    "language",
+                                    "loaded_image",
+                                    "number",
+                                    "os",
+                                    "path",
+                                    "platform",
+                                    "referrer",
+                                    "referrer_host",
+                                    "region",
+                                    "remote_address",
+                                    "screen",
+                                    "session",
+                                    "session_length",
+                                    "timestamp",
+                                    "timezone",
+                                    "timezone_offset",
+                                    "version",
+                                    "window"
+                                ],
+                                "legacy": false,
+                                "context": {
+                                    "finalize": true,
+                                    "msqMaxNumTasks": 3,
+                                    "msqSignature": "[{\"name\":\"adblock_list\",\"type\":\"STRING\"},{\"name\":\"agent_category\",\"type\":\"STRING\"},{\"name\":\"agent_type\",\"type\":\"STRING\"},{\"name\":\"browser\",\"type\":\"STRING\"},{\"name\":\"browser_version\",\"type\":\"STRING\"},{\"name\":\"city\",\"type\":\"STRING\"},{\"name\":\"continent\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"},{\"name\":\"event_subtype\",\"type\":\"STRING\"},{\"name\":\"event_type\ [...]
+                                    "multiStageQuery": true,
+                                    "sqlInsertSegmentGranularity": "{\"type\":\"all\"}",
+                                    "sqlQueryId": "a6b65442-f77e-44e4-af28-ab3b711a27ac",
+                                    "sqlReplaceTimeChunks": "all"
+                                },
+                                "granularity": {
+                                    "type": "all"
+                                }
+                            }
+                        },
+                        "signature": [
+                            {
+                                "name": "__boost",
+                                "type": "LONG"
+                            },
+                            {
+                                "name": "adblock_list",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "agent_category",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "agent_type",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "browser",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "browser_version",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "city",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "continent",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "country",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "event_subtype",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "event_type",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "forwarded_for",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "language",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "loaded_image",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "number",
+                                "type": "LONG"
+                            },
+                            {
+                                "name": "os",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "path",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "platform",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "referrer",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "referrer_host",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "region",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "remote_address",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "screen",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "session",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "session_length",
+                                "type": "LONG"
+                            },
+                            {
+                                "name": "timestamp",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "timezone",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "timezone_offset",
+                                "type": "LONG"
+                            },
+                            {
+                                "name": "version",
+                                "type": "STRING"
+                            },
+                            {
+                                "name": "window",
+                                "type": "STRING"
+                            }
+                        ],
+                        "shuffleSpec": {
+                            "type": "targetSize",
+                            "clusterBy": {
+                                "columns": [
+                                    {
+                                        "columnName": "__boost"
+                                    }
+                                ]
+                            },
+                            "targetSize": 3000000,
+                            "aggregate": false
+                        },
+                        "maxWorkerCount": 2,
+                        "shuffleCheckHasMultipleValues": true
+                    },
+                    "phase": "FINISHED",
+                    "workerCount": 1,
+                    "partitionCount": 1,
+                    "startTime": "2022-07-27T20:09:43.168Z",
+                    "duration": 62837,
+                    "sort": true
+                },
+                {
+                    "stageNumber": 1,
+                    "definition": {
+                        "id": "f224410f-1cad-4ee7-b10d-f10ddf8bb517_1",
+                        "input": [
+                            {
+                                "type": "stage",
+                                "stage": 0
+                            }
+                        ],
+                        "processor": {
+                            "type": "segmentGenerator",
+                            "dataSchema": {
+                                "dataSource": "kttm_simple",
+                                "timestampSpec": {
+                                    "column": "__time",
+                                    "format": "millis",
+                                    "missingValue": null
+                                },
+                                "dimensionsSpec": {
+                                    "dimensions": [
+                                        {
+                                            "type": "string",
+                                            "name": "timestamp",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "agent_category",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "agent_type",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "browser",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "browser_version",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "city",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "continent",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "country",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "version",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "event_type",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "event_subtype",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "loaded_image",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "adblock_list",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "forwarded_for",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "language",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "long",
+                                            "name": "number",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": false
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "os",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "path",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "platform",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "referrer",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "referrer_host",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "region",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "remote_address",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "screen",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "session",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "long",
+                                            "name": "session_length",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": false
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "timezone",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        },
+                                        {
+                                            "type": "long",
+                                            "name": "timezone_offset",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": false
+                                        },
+                                        {
+                                            "type": "string",
+                                            "name": "window",
+                                            "multiValueHandling": "SORTED_ARRAY",
+                                            "createBitmapIndex": true
+                                        }
+                                    ],
+                                    "dimensionExclusions": [
+                                        "__time"
+                                    ],
+                                    "includeAllDimensions": false
+                                },
+                                "metricsSpec": [],
+                                "granularitySpec": {
+                                    "type": "arbitrary",
+                                    "queryGranularity": {
+                                        "type": "none"
+                                    },
+                                    "rollup": false,
+                                    "intervals": [
+                                        "-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
+                                    ]
+                                },
+                                "transformSpec": {
+                                    "filter": null,
+                                    "transforms": []
+                                }
+                            },
+                            "columnMappings": [
+                                {
+                                    "queryColumn": "timestamp",
+                                    "outputColumn": "timestamp"
+                                },
+                                {
+                                    "queryColumn": "agent_category",
+                                    "outputColumn": "agent_category"
+                                },
+                                {
+                                    "queryColumn": "agent_type",
+                                    "outputColumn": "agent_type"
+                                },
+                                {
+                                    "queryColumn": "browser",
+                                    "outputColumn": "browser"
+                                },
+                                {
+                                    "queryColumn": "browser_version",
+                                    "outputColumn": "browser_version"
+                                },
+                                {
+                                    "queryColumn": "city",
+                                    "outputColumn": "city"
+                                },
+                                {
+                                    "queryColumn": "continent",
+                                    "outputColumn": "continent"
+                                },
+                                {
+                                    "queryColumn": "country",
+                                    "outputColumn": "country"
+                                },
+                                {
+                                    "queryColumn": "version",
+                                    "outputColumn": "version"
+                                },
+                                {
+                                    "queryColumn": "event_type",
+                                    "outputColumn": "event_type"
+                                },
+                                {
+                                    "queryColumn": "event_subtype",
+                                    "outputColumn": "event_subtype"
+                                },
+                                {
+                                    "queryColumn": "loaded_image",
+                                    "outputColumn": "loaded_image"
+                                },
+                                {
+                                    "queryColumn": "adblock_list",
+                                    "outputColumn": "adblock_list"
+                                },
+                                {
+                                    "queryColumn": "forwarded_for",
+                                    "outputColumn": "forwarded_for"
+                                },
+                                {
+                                    "queryColumn": "language",
+                                    "outputColumn": "language"
+                                },
+                                {
+                                    "queryColumn": "number",
+                                    "outputColumn": "number"
+                                },
+                                {
+                                    "queryColumn": "os",
+                                    "outputColumn": "os"
+                                },
+                                {
+                                    "queryColumn": "path",
+                                    "outputColumn": "path"
+                                },
+                                {
+                                    "queryColumn": "platform",
+                                    "outputColumn": "platform"
+                                },
+                                {
+                                    "queryColumn": "referrer",
+                                    "outputColumn": "referrer"
+                                },
+                                {
+                                    "queryColumn": "referrer_host",
+                                    "outputColumn": "referrer_host"
+                                },
+                                {
+                                    "queryColumn": "region",
+                                    "outputColumn": "region"
+                                },
+                                {
+                                    "queryColumn": "remote_address",
+                                    "outputColumn": "remote_address"
+                                },
+                                {
+                                    "queryColumn": "screen",
+                                    "outputColumn": "screen"
+                                },
+                                {
+                                    "queryColumn": "session",
+                                    "outputColumn": "session"
+                                },
+                                {
+                                    "queryColumn": "session_length",
+                                    "outputColumn": "session_length"
+                                },
+                                {
+                                    "queryColumn": "timezone",
+                                    "outputColumn": "timezone"
+                                },
+                                {
+                                    "queryColumn": "timezone_offset",
+                                    "outputColumn": "timezone_offset"
+                                },
+                                {
+                                    "queryColumn": "window",
+                                    "outputColumn": "window"
+                                }
+                            ],
+                            "tuningConfig": {
+                                "type": "index_parallel",
+                                "maxRowsPerSegment": 3000000,
+                                "appendableIndexSpec": {
+                                    "type": "onheap",
+                                    "preserveExistingMetrics": false
+                                },
+                                "maxRowsInMemory": 100000,
+                                "maxBytesInMemory": 0,
+                                "skipBytesInMemoryOverheadCheck": false,
+                                "maxTotalRows": null,
+                                "numShards": null,
+                                "splitHintSpec": null,
+                                "partitionsSpec": {
+                                    "type": "dynamic",
+                                    "maxRowsPerSegment": 3000000,
+                                    "maxTotalRows": null
+                                },
+                                "indexSpec": {
+                                    "bitmap": {
+                                        "type": "roaring",
+                                        "compressRunOnSerialization": true
+                                    },
+                                    "dimensionCompression": "lz4",
+                                    "metricCompression": "lz4",
+                                    "longEncoding": "longs",
+                                    "segmentLoader": null
+                                },
+                                "indexSpecForIntermediatePersists": {
+                                    "bitmap": {
+                                        "type": "roaring",
+                                        "compressRunOnSerialization": true
+                                    },
+                                    "dimensionCompression": "lz4",
+                                    "metricCompression": "lz4",
+                                    "longEncoding": "longs",
+                                    "segmentLoader": null
+                                },
+                                "maxPendingPersists": 0,
+                                "forceGuaranteedRollup": false,
+                                "reportParseExceptions": false,
+                                "pushTimeout": 0,
+                                "segmentWriteOutMediumFactory": null,
+                                "maxNumConcurrentSubTasks": 2,
+                                "maxRetry": 1,
+                                "taskStatusCheckPeriodMs": 1000,
+                                "chatHandlerTimeout": "PT10S",
+                                "chatHandlerNumRetries": 5,
+                                "maxNumSegmentsToMerge": 100,
+                                "totalNumMergeTasks": 10,
+                                "logParseExceptions": false,
+                                "maxParseExceptions": 2147483647,
+                                "maxSavedParseExceptions": 0,
+                                "maxColumnsToMerge": -1,
+                                "awaitSegmentAvailabilityTimeoutMillis": 0,
+                                "maxAllowedLockCount": -1,
+                                "partitionDimensions": []
+                            }
+                        },
+                        "signature": [],
+                        "maxWorkerCount": 2
+                    },
+                    "phase": "FINISHED",
+                    "workerCount": 1,
+                    "partitionCount": 1,
+                    "startTime": "2022-07-27T20:10:45.840Z",
+                    "duration": 50590
+                }
+            ],
+            "counters": {
+                "0": {
+                    "0": {
+                        "input0": {
+                            "type": "channel",
+                            "rows": [
+                                465346
+                            ],
+                            "files": [
+                                1
+                            ],
+                            "totalFiles": [
+                                1
+                            ]
+                        },
+                        "output": {
+                            "type": "channel",
+                            "rows": [
+                                465346
+                            ],
+                            "bytes": [
+                                267146161
+                            ],
+                            "frames": [
+                                42
+                            ]
+                        },
+                        "sort": {
+                            "type": "channel",
+                            "rows": [
+                                465346
+                            ],
+                            "bytes": [
+                                265300383
+                            ],
+                            "frames": [
+                                501
+                            ]
+                        },
+                        "sortProgress": {
+                            "type": "sortProgress",
+                            "totalMergingLevels": 3,
+                            "levelToTotalBatches": {
+                                "0": 9,
+                                "1": 2,
+                                "2": 1
+                            },
+                            "levelToMergedBatches": {
+                                "0": 9,
+                                "1": 2,
+                                "2": 1
+                            },
+                            "totalMergersForUltimateLevel": 1,
+                            "progressDigest": 1.0
+                        }
+                    }
+                },
+                "1": {
+                    "0": {
+                        "input0": {
+                            "type": "channel",
+                            "rows": [
+                                465346
+                            ],
+                            "bytes": [
+                                265300383
+                            ],
+                            "frames": [
+                                501
+                            ]
+                        }
+                    }
+                }
+            }
+        }
+    }
+}
+```
+
+### Report response fields
+
+The following table describes the response fields when you retrieve a report for a MSQ task engine using the `/druid/indexer/v1/task/<taskId>/report` endpoint:
+
+|Field|Description|
+|-----|-----------|
+|multiStageQuery.taskId|Controller task ID.|
+|multiStageQuery.payload.status|Query status container.|
+|multiStageQuery.payload.status.status|RUNNING, SUCCESS, or FAILED.|
+|multiStageQuery.payload.status.startTime|Start time of the query in ISO format. Only present if the query has started running.|
+|multiStageQuery.payload.status.durationMs|Milliseconds elapsed after the query has started running. -1 denotes that the query hasn't started running yet.|
+|multiStageQuery.payload.status.errorReport|Error object. Only present if there was an error.|
+|multiStageQuery.payload.status.errorReport.taskId|The task that reported the error, if known. May be a controller task or a worker task.|
+|multiStageQuery.payload.status.errorReport.host|The hostname and port of the task that reported the error, if known.|
+|multiStageQuery.payload.status.errorReport.stageNumber|The stage number that reported the error, if it happened during execution of a specific stage.|
+|multiStageQuery.payload.status.errorReport.error|Error object. Contains `errorCode` at a minimum, and may contain other fields as described in the [error code table](./msq-concepts.md#error-codes). Always present if there is an error.|
+|multiStageQuery.payload.status.errorReport.error.errorCode|One of the error codes from the [error code table](./msq-concepts.md#error-codes). Always present if there is an error.|
+|multiStageQuery.payload.status.errorReport.error.errorMessage|User-friendly error message. Not always present, even if there is an error.|
+|multiStageQuery.payload.status.errorReport.exceptionStackTrace|Java stack trace in string form, if the error was due to a server-side exception.|
+|multiStageQuery.payload.stages|Array of query stages.|
+|multiStageQuery.payload.stages[].stageNumber|Each stage has a number that differentiates it from other stages.|
+|multiStageQuery.payload.stages[].phase|Either NEW, READING_INPUT, POST_READING, RESULTS_COMPLETE, or FAILED. Only present if the stage has started.|
+|multiStageQuery.payload.stages[].workerCount|Number of parallel tasks that this stage is running on. Only present if the stage has started.|
+|multiStageQuery.payload.stages[].partitionCount|Number of output partitions generated by this stage. Only present if the stage has started and has computed its number of output partitions.|
+|multiStageQuery.payload.stages[].startTime|Start time of this stage. Only present if the stage has started.|
+|multiStageQuery.payload.stages[].duration|The number of milliseconds that the stage has been running. Only present if the stage has started.|
+|multiStageQuery.payload.stages[].sort|A boolean that is set to `true` if the stage does a sort as part of its execution.|
+|multiStageQuery.payload.stages[].definition|The object defining what the stage does.|
+|multiStageQuery.payload.stages[].definition.id|The unique identifier of the stage.|
+|multiStageQuery.payload.stages[].definition.input|Array of inputs that the stage has.|
+|multiStageQuery.payload.stages[].definition.broadcast|Array of input indexes that get broadcasted. Only present if there are inputs that get broadcasted.|
+|multiStageQuery.payload.stages[].definition.processor|An object defining the processor logic.|
+|multiStageQuery.payload.stages[].definition.signature|The output signature of the stage.|
+
+## Cancel a query task
+
+### Request
+
+<!--DOCUSAURUS_CODE_TABS-->
+
+<!--HTTP-->
+
+```
+POST /druid/indexer/v1/task/<taskId>/shutdown
+```
+
+<!--curl-->
+
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```bash
+curl --location --request POST 'https://<username>:<password>@<your-instance>:<port>/druid/indexer/v1/task/<taskId>/shutdown'
+```
+
+<!--Python-->
+
+Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
+
+```
+import requests
+
+url = "https://<username>:<password>@<your-instance>:<port>/druid/indexer/v1/task/<taskId>/shutdown"
+
+payload={}
+headers = {}
+
+response = requests.request("POST", url, headers=headers, data=payload)
+
+print(response.text)
+```
+
+<!--END_DOCUSAURUS_CODE_TABS-->
+
+### Response
+
+```
+{
+    "task": "query-655efe33-781a-4c50-ae84-c2911b42d63c"
+}
+```
diff --git a/docs/multi-stage-query/msq-concepts.md b/docs/multi-stage-query/msq-concepts.md
new file mode 100644
index 0000000000..4f1cf7ce1d
--- /dev/null
+++ b/docs/multi-stage-query/msq-concepts.md
@@ -0,0 +1,168 @@
+---
+id: concepts
+title: SQL-based ingestion concepts
+sidebar_label: Key concepts
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+This topic covers the main concepts and terminology of the multi-stage query architecture.
+
+## Vocabulary
+
+You might see the following terms in the documentation or while you're using the multi-stage query architecture and task engine, such as when you view the report for a query:
+
+- **Controller**: An indexing service task of type `query_controller` that manages
+  the execution of a query. There is one controller task per query.
+
+- **Worker**: Indexing service tasks of type `query_worker` that execute a
+  query. There can be multiple worker tasks per query. Internally,
+  the tasks process items in parallel using their processing pools (up to `druid.processing.numThreads` of execution parallelism
+  within a worker task).
+
+- **Stage**: A stage of query execution that is parallelized across
+  worker tasks. Workers exchange data with each other between stages.
+
+- **Partition**: A slice of data output by worker tasks. In INSERT or REPLACE
+  queries, the partitions of the final stage become Druid segments.
+
+- **Shuffle**: Workers exchange data between themselves on a per-partition basis in a process called
+  shuffling. During a shuffle, each output partition is sorted by a clustering key.
+
+## How the MSQ task engine works
+
+Query tasks, specifically queries for INSERT, REPLACE, and SELECT, execute using indexing service tasks. Every query occupies at least two task slots while running. 
+
+When you submit a query task to the MSQ task engine, the following happens:
+
+1.  The Broker plans your SQL query into a native query, as usual.
+
+2.  The Broker wraps the native query into a task of type `query_controller`
+    and submits it to the indexing service.
+
+3. The Broker returns the task ID to you and exits.
+
+4.  The controller task launches some number of worker tasks determined by
+    the `maxNumTasks` and `taskAssignment` [context parameters](./msq-reference.md#context-parameters). You can set these settings individually for each query.
+
+5.  The worker tasks execute the query.
+
+6.  If the query is a SELECT query, the worker tasks send the results
+    back to the controller task, which writes them into its task report.
+    If the query is an INSERT or REPLACE query, the worker tasks generate and
+    publish new Druid segments to the provided datasource.
+
+
+## Parallelism
+
+Parallelism affects performance.
+
+The [`maxNumTasks`](./msq-reference.md#context-parameters) query parameter determines the maximum number of tasks (workers and one controller) your query will use. Generally, queries perform better with more workers. The lowest possible value of `maxNumTasks` is two (one worker and one controller), and the highest possible value is equal to the number of free task slots in your cluster.
+
+The `druid.worker.capacity` server property on each Middle Manager determines the maximum number
+of worker tasks that can run on each server at once. Worker tasks run single-threaded, which
+also determines the maximum number of processors on the server that can contribute towards
+multi-stage queries. Since data servers are shared between Historicals and
+Middle Managers, the default setting for `druid.worker.capacity` is lower than the number of
+processors on the server. Advanced users may consider enhancing parallelism by increasing this
+value to one less than the number of processors on the server. In most cases, this increase must
+be accompanied by an adjustment of the memory allotment of the Historical process,
+Middle-Manager-launched tasks, or both, to avoid memory overcommitment and server instability. If
+you are not comfortable tuning these memory usage parameters to avoid overcommitment, it is best
+to stick with the default `druid.worker.capacity`.
+
+## Memory usage
+
+Increasing the amount of available memory can improve performance as follows:
+
+- Segment generation becomes more efficient when data doesn't spill to disk as often.
+- Sorting stage output data becomes more efficient since available memory affects the
+  number of required sorting passes.
+
+Worker tasks use both JVM heap memory and off-heap ("direct") memory.
+
+On Peons launched by Middle Managers, the bulk of the JVM heap (75%) is split up into two bundles of equal size: one processor bundle and one worker bundle. Each one comprises 37.5% of the available JVM heap.
+
+The processor memory bundle is used for query processing and segment generation. Each processor bundle must
+also provides space to buffer I/O between stages. Specifically, each downstream stage requires 1 MB of buffer space for
+each upstream worker. For example, if you have 100 workers running in stage 0, and stage 1 reads from stage 0,
+then each worker in stage 1 requires 1M * 100 = 100 MB of memory for frame buffers.
+
+The worker memory bundle is used for sorting stage output data prior to shuffle. Workers can sort
+more data than fits in memory; in this case, they will switch to using disk.
+
+Worker tasks also use off-heap ("direct") memory. Set the amount of direct
+memory available (`-XX:MaxDirectMemorySize`) to at least
+`(druid.processing.numThreads + 1) * druid.processing.buffer.sizeBytes`. Increasing the
+amount of direct memory available beyond the minimum does not speed up processing.
+
+It may be necessary to override one or more memory-related parameters if you run into one of the [known issues around memory usage](./msq-known-issues.md#memory-usage).
+
+## Limits
+
+Knowing the limits for the MSQ task engine can help you troubleshoot any [errors](#error-codes) that you encounter. Many of the errors occur as a result of reaching a limit.
+
+The following table lists query limits:
+
+|Limit|Value|Error if exceeded|
+|-----|-----|-----------------|
+| Size of an individual row written to a frame. Row size when written to a frame may differ from the original row size. | 1 MB | `RowTooLarge` |
+| Number of segment-granular time chunks encountered during ingestion. | 5,000 | `TooManyBuckets` |
+| Number of input files/segments per worker. | 10,000 | `TooManyInputFiles` |
+| Number of output partitions for any one stage. Number of segments generated during ingestion. |25,000 | `TooManyPartitions` |
+| Number of output columns for any one stage. | 2,000 | `TooManyColumns` |
+| Number of workers for any one stage. | Hard limit is 1,000. Memory-dependent soft limit may be lower. | `TooManyWorkers` |
+| Maximum memory occupied by broadcasted tables. | 30% of each [processor memory bundle](#memory-usage). | `BroadcastTablesTooLarge` |
+
+## Error codes
+
+The following table describes error codes you may encounter in the `multiStageQuery.payload.status.errorReport.error.errorCode` field:
+
+|Code|Meaning|Additional fields|
+|----|-----------|----|
+|  BroadcastTablesTooLarge  | The size of the broadcast tables, used in right hand side of the joins, exceeded the memory reserved for them in a worker task.  | `maxBroadcastTablesSize`: Memory reserved for the broadcast tables, measured in bytes. |
+|  Canceled  |  The query was canceled. Common reasons for cancellation:<br /><br /><ul><li>User-initiated shutdown of the controller task via the `/druid/indexer/v1/task/{taskId}/shutdown` API.</li><li>Restart or failure of the server process that was running the controller task.</li></ul>|    |
+|  CannotParseExternalData |  A worker task could not parse data from an external datasource.  |    |
+|  ColumnNameRestricted|  The query uses a restricted column name.  |    |
+|  ColumnTypeNotSupported|  Support for writing or reading from a particular column type is not supported. |    |
+|  ColumnTypeNotSupported | The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.  | `columnName`<br /> <br />`columnType`   |
+|  InsertCannotAllocateSegment |  The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:<br /> <br /><ul><li>Attempting to mix different granularities in the same intervals of the same datasource.</li><li>Prior ingestions that used non-extendable shard specs.</li></ul>| `dataSource`<br /> <br />`interval`: The interval for the attempted new segment allocation.  |
+|  InsertCannotBeEmpty |  An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`.  |  `dataSource`  |
+|  InsertCannotOrderByDescending  |  An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order.  |   `columnName` |
+|  InsertCannotReplaceExistingSegment |  A REPLACE query cannot proceed because an existing segment partially overlaps those bounds, and the portion within the bounds is not fully overshadowed by query results. <br /> <br />There are two ways to address this without modifying your query:<ul><li>Shrink the OVERLAP filter to match the query results.</li><li>Expand the OVERLAP filter to fully contain the existing segment.</li></ul>| `segmentId`: The existing segment <br /> 
+|  InsertLockPreempted  | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task.  | |
+|  InsertTimeNull  | An INSERT or REPLACE query encountered a null timestamp in the `__time` field.<br /><br />This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. (TIME_PARSE returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern.<br /><br />If your timestamps may genuinely be null, consider using COALESCE to provide a default value. One option is [...]
+| InsertTimeOutOfBounds  |  A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause.<br /> <br />To avoid this error, verify that the   you specified is valid.  |  `interval`: time chunk interval corresponding to the out-of-bounds timestamp  |
+|  InvalidNullByte  | A string column included a null byte. Null bytes in strings are not permitted. |  `column`: The column that included the null byte |
+| QueryNotSupported   | QueryKit could not translate the provided native query to a multi-stage query.<br /> <br />This can happen if the query uses features that aren't supported, like GROUPING SETS. |    |
+|  RowTooLarge  |  The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for the specific limit on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing.  |   `maxFrameSize`: The limit on the frame size. |
+|  TaskStartTimeout  | Unable to launch all the worker tasks in time. <br /> <br />There might be insufficient available slots to start all the worker tasks simultaneously.<br /> <br /> Try splitting up the query into smaller chunks with lesser `maxNumTasks` number. Another option is to increase capacity.  | |
+|  TooManyBuckets  |  Exceeded the number of partition buckets for a stage. Partition buckets are only used for `segmentGranularity` during INSERT queries. The most common reason for this error is that your `segmentGranularity` is too narrow relative to the data. See the [Limits](./msq-concepts.md#limits) table for the specific limit.  |  `maxBuckets`: The limit on buckets.  |
+| TooManyInputFiles | Exceeded the number of input files/segments per worker. See the [Limits](./msq-concepts.md#limits) table for the specific limit. | `umInputFiles`: The total number of input files/segments for the stage.<br /><br />`maxInputFiles`: The maximum number of input files/segments per worker per stage.<br /><br />`minNumWorker`: The minimum number of workers required for a successful run. |
+|  TooManyPartitions   |  Exceeded the number of partitions for a stage. The most common reason for this is that the final stage of an INSERT or REPLACE query generated too many segments. See the [Limits](./msq-concepts.md#limits) table for the specific limit.  | `maxPartitions`: The limit on partitions which was exceeded    |
+|  TooManyColumns |  Exceeded the number of columns for a stage. See the [Limits](#limits) table for the specific limit.  | `maxColumns`: The limit on columns which was exceeded.  |
+|  TooManyWarnings |  Exceeded the allowed number of warnings of a particular type. | `rootErrorCode`: The error code corresponding to the exception that exceeded the required limit. <br /><br />`maxWarnings`: Maximum number of warnings that are allowed for the corresponding `rootErrorCode`.   |
+|  TooManyWorkers |  Exceeded the supported number of workers running simultaneously. See the [Limits](#limits) table for the specific limit.  | `workers`: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously. <br /><br />`maxWorkers`: The hard or soft limit on workers that was exceeded.  |
+|  NotEnoughMemory  |  Insufficient memory to launch a stage.  |  `serverMemory`: The amount of memory available to a single process.<br /><br />`serverWorkers`: The number of workers running in a single process.<br /><br />`serverThreads`: The number of threads in a single process.  |
+|  WorkerFailed  |  A worker task failed unexpectedly.  |  `workerTaskId`: The ID of the worker task.  |
+|  WorkerRpcFailed  |  A remote procedure call to a worker task failed and could not recover.  |  `workerTaskId`: the id of the worker task  |
+|  UnknownError   |  All other errors.  |    |
\ No newline at end of file
diff --git a/docs/multi-stage-query/msq-example-queries.md b/docs/multi-stage-query/msq-example-queries.md
new file mode 100644
index 0000000000..f1b09e332f
--- /dev/null
+++ b/docs/multi-stage-query/msq-example-queries.md
@@ -0,0 +1,503 @@
+---
+id: examples
+title: SQL-based ingestion query examples
+sidebar_label: Examples
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+These example queries show you some of the things you can do when modifying queries for your use case. Copy the example queries into the **Query** view of the Druid console and run them to see what they do.
+
+## INSERT with no rollup
+
+This example inserts data into a table named `w000` without performing any data rollup:
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+INSERT INTO w000
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  isRobot,
+  channel,
+  flags,
+  isUnpatrolled,
+  page,
+  diffUrl,
+  added,
+  comment,
+  commentLength,
+  isNew,
+  isMinor,
+  delta,
+  isAnonymous,
+  user,
+  deltaBucket,
+  deleted,
+  namespace,
+  cityName,
+  countryName,
+  regionIsoCode,
+  metroCode,
+  countryIsoCode,
+  regionName
+FROM TABLE(
+    EXTERN(
+      '{"type":"http","uris":["https://static.imply.io/data/wikipedia.json.gz"]}',
+      '{"type":"json"}',
+      '[{"name":"isRobot","type":"string"},{"name":"channel","type":"string"},{"name":"timestamp","type":"string"},{"name":"flags","type":"string"},{"name":"isUnpatrolled","type":"string"},{"name":"page","type":"string"},{"name":"diffUrl","type":"string"},{"name":"added","type":"long"},{"name":"comment","type":"string"},{"name":"commentLength","type":"long"},{"name":"isNew","type":"string"},{"name":"isMinor","type":"string"},{"name":"delta","type":"long"},{"name":"isAnonymous","type":"st [...]
+    )
+  )
+PARTITIONED BY HOUR
+CLUSTERED BY channel
+```
+
+</details>
+
+## INSERT with rollup
+
+This example inserts data into a table named `kttm_data` and performs data rollup. This example implements the recommendations described in [multi-value dimensions](./index.md#multi-value-dimensions).
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+INSERT INTO "kttm_rollup"
+
+WITH kttm_data AS (
+SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/kttm/kttm-v2-2019-08-25.json.gz"]}',
+    '{"type":"json"}',
+    '[{"name":"timestamp","type":"string"},{"name":"agent_category","type":"string"},{"name":"agent_type","type":"string"},{"name":"browser","type":"string"},{"name":"browser_version","type":"string"},{"name":"city","type":"string"},{"name":"continent","type":"string"},{"name":"country","type":"string"},{"name":"version","type":"string"},{"name":"event_type","type":"string"},{"name":"event_subtype","type":"string"},{"name":"loaded_image","type":"string"},{"name":"adblock_list","type":"st [...]
+  )
+))
+
+SELECT
+  FLOOR(TIME_PARSE("timestamp") TO MINUTE) AS __time,
+  session,
+  agent_category,
+  agent_type,
+  browser,
+  browser_version,
+  MV_TO_ARRAY("language") AS "language", -- Multi-value string dimension
+  os,
+  city,
+  country,
+  forwarded_for AS ip_address,
+
+  COUNT(*) AS "cnt",
+  SUM(session_length) AS session_length,
+  APPROX_COUNT_DISTINCT_DS_HLL(event_type) AS unique_event_types
+FROM kttm_data
+WHERE os = 'iOS'
+GROUP BY 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11
+PARTITIONED BY HOUR
+CLUSTERED BY browser, session
+```
+
+</details>
+
+## INSERT for reindexing an existing datasource
+
+This example aggregates data from a table named `w000` and inserts the result into `w002`.
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+INSERT INTO w002
+SELECT
+  FLOOR(__time TO MINUTE) AS __time,
+  channel,
+  countryIsoCode,
+  countryName,
+  regionIsoCode,
+  regionName,
+  page,
+  COUNT(*) AS cnt,
+  SUM(added) AS sum_added,
+  SUM(deleted) AS sum_deleted
+FROM w000
+GROUP BY 1, 2, 3, 4, 5, 6, 7
+PARTITIONED BY HOUR
+CLUSTERED BY page
+```
+
+</details>
+
+
+## INSERT with JOIN
+
+This example inserts data into a table named `w003` and joins data from two sources:
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+INSERT INTO w003
+WITH
+wikidata AS (SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/wikipedia.json.gz"]}',
+    '{"type":"json"}',
+    '[{"name":"isRobot","type":"string"},{"name":"channel","type":"string"},{"name":"timestamp","type":"string"},{"name":"flags","type":"string"},{"name":"isUnpatrolled","type":"string"},{"name":"page","type":"string"},{"name":"diffUrl","type":"string"},{"name":"added","type":"long"},{"name":"comment","type":"string"},{"name":"commentLength","type":"long"},{"name":"isNew","type":"string"},{"name":"isMinor","type":"string"},{"name":"delta","type":"long"},{"name":"isAnonymous","type":"stri [...]
+  )
+)),
+countries AS (SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/lookup/country.tsv"]}',
+    '{"type":"tsv","findColumnsFromHeader":true}',
+    '[{"name":"Country","type":"string"},{"name":"Capital","type":"string"},{"name":"ISO3","type":"string"},{"name":"ISO2","type":"string"}]'
+  )
+))
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  isRobot,
+  channel,
+  flags,
+  isUnpatrolled,
+  page,
+  diffUrl,
+  added,
+  comment,
+  commentLength,
+  isNew,
+  isMinor,
+  delta,
+  isAnonymous,
+  user,
+  deltaBucket,
+  deleted,
+  namespace,
+  cityName,
+  countryName,
+  regionIsoCode,
+  metroCode,
+  countryIsoCode,
+  countries.Capital AS countryCapital,
+  regionName
+FROM wikidata
+LEFT JOIN countries ON wikidata.countryIsoCode = countries.ISO2
+PARTITIONED BY HOUR
+```
+
+</details>
+
+## REPLACE an entire datasource
+
+This example replaces the entire datasource used in the table `w007` with the new query data while dropping the old data:
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+REPLACE INTO w007
+OVERWRITE ALL
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  isRobot,
+  channel,
+  flags,
+  isUnpatrolled,
+  page,
+  diffUrl,
+  added,
+  comment,
+  commentLength,
+  isNew,
+  isMinor,
+  delta,
+  isAnonymous,
+  user,
+  deltaBucket,
+  deleted,
+  namespace,
+  cityName,
+  countryName,
+  regionIsoCode,
+  metroCode,
+  countryIsoCode,
+  regionName
+FROM TABLE(
+    EXTERN(
+      '{"type":"http","uris":["https://static.imply.io/data/wikipedia.json.gz"]}',
+      '{"type":"json"}',
+      '[{"name":"isRobot","type":"string"},{"name":"channel","type":"string"},{"name":"timestamp","type":"string"},{"name":"flags","type":"string"},{"name":"isUnpatrolled","type":"string"},{"name":"page","type":"string"},{"name":"diffUrl","type":"string"},{"name":"added","type":"long"},{"name":"comment","type":"string"},{"name":"commentLength","type":"long"},{"name":"isNew","type":"string"},{"name":"isMinor","type":"string"},{"name":"delta","type":"long"},{"name":"isAnonymous","type":"st [...]
+    )
+  )
+PARTITIONED BY HOUR
+CLUSTERED BY channel
+```
+
+</details>
+
+## REPLACE for replacing a specific time segment
+
+This example replaces certain segments in a datasource with the new query data while dropping old segments:
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+REPLACE INTO w007
+OVERWRITE WHERE __time >= TIMESTAMP '2019-08-25 02:00:00' AND __time < TIMESTAMP '2019-08-25 03:00:00'
+SELECT
+  FLOOR(__time TO MINUTE) AS __time,
+  channel,
+  countryIsoCode,
+  countryName,
+  regionIsoCode,
+  regionName,
+  page
+FROM w007
+WHERE __time >= TIMESTAMP '2019-08-25 02:00:00' AND __time < TIMESTAMP '2019-08-25 03:00:00' AND countryName = "Canada"
+PARTITIONED BY HOUR
+CLUSTERED BY page
+```
+
+</details>
+
+## REPLACE for reindexing an existing datasource into itself
+
+<details><summary>Show the query</summary>
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+REPLACE INTO w000
+OVERWRITE ALL
+SELECT
+  FLOOR(__time TO MINUTE) AS __time,
+  channel,
+  countryIsoCode,
+  countryName,
+  regionIsoCode,
+  regionName,
+  page,
+  COUNT(*) AS cnt,
+  SUM(added) AS sum_added,
+  SUM(deleted) AS sum_deleted
+FROM w000
+GROUP BY 1, 2, 3, 4, 5, 6, 7
+PARTITIONED BY HOUR
+CLUSTERED BY page
+```
+
+</details>
+
+## SELECT with EXTERN and JOIN
+
+
+<details><summary>Show the query</summary>
+
+
+```sql
+--:context finalizeAggregations: false
+--:context groupByEnableMultiValueUnnesting: false
+
+WITH flights AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/flights/On_Time_Reporting_Carrier_On_Time_Performance_(1987_present)_2005_11.csv.zip"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"depaturetime","type":"string"},{"name":"arrivalime","type":"string"},{"name":"Year","type":"long"},{"name":"Quarter","type":"long"},{"name":"Month","type":"long"},{"name":"DayofMonth","type":"long"},{"name":"DayOfWeek","type":"long"},{"name":"FlightDate","type":"string"},{"name":"Reporting_Airline","type":"string"},{"name":"DOT_ID_Reporting_Airline","type":"long"},{"name":"IATA_CODE_Reporting_Airline","type":"string"},{"name":"Tail_Number","type":"string"},{"name":"Flight_ [...]
+  )
+)),
+L_AIRPORT AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/dimensions/L_AIRPORT.csv"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"Code","type":"string"},{"name":"Description","type":"string"}]'
+  )
+)),
+L_AIRPORT_ID AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/dimensions/L_AIRPORT_ID.csv"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"Code","type":"long"},{"name":"Description","type":"string"}]'
+  )
+)),
+L_AIRLINE_ID AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/dimensions/L_AIRLINE_ID.csv"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"Code","type":"long"},{"name":"Description","type":"string"}]'
+  )
+)),
+L_CITY_MARKET_ID AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/dimensions/L_CITY_MARKET_ID.csv"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"Code","type":"long"},{"name":"Description","type":"string"}]'
+  )
+)),
+L_CANCELLATION AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/dimensions/L_CANCELLATION.csv"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"Code","type":"string"},{"name":"Description","type":"string"}]'
+  )
+)),
+L_STATE_FIPS AS (
+  SELECT * FROM TABLE(
+  EXTERN(
+    '{"type":"http","uris":["https://static.imply.io/data/FlightCarrierOnTime/dimensions/L_STATE_FIPS.csv"]}',
+    '{"type":"csv","findColumnsFromHeader":true}',
+    '[{"name":"Code","type":"long"},{"name":"Description","type":"string"}]'
+  )
+))
+SELECT
+  depaturetime,
+  arrivalime,
+  -- "Year",
+  -- Quarter,
+  -- "Month",
+  -- DayofMonth,
+  -- DayOfWeek,
+  -- FlightDate,
+  Reporting_Airline,
+
+  DOT_ID_Reporting_Airline,
+  DOTAirlineLookup.Description AS DOT_Reporting_Airline,
+
+  IATA_CODE_Reporting_Airline,
+  Tail_Number,
+  Flight_Number_Reporting_Airline,
+
+  OriginAirportID,
+  OriginAirportIDLookup.Description AS OriginAirport,
+
+  OriginAirportSeqID,
+
+  OriginCityMarketID,
+  OriginCityMarketIDLookup.Description AS OriginCityMarket,
+
+  Origin,
+  OriginAirportLookup.Description AS OriginDescription,
+
+  OriginCityName,
+  OriginState,
+
+  OriginStateFips,
+  OriginStateFipsLookup.Description AS OriginStateFipsDescription,
+
+  OriginStateName,
+  OriginWac,
+
+  DestAirportID,
+  DestAirportIDLookup.Description AS DestAirport,
+
+  DestAirportSeqID,
+
+  DestCityMarketID,
+  DestCityMarketIDLookup.Description AS DestCityMarket,
+
+  Dest,
+  DestAirportLookup.Description AS DestDescription,
+
+  DestCityName,
+  DestState,
+
+  DestStateFips,
+  DestStateFipsLookup.Description AS DestStateFipsDescription,
+
+  DestStateName,
+  DestWac,
+
+  CRSDepTime,
+  DepTime,
+  DepDelay,
+  DepDelayMinutes,
+  DepDel15,
+  DepartureDelayGroups,
+  DepTimeBlk,
+  TaxiOut,
+  WheelsOff,
+  WheelsOn,
+  TaxiIn,
+  CRSArrTime,
+  ArrTime,
+  ArrDelay,
+  ArrDelayMinutes,
+  ArrDel15,
+  ArrivalDelayGroups,
+  ArrTimeBlk,
+
+  Cancelled,
+  CancellationCode,
+  CancellationCodeLookup.Description AS CancellationReason,
+
+  Diverted,
+  CRSElapsedTime,
+  ActualElapsedTime,
+  AirTime,
+  Flights,
+  Distance,
+  DistanceGroup,
+  CarrierDelay,
+  WeatherDelay,
+  NASDelay,
+  SecurityDelay,
+  LateAircraftDelay,
+  FirstDepTime,
+  TotalAddGTime,
+  LongestAddGTime
+FROM "flights"
+LEFT JOIN L_AIRLINE_ID AS DOTAirlineLookup ON DOT_ID_Reporting_Airline = DOTAirlineLookup.Code
+LEFT JOIN L_AIRPORT AS OriginAirportLookup ON Origin = OriginAirportLookup.Code
+LEFT JOIN L_AIRPORT AS DestAirportLookup ON Dest = DestAirportLookup.Code
+LEFT JOIN L_AIRPORT_ID AS OriginAirportIDLookup ON OriginAirportID = OriginAirportIDLookup.Code
+LEFT JOIN L_AIRPORT_ID AS DestAirportIDLookup ON DestAirportID = DestAirportIDLookup.Code
+LEFT JOIN L_CITY_MARKET_ID AS OriginCityMarketIDLookup ON OriginCityMarketID = OriginCityMarketIDLookup.Code
+LEFT JOIN L_CITY_MARKET_ID AS DestCityMarketIDLookup ON DestCityMarketID = DestCityMarketIDLookup.Code
+LEFT JOIN L_STATE_FIPS AS OriginStateFipsLookup ON OriginStateFips = OriginStateFipsLookup.Code
+LEFT JOIN L_STATE_FIPS AS DestStateFipsLookup ON DestStateFips = DestStateFipsLookup.Code
+LEFT JOIN L_CANCELLATION AS CancellationCodeLookup ON CancellationCode = CancellationCodeLookup.Code
+LIMIT 1000
+```
+
+</details>
+
+## Next steps
+
+* [Read Multi-stage queries](./msq-example-queries.md) to learn more about how multi-stage queries work.
+* [Explore the Query view](../operations/druid-console.md) to learn about the UI tools to help you get started.
diff --git a/docs/multi-stage-query/msq-known-issues.md b/docs/multi-stage-query/msq-known-issues.md
new file mode 100644
index 0000000000..ac4b222fe6
--- /dev/null
+++ b/docs/multi-stage-query/msq-known-issues.md
@@ -0,0 +1,118 @@
+---
+id: known-issues
+title: SQL-based ingestion known issues
+sidebar_label: Known issues
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+## General query execution
+
+- There's no fault tolerance. If any task fails, the entire query fails. 
+
+- Only one local file system per server is used for stage output data during multi-stage query
+  execution. If your servers have multiple local file systems, this causes queries to exhaust
+  available disk space earlier than expected. 
+
+- When `msqMaxNumTasks` is higher than the total
+  capacity of the cluster, more tasks may be launched than can run at once. This leads to a
+  [TaskStartTimeout](./msq-reference.md#context-parameters) error code, as there is never enough capacity to run the query.
+  To avoid this, set `msqMaxNumTasks` to a number of tasks that can run simultaneously on your cluster.
+
+- When `msqTaskAssignment` is set to `auto`, the system generates one task per input file for certain splittable
+  input sources where file sizes are not known ahead of time. This includes the `http` input source, where the system
+  generates one task per URI.
+
+## Memory usage
+
+- INSERT queries can consume excessive memory when using complex types due to inaccurate footprint
+  estimation. This can appear as an OutOfMemoryError during the SegmentGenerator stage when using
+  sketches. If you run into this issue, try manually lowering the value of the
+  [`msqRowsInMemory`](./msq-reference.md#context-parameters) parameter.
+
+- EXTERN loads an entire row group into memory at once when reading from Parquet files. Row groups
+  can be up to 1 GB in size, which can lead to excessive heap usage when reading many files in
+  parallel. This can appear as an OutOfMemoryError during stages that read Parquet input files. If
+  you run into this issue, try using a smaller number of worker tasks or you can increase the heap
+  size of your Indexers or of your Middle Manager-launched indexing tasks.
+
+- Ingesting a very long row may consume excessive memory and result in an OutOfMemoryError. If a row is read 
+  which requires more memory than is available, the service might throw OutOfMemoryError. If you run into this
+  issue, allocate enough memory to be able to store the largest row to the indexer. 
+
+## SELECT queries
+
+- SELECT query results do not include real-time data until it has been published.
+
+- TIMESTAMP types are formatted as numbers rather than ISO8601 timestamp
+  strings, which differs from Druid's standard result format. 
+
+- BOOLEAN types are formatted as numbers like `1` and `0` rather
+  than `true` or `false`, which differs from Druid's standard result
+  format. 
+
+- TopN is not implemented. The context parameter
+  `useApproximateTopN` is ignored and always treated as if it
+  were `false`. Therefore, topN-shaped queries will
+  always run using the groupBy engine. There is no loss of
+  functionality, but there may be a performance impact, since
+  these queries will run using an exact algorithm instead of an
+  approximate one.
+- GROUPING SETS is not implemented. Queries that use GROUPING SETS
+  will fail.
+- The numeric flavors of the EARLIEST and LATEST aggregators do not work properly. Attempting to use the numeric flavors of these aggregators will lead to an error like `java.lang.ClassCastException: class java.lang.Double cannot be cast to class org.apache.druid.collections.SerializablePair`. The string flavors, however, do work properly.
+
+##  INSERT queries
+
+- The [schemaless dimensions](../ingestion/ingestion-spec.md#inclusions-and-exclusions)
+feature is not available. All columns and their types must be specified explicitly.
+
+- [Segment metadata queries](../querying/segmentmetadataquery.md)
+  on datasources ingested with the Multi-Stage Query Engine will return values for`timestampSpec` that are not usable
+  for introspection.
+
+- When INSERT with GROUP BY does the match the criteria mentioned in [GROUP BY](./index.md#group-by),  the multi-stage engine generates segments that Druid's compaction
+  functionality is not able to further roll up. This applies to automatic compaction as well as manually
+  issued `compact` tasks. Individual queries executed with the multi-stage engine always guarantee
+  perfect rollup for their output, so this only matters if you are performing a sequence of INSERT
+  queries that each append data to the same time chunk. If necessary, you can compact such data
+  using another SQL query instead of a `compact` task.
+
+- When using INSERT with GROUP BY, splitting of large partitions is not currently
+  implemented. If a single partition key appears in a
+  very large number of rows, an oversized segment will be created.
+  You can mitigate this by adding additional columns to your
+  partition key. Note that partition splitting _does_ work properly
+  when performing INSERT without GROUP BY.
+
+- INSERT with column lists, like
+  `INSERT INTO tbl (a, b, c) SELECT ...`, is not implemented.
+
+## EXTERN queries
+
+- EXTERN does not accept `druid` input sources.
+
+## Missing guardrails
+
+- Maximum number of input files. Since there's no limit, the controller can potentially run out of memory tracking all input files
+
+- Maximum amount of local disk space to use for temporary data. No guardrail today means worker tasks may exhaust all available disk space. In this case, you will receive an [UnknownError](./msq-reference.md#error-codes)) with a message including "No space left on device".
\ No newline at end of file
diff --git a/docs/multi-stage-query/msq-reference.md b/docs/multi-stage-query/msq-reference.md
new file mode 100644
index 0000000000..f5ef07124c
--- /dev/null
+++ b/docs/multi-stage-query/msq-reference.md
@@ -0,0 +1,169 @@
+---
+id: reference
+title: SQL-based ingestion reference
+sidebar_label: Reference
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+This topic is a reference guide for the multi-stage query architecture in Apache Druid.
+
+## Context parameters
+
+In addition to the Druid SQL [context parameters](../querying/sql-query-context.md), the multi-stage query task engine accepts certain context parameters that are specific to it. 
+
+Use context parameters alongside your queries to customize the behavior of the query. If you're using the API, include the context parameters in the query context when you submit a query:
+
+```json
+{
+  "query": "SELECT 1 + 1",
+  "context": {
+    "<key>": "<value>",
+    "maxNumTasks": 3
+  }
+}
+```
+
+If you're using the Druid console, you can specify the context parameters through various UI options.
+
+The following table lists the context parameters for the MSQ task engine:
+
+|Parameter|Description|Default value|
+|---------|-----------|-------------|
+| maxNumTasks | SELECT, INSERT, REPLACE<br /><br />The maximum total number of tasks to launch, including the controller task. The lowest possible value for this setting is 2: one controller and one worker. All tasks must be able to launch simultaneously. If they cannot, the query returns a `TaskStartTimeout` error code after approximately 10 minutes.<br /><br />May also be provided as `numTasks`. If both are present, `maxNumTasks` takes priority.| 2 |
+| taskAssignment | SELECT, INSERT, REPLACE<br /><br />Determines how many tasks to use. Possible values include: <ul><li>`max`: Use as many tasks as possible, up to the maximum `maxNumTasks`.</li><li>`auto`: Use as few tasks as possible without exceeding 10 GiB or 10,000 files per task. Review the [limitations](./msq-known-issues.md#general-query-execution) of `auto` mode before using it.</li></ui>| `max` |
+| finalizeAggregations | SELECT, INSERT, REPLACE<br /><br />Determines the type of aggregation to return. If true, Druid finalizes the results of complex aggregations that directly appear in query results. If false, Druid returns the aggregation's intermediate type rather than finalized type. This parameter is useful during ingestion, where it enables storing sketches directly in Druid tables. For more information about aggregations, see [SQL aggregation functions](../querying/sql-aggreg [...]
+| rowsInMemory | INSERT or REPLACE<br /><br />Maximum number of rows to store in memory at once before flushing to disk during the segment generation process. Ignored for non-INSERT queries. In most cases, use the default value. You may need to override the default if you run into one of the [known issues around memory usage](./msq-known-issues.md#memory-usage)</a>. | 100,000 |
+| segmentSortOrder | INSERT or REPLACE<br /><br />Normally, Druid sorts rows in individual segments using `__time` first, followed by the [CLUSTERED BY](./index.md#clustered-by) clause. When you set `segmentSortOrder`, Druid sorts rows in segments using this column list first, followed by the CLUSTERED BY order.<br /><br />You provide the column list as comma-separated values or as a JSON array in string form. If your query includes `__time`, then this list must begin with `__time`. For  [...]
+| maxParseExceptions| SELECT, INSERT, REPLACE<br /><br />Maximum number of parse exceptions that are ignored while executing the query before it stops with `TooManyWarningsFault`. To ignore all the parse exceptions, set the value to -1.| 0 |
+| rowsPerSegment | INSERT or REPLACE<br /><br />The number of rows per segment to target. The actual number of rows per segment may be somewhat higher or lower than this number. In most cases, use the default. For general information about sizing rows per segment, see [Segment Size Optimization](../operations/segment-optimization.md). | 3,000,000 |
+| sqlTimeZone | Sets the time zone for this connection, which affects how time functions and timestamp literals behave. Use a time zone name like "America/Los_Angeles" or offset like "-08:00".| `druid.sql.planner.sqlTimeZone` on the Broker (default: UTC)|
+| useApproximateCountDistinct | Whether to use an approximate cardinality algorithm for `COUNT(DISTINCT foo)`.| `druid.sql.planner.useApproximateCountDistinct` on the Broker (default: true)|
+
+## Error codes
+
+Error codes have corresponding human-readable messages that explain the error. For more information about the error codes, see [Error codes](./msq-concepts.md#error-codes).
+
+## SQL syntax
+
+The MSQ task engine has three primary SQL functions: 
+
+- EXTERN
+- INSERT
+- REPLACE
+
+For information about using these functions and their corresponding examples, see [MSQ task engine query syntax](./index.md#msq-task-engine-query-syntax). For information about adjusting the shape of your data, see [Adjust query behavior](./index.md#adjust-query-behavior).
+
+### EXTERN
+
+Use the EXTERN function to read external data.
+
+Function format:
+
+```sql
+SELECT
+ <column>
+FROM TABLE(
+  EXTERN(
+    '<Druid input source>',
+    '<Druid input format>',
+    '<row signature>'
+  )
+)
+```
+
+EXTERN consists of the following parts:
+
+1.  Any [Druid input source](../ingestion/native-batch-input-source.md) as a JSON-encoded string.
+2.  Any [Druid input format](../ingestion/data-formats.md) as a JSON-encoded string.
+3.  A row signature, as a JSON-encoded array of column descriptors. Each column descriptor must have a `name` and a `type`. The type can be `string`, `long`, `double`, or `float`. This row signature is used to map the external data into the SQL layer.
+
+### INSERT
+
+Use the INSERT function to insert data.
+
+Unlike standard SQL, INSERT inserts data according to column name and not positionally. This means that it is important for the output column names of subsequent INSERT queries to be the same as the table. Do not rely on their positions within the SELECT clause.
+
+Function format:
+
+```sql
+INSERT INTO <table name>
+SELECT
+  <column>
+FROM <table>
+PARTITIONED BY <time frame>
+```
+
+INSERT consists of the following parts:
+
+1. Optional [context parameters](./msq-reference.md#context-parameters).
+2. An `INSERT INTO <dataSource>` clause at the start of your query, such as `INSERT INTO your-table`.
+3. A clause for the data you want to insert, such as `SELECT...FROM TABLE...`. You can use EXTERN to reference external tables using the following format: ``TABLE(EXTERN(...))`.
+4. A [PARTITIONED BY](./index.md#partitioned-by) clause for your INSERT statement. For example, use PARTITIONED BY DAY for daily partitioning or PARTITIONED BY ALL TIME to skip time partitioning completely.
+5. An optional [CLUSTERED BY](./index.md#clustered-by) clause.
+
+### REPLACE
+
+You can use the REPLACE function to replace all or some of the data.
+
+Unlike standard SQL, REPLACE inserts data according to column name and not positionally. This means that it is important for the output column names of subsequent REPLACE queries to be the same as the table. Do not rely on their positions within the SELECT clause.
+
+#### REPLACE all data
+
+Function format to replace all data:
+
+```sql
+REPLACE INTO <target table>
+OVERWRITE ALL
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  <column>
+FROM <source table>
+
+PARTITIONED BY <time>
+```
+
+#### REPLACE specific data
+
+Function format to replace specific data:
+
+```sql
+REPLACE INTO <target table>
+OVERWRITE WHERE __time >= TIMESTAMP '<lower bound>' AND __time < TIMESTAMP '<upper bound>'
+SELECT
+  TIME_PARSE("timestamp") AS __time,
+  <column>
+FROM <source table>
+
+PARTITIONED BY <time>
+```
+
+REPLACE consists of the following parts:
+
+1. Optional [context parameters](./msq-reference.md#context-parameters).
+2. A `REPLACE INTO <dataSource>` clause at the start of your query, such as `REPLACE INTO your-table.`
+3. An OVERWRITE clause after the datasource, either OVERWRITE ALL or OVERWRITE WHERE:
+  - OVERWRITE ALL replaces the entire existing datasource with the results of the query.
+  - OVERWRITE WHERE drops the time segments that match the condition you set. Conditions are based on the `__time` column and use the format `__time [< > = <= >=] TIMESTAMP`. Use them with AND, OR, and NOT between them, inclusive of the timestamps specified. For example, see [REPLACE INTO ... OVERWRITE WHERE ... SELECT](./index.md#replace-some-data).
+4. A clause for the actual data you want to use for the replacement.
+5. A [PARTITIONED BY](./index.md#partitioned-by) clause to your REPLACE statement. For example, use PARTITIONED BY DAY for daily partitioning, or PARTITIONED BY ALL TIME to skip time partitioning completely.
+6. An optional [CLUSTERED BY](./index.md#clustered-by) clause.
diff --git a/docs/multi-stage-query/msq-security.md b/docs/multi-stage-query/msq-security.md
new file mode 100644
index 0000000000..51c74712d0
--- /dev/null
+++ b/docs/multi-stage-query/msq-security.md
@@ -0,0 +1,43 @@
+---
+id: security
+title: SQL-based ingestion security
+sidebar_label: Security
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+All authenticated users can use the multi-stage query task engine (MSQ task engine) through the UI and API if the extension is loaded. However, without additional permissions, users are not able to issue queries that read or write Druid datasources or external data. The permission you need depends on what you are trying to do.
+
+The permission required to submit a query depends on the type of query:
+
+  - SELECT from a Druid datasource requires the READ DATASOURCE permission on that
+  datasource.
+  - INSERT or REPLACE into a Druid datasource requires the WRITE DATASOURCE permission on that
+  datasource.
+  - EXTERN references to external data require READ permission on the resource name "EXTERNAL" of the resource type "EXTERNAL". Users without the correct permission encounter a 403 error when trying to run queries that include EXTERN.
+
+Query tasks that you submit to the MSQ task engine are Overlord tasks, so they follow the Overlord's (indexer) model. This means that users with access to the Overlord API can perform some actions even if they didn't submit the query. The actions include retrieving the status or canceling a query. For more information about the Overlord API and the task API, see [APIs for SQL-based ingestion](./msq-api.md).
+
+To interact with a query through the Overlord API, you need the following permissions:
+
+- INSERT or REPLACE queries: You must have READ DATASOURCE permission on the output datasource.
+- SELECT queries: You must have read permissions on the `__query_select` datasource, which is a stub datasource that gets created.
diff --git a/docs/multi-stage-query/msq-tutorial-connect-external-data.md b/docs/multi-stage-query/msq-tutorial-connect-external-data.md
new file mode 100644
index 0000000000..26f2c8f62a
--- /dev/null
+++ b/docs/multi-stage-query/msq-tutorial-connect-external-data.md
@@ -0,0 +1,145 @@
+---
+id: connect-external-data
+title: Tutorial - Connect external data for SQL-based ingestion
+description: How to generate a query that references externally hosted data
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+This tutorial demonstrates how to generate a query that references externally hosted data using the **Connect external data** wizard.
+
+The following example uses EXTERN to query a JSON file located at https://static.imply.io/data/wikipedia.json.gz.
+
+Although you can manually create a query in the UI, you can use Druid to generate a base query for you that you can modify to meet your requirements.
+
+To generate a query from external data, do the following:
+
+1. In the **Query** view of the Druid console, click **Connect external data**.
+2. On the **Select input type** screen, choose **HTTP(s)** and enter the following value in the **URIs** field: `https://static.imply.io/data/wikipedia.json.gz`. Leave the HTTP auth username and password blank.
+3. Click **Connect data**.
+4. On the **Parse** screen, you can perform additional actions before you load the data into Druid:
+   - Expand a row to see what data it corresponds to from the source.
+   - Customize how Druid handles the data by selecting the **Input format** and its related options, such as adding **JSON parser features** for JSON files.
+5. When you're ready, click **Done**. You're returned to the **Query** view where you can see the newly generated query:
+
+   - The query inserts the data from the external source into a table named `wikipedia`.
+   - Context parameters appear before the query in the syntax unique to the Druid console: `--: context {key}: {value}`. When submitting queries to Druid directly, set the `context` parameters in the context section of the SQL query object. For more information about context parameters, see [Context parameters](./msq-reference.md#context-parameters).
+
+   <details><summary>Show the query</summary>
+
+   ```sql
+   REPLACE INTO "wikipedia" OVERWRITE ALL
+   WITH ext AS (SELECT *
+   FROM TABLE(
+     EXTERN(
+       '{"type":"http","uris":["https://static.imply.io/data/wikipedia.json.gz"]}',
+       '{"type":"json"}',
+       '[{"name":"isRobot","type":"string"},{"name":"channel","type":"string"},{"name":"timestamp","type":"string"},{"name":"flags","type":"string"},{"name":"isUnpatrolled","type":"string"},{"name":"page","type":"string"},{"name":"diffUrl","type":"string"},{"name":"added","type":"long"},{"name":"comment","type":"string"},{"name":"commentLength","type":"long"},{"name":"isNew","type":"string"},{"name":"isMinor","type":"string"},{"name":"delta","type":"long"},{"name":"isAnonymous","type":"s [...]
+     )
+   ))
+   SELECT
+     TIME_PARSE("timestamp") AS __time,
+     isRobot,
+     channel,
+     flags,
+     isUnpatrolled,
+     page,
+     diffUrl,
+     added,
+     comment,
+     commentLength,
+     isNew,
+     isMinor,
+     delta,
+     isAnonymous,
+     user,
+     deltaBucket,
+     deleted,
+     namespace,
+     cityName,
+     countryName,
+     regionIsoCode,
+     metroCode,
+     countryIsoCode,
+     regionName
+   FROM ext
+   PARTITIONED BY DAY
+   ```
+   </details>
+
+6. Review and modify the query to meet your needs. For example, you can rename the table or change segment granularity. To partition by something other than ALL, include `TIME_PARSE("timestamp") AS __time` in your SELECT statement.
+
+   For example, to specify day-based segment granularity, change the partitioning to `PARTITIONED BY DAY`:
+      
+     ```sql
+      ...
+      SELECT
+        TIME_PARSE("timestamp") AS __time,
+      ...
+      ...
+       PARTITIONED BY DAY
+     ```
+
+1. Optionally, select **Preview** to review the data before you ingest it. A preview runs the query without the INSERT INTO clause and with an added LIMIT to the main query and to all helper queries. You can see the general shape of the data before you commit to inserting it. The LIMITs make the query run faster but can cause incomplete results.
+2. Click **Run** to launch your query. The query returns information including its duration and the number of rows inserted into the table.
+
+## Query the data
+
+You can query the `wikipedia` table after the ingestion completes.
+For example, you can analyze the data in the table to produce a list of top channels:
+
+```sql
+SELECT
+  channel,
+  COUNT(*)
+FROM "wikipedia"
+GROUP BY channel
+ORDER BY COUNT(*) DESC
+```
+
+With the EXTERN function, you could run the same query on the external data directly without ingesting it first:
+
+<details><summary>Show the query</summary>
+
+```sql
+SELECT
+  channel,
+  COUNT(*)
+FROM TABLE(
+  EXTERN(
+    '{"type": "http", "uris": ["https://static.imply.io/data/wikipedia.json.gz"]}',
+    '{"type": "json"}',
+    '[{"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": " [...]
+  )
+)
+GROUP BY channel
+ORDER BY COUNT(*) DESC
+```
+
+</details>
+
+## Further reading
+
+See the following topics to learn more:
+
+* [MSQ task engine query syntax](./index.md#msq-task-engine-query-syntax) for information about the different query components.
+* [Reference](./msq-reference.md) for reference on context parameters, functions, and error codes.
diff --git a/docs/multi-stage-query/msq-tutorial-convert-ingest-spec.md b/docs/multi-stage-query/msq-tutorial-convert-ingest-spec.md
new file mode 100644
index 0000000000..55038e5cfa
--- /dev/null
+++ b/docs/multi-stage-query/msq-tutorial-convert-ingest-spec.md
@@ -0,0 +1,169 @@
+---
+id: convert-json-spec
+title: Tutorial - Convert an ingestion spec for SQL-based ingestion
+description: How to convert an ingestion spec to a query for SQL-based ingestion in the Druid console.
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+> SQL-based ingestion using the multi-stage query task engine is our recommended solution starting in Druid 24.0. Alternative ingestion solutions, such as native batch and Hadoop-based ingestion systems, will still be supported. We recommend you read all [known issues](./msq-known-issues.md) and test the feature in a development environment before rolling it out in production. Using the multi-stage query task engine with `SELECT` statements that do not write to a datasource is experimental.
+
+If you're already ingesting data with Druid's native SQL engine, you can use the Druid console to convert the ingestion spec to a SQL query that the multi-stage query task engine can use to ingest data.
+
+This tutorial demonstrates how to convert the ingestion spec to a query task in the Druid console.
+
+To convert the ingestion spec to a query task, do the following:
+
+1. In the **Query** view of the Druid console, navigate to the menu bar that includes **Run**.
+2. Click the ellipsis icon and select **Convert ingestion spec to SQL**.
+  ![Convert ingestion spec to SQL](../assets/multi-stage-query/tutorial-msq-convert.png "Convert ingestion spec to SQL")
+3. In the **Ingestion spec to covert** window, insert your ingestion spec. You can use your own spec or the sample ingestion spec provided in the tutorial. The sample spec uses data hosted at `https://static.imply.io/data/wikipedia.json.gz` and loads it into a table named `wikipedia`:
+
+   <details><summary>Show the spec</summary>
+   
+   ```json
+   {
+     "type": "index_parallel",
+     "spec": {
+       "ioConfig": {
+         "type": "index_parallel",
+         "inputSource": {
+           "type": "http",
+           "uris": [
+             "https://static.imply.io/data/wikipedia.json.gz"
+           ]
+         },
+         "inputFormat": {
+           "type": "json"
+         }
+       },
+       "tuningConfig": {
+         "type": "index_parallel",
+         "partitionsSpec": {
+           "type": "dynamic"
+         }
+       },
+       "dataSchema": {
+         "dataSource": "wikipedia",
+         "timestampSpec": {
+           "column": "timestamp",
+           "format": "iso"
+         },
+         "dimensionsSpec": {
+           "dimensions": [
+             "isRobot",
+             "channel",
+             "flags",
+             "isUnpatrolled",
+             "page",
+             "diffUrl",
+             {
+               "type": "long",
+               "name": "added"
+             },
+             "comment",
+             {
+               "type": "long",
+               "name": "commentLength"
+             },
+             "isNew",
+             "isMinor",
+             {
+               "type": "long",
+               "name": "delta"
+             },
+             "isAnonymous",
+             "user",
+             {
+               "type": "long",
+               "name": "deltaBucket"
+             },
+             {
+               "type": "long",
+               "name": "deleted"
+             },
+             "namespace",
+             "cityName",
+             "countryName",
+             "regionIsoCode",
+             "metroCode",
+             "countryIsoCode",
+             "regionName"
+           ]
+         },
+         "granularitySpec": {
+           "queryGranularity": "none",
+           "rollup": false,
+           "segmentGranularity": "day"
+         }
+       }
+     }
+   }
+   ```
+   
+   </details>
+
+4. Click **Submit** to submit the spec. The Druid console uses the JSON-based ingestion spec to generate a SQL query that you can use instead. This is what the query looks like for the sample ingestion spec:
+   
+   <details><summary>Show the query</summary>
+
+   ```sql
+   -- This SQL query was auto generated from an ingestion spec
+   REPLACE INTO wikipedia OVERWRITE ALL
+   WITH source AS (SELECT * FROM TABLE(
+     EXTERN(
+       '{"type":"http","uris":["https://static.imply.io/data/wikipedia.json.gz"]}',
+       '{"type":"json"}',
+       '[{"name":"timestamp","type":"string"},{"name":"isRobot","type":"string"},{"name":"channel","type":"string"},{"name":"flags","type":"string"},{"name":"isUnpatrolled","type":"string"},{"name":"page","type":"string"},{"name":"diffUrl","type":"string"},{"name":"added","type":"long"},{"name":"comment","type":"string"},{"name":"commentLength","type":"long"},{"name":"isNew","type":"string"},{"name":"isMinor","type":"string"},{"name":"delta","type":"long"},{"name":"isAnonymous","type":"s [...]
+     )
+   ))
+   SELECT
+     TIME_PARSE("timestamp") AS __time,
+     "isRobot",
+     "channel",
+     "flags",
+     "isUnpatrolled",
+     "page",
+     "diffUrl",
+     "added",
+     "comment",
+     "commentLength",
+     "isNew",
+     "isMinor",
+     "delta",
+     "isAnonymous",
+     "user",
+     "deltaBucket",
+     "deleted",
+     "namespace",
+     "cityName",
+     "countryName",
+     "regionIsoCode",
+     "metroCode",
+     "countryIsoCode",
+     "regionName"
+   FROM source
+   PARTITIONED BY DAY 
+   ```
+   
+   </details>
+
+4. Review the generated SQL query to make sure it matches your requirements and does what you expect.
+5. Click **Run** to start the ingestion.
\ No newline at end of file
diff --git a/docs/operations/druid-console.md b/docs/operations/druid-console.md
index 4ab3756917..1e0bf2545d 100644
--- a/docs/operations/druid-console.md
+++ b/docs/operations/druid-console.md
@@ -60,6 +60,48 @@ You can access the [data loader](#data-loader) and [lookups view](#lookups) from
 
 ![home-view](../assets/web-console-01-home-view.png "home view")
 
+## Query
+
+SQL-based ingestion and the multi-stage query task engine use the **Query** view, which provides you with a UI to edit and use SQL queries. You should see this UI automatically in Druid 24.0 and later since the multi-stage query extension is loaded by default. 
+
+The following screenshot shows a populated enhanced **Query** view along with a description of its parts:
+
+![Annotated multi-stage Query view](../assets/multi-stage-query/ui-annotated.png)
+
+1. The multi-stage, tab-enabled, **Query** view is where you can issue queries and see results.
+All other views are unchanged from the non-enhanced version. You can still access the original **Query** view by navigating to `#query` in the URL.
+You can tell that you're looking at the updated **Query** view by the presence of the tabs (3).
+2. The **druid** panel shows the available schemas, datasources, and columns.
+3. Query tabs allow you to manage and run several queries at once.
+Click the plus icon to open a new tab.
+To manipulate existing tabs, click the tab name.
+4. The tab bar contains some helpful tools including the **Connect external data** button that samples external data and creates an initial query with the appropriate `EXTERN` definition that you can then edit as needed.
+5. The **Recent query tasks** panel lets you see currently running and previous queries from all users in the cluster.
+It is equivalent to the **Task** view in the **Ingestion** view with the filter of `type='query_controller'`.
+6. You can click on each query entry to attach to that query in a new tab.
+7. You can download an archive of all the pertinent details about the query that you can share.
+8. The **Run** button runs the query.
+9. The **Preview** button appears when you enter an INSERT/REPLACE query. It runs the query inline without the INSERT/REPLACE clause and with an added LIMIT to give you a preview of the data that would be ingested if you click **Run**.
+The added LIMIT makes the query run faster but provides incomplete results.
+10. The engine selector lets you choose which engine (API endpoint) to send a query to. By default, it automatically picks which endpoint to use based on an analysis of the query, but you can select a specific engine explicitly. You can also configure the engine specific context parameters from this menu.
+11. The **Max tasks** picker appears when you have the **sql-msq-task** engine selected. It lets you configure the degree of parallelism.
+12. The More menu (**...**) contains the following helpful tools:
+- **Explain SQL query** shows you the logical plan returned by `EXPLAIN PLAN FOR` for a SQL query.
+- **Query history** shows you previously executed queries.
+- **Convert ingestion spec to SQL** lets you convert a native batch ingestion spec to an equivalent SQL query.
+- **Attach tab from task ID** lets you create a new tab from the task ID of a query executed on this cluster.
+- **Open query detail archive** lets you open a detail archive generated on any cluster by (7).
+13. The query timer indicates how long the query has been running for.
+14. The **(cancel)** link cancels the currently running query.
+15. The main progress bar shows the overall progress of the query.
+The progress is computed from the various counters in the live reports (16).
+16. The **Current stage** progress bar shows the progress for the currently running query stage.
+If several stages are executing concurrently, it conservatively shows the information for the earliest executing stage.
+17. The live query reports show detailed information of all the stages (past, present, and future). The live reports are shown while the query is running. You can hide the report if you want.
+After queries finish, you can access them by clicking on the query time indicator or from the **Recent query tasks** panel (6).
+18. You can expand each stage of the live query report by clicking on the triangle to show per worker and per partition statistics.
+
+
 ## Data loader
 
 You can use the data loader to build an ingestion spec with a step-by-step wizard.
@@ -124,18 +166,6 @@ You can group the nodes by type or by tier to get meaningful summary statistics.
 
 ![servers](../assets/web-console-10-servers.png)
 
-## Query
-
-The **Query** view lets you issue [Druid SQL](../querying/sql.md) queries and display the results as a table.
-The view will attempt to infer your query and let you modify the query via contextual actions such as adding filters and changing the sort order when possible.
-
-From the ellipsis menu beside **Run**, you can view your query history, see the native query translation for a given Druid SQL query, and set the [query context](../querying/query-context.md).
-
-![query-sql](../assets/web-console-11-query-sql.png)
-
-You can also use the query editor to issue queries in Druid's [native query format](../querying/querying.md), which is JSON over HTTP.
-
-![query-rune](../assets/web-console-12-query-rune.png)
 
 ## Lookups
 
diff --git a/docs/operations/security-user-auth.md b/docs/operations/security-user-auth.md
index 327d66de18..807db960ae 100644
--- a/docs/operations/security-user-auth.md
+++ b/docs/operations/security-user-auth.md
@@ -29,20 +29,30 @@ This document describes the Druid security model that extensions use to enable u
 
 At the center of the Druid user authentication and authorization model are _resources_ and _actions_. A resource is something that authenticated users are trying to access or modify. An action is something that users are trying to do. 
 
+### Resource types
+
 Druid uses the following resource types:
 
 * DATASOURCE &ndash; Each Druid table (i.e., `tables` in the `druid` schema in SQL) is a resource.
 * CONFIG &ndash; Configuration resources exposed by the cluster components. 
+* EXTERNAL &ndash; External data read through the [EXTERN function](../multi-stage-query/index.md#read-external-data) in SQL.
 * STATE &ndash; Cluster-wide state resources.
 * SYSTEM_TABLE &ndash; when the Broker property `druid.sql.planner.authorizeSystemTablesDirectly` is true, then Druid uses this resource type to authorize the system tables in the `sys` schema in SQL.
 
-For specific resources associated with the types, see the endpoint list below and corresponding descriptions in [API Reference](./api-reference.md).
+For specific resources associated with the resource types, see [Defining permissions](#defining-permissions) and the corresponding endpoint descriptions in [API reference](./api-reference.md).
+
+### Actions
 
-There are two actions:
+Users perform one of the following actions on resources:
 
 * READ &ndash; Used for read-only operations.
 * WRITE &ndash; Used for operations that are not read-only.
 
+WRITE permission on a resource does not include READ permission. If a user requires both READ and WRITE permissions on a resource, you must grant them both explicitly. For instance, a user with only `DATASOURCE READ` permission
+might have access to an API or a system schema record that a user with `DATASOURCE WRITE` permission would not have access to.
+
+### User types
+
 In practice, most deployments will only need to define two classes of users: 
 
 * Administrators, who have WRITE action permissions on all resource types. These users will add datasources and administer the system.  
@@ -50,9 +60,6 @@ In practice, most deployments will only need to define two classes of users:
 
 It is important to note that WRITE access to DATASOURCE grants a user broad access. For instance, such users will have access to the Druid file system, S3 buckets, and credentials, among other things. As such, the ability to add and manage datasources should be allocated selectively to administrators.   
 
-`WRITE` permission on a resource does not include `READ` permission. If a user requires both `READ` and `WRITE` permissions on a resource, you must grant them both explicitly. For instance, a user with only `DATASOURCE READ` permission
-might have access to an API or a system schema record that a user with `DATASOURCE WRITE` permission would not have access to.
-
 ## Default user accounts
 
 ### Authenticator
@@ -67,9 +74,9 @@ Each Authorizer will always have a default "admin" and "druid_system" user with
 
 ## Defining permissions
 
-There are two action types in Druid: READ and WRITE
-
-Druid uses the following resource types: `DATASOURCE`, `CONFIG`, `STATE`, and `SYSTEM_TABLE`.
+You define permissions that you then grant to user groups.
+Permissions are defined by resource type, action, and resource name.
+This section describes the resource names available for each resource type.
 
 ### `DATASOURCE`
 Resource names for this type are datasource names. Specifying a datasource permission allows the administrator to grant users access to specific datasources.
@@ -94,6 +101,13 @@ There are two possible resource names for the "CONFIG" resource type, "CONFIG" a
 |`/druid-ext/basic-security/authentication`|coordinator|
 |`/druid-ext/basic-security/authorization`|coordinator|
 
+### `EXTERNAL`
+
+The EXTERNAL resource type only accepts the resource name "EXTERNAL".
+Granting a user access to EXTERNAL resources allows them to run queries that include
+the [EXTERN function](../multi-stage-query/index.md#read-external-data) in SQL
+to read external data.
+
 ### `STATE`
 There is only one possible resource name for the "STATE" config resource type, "STATE". Granting a user access to STATE resources allows them to access the following endpoints.
 
@@ -127,14 +141,16 @@ There is only one possible resource name for the "STATE" config resource type, "
 Resource names for this type are system schema table names in the `sys` schema in SQL, for example `sys.segments` and `sys.server_segments`. Druid only enforces authorization for `SYSTEM_TABLE` resources when the Broker property `druid.sql.planner.authorizeSystemTablesDirectly` is true.
 ### HTTP methods
 
-For information on what HTTP methods are supported on a particular request endpoint, please refer to the [API documentation](./api-reference.md).
+For information on what HTTP methods are supported on a particular request endpoint, refer to [API reference](./api-reference.md).
 
-GET requires READ permission, while POST and DELETE require WRITE permission.
+`GET` requests require READ permissions, while `POST` and `DELETE` requests require WRITE permissions.
 
-### SQL Permissions
+### SQL permissions
 
 Queries on Druid datasources require DATASOURCE READ permissions for the specified datasource.
 
+Queries to access external data through the [EXTERN function](../multi-stage-query/index.md#read-external-data) require EXTERNAL READ permissions.
+
 Queries on [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md#information-schema) return information about datasources that the caller has DATASOURCE READ access to. Other
 datasources are omitted.
 
@@ -147,7 +163,7 @@ Queries on the [system schema tables](../querying/sql-metadata-tables.md#system-
 
 When the Broker property `druid.sql.planner.authorizeSystemTablesDirectly` is true, users also require  `SYSTEM_TABLE` authorization on a system schema table to query it.
 
-## Configuration Propagation
+## Configuration propagation
 
 To prevent excessive load on the Coordinator, the Authenticator and Authorizer user/role Druid metadata store state is cached on each Druid process.
 
diff --git a/docs/tutorials/docker.md b/docs/tutorials/docker.md
index 2c3353f3be..2491c34798 100644
--- a/docs/tutorials/docker.md
+++ b/docs/tutorials/docker.md
@@ -116,7 +116,7 @@ It takes a few seconds for all the Druid processes to fully start up. If you ope
 
 ## Using the cluster
 
-From here you can follow along with the [Quickstart](./index.md#step-4-load-data). For production use, refine your `docker-compose.yml` file to add any additional external service dependencies as necessary.
+From here you can follow along with the [Quickstart](./index.md#load-data). For production use, refine your `docker-compose.yml` file to add any additional external service dependencies as necessary.
 
 You can explore the Druid containers using Docker to start a shell:
 
diff --git a/docs/tutorials/index.md b/docs/tutorials/index.md
index 9909e24fa3..ffefdc07fd 100644
--- a/docs/tutorials/index.md
+++ b/docs/tutorials/index.md
@@ -23,31 +23,34 @@ title: "Quickstart"
   -->
 
 
-This quickstart gets you started with Apache Druid and introduces you to some of its basic features. 
-Following these steps, you will install Druid and load sample 
-data using its native batch ingestion feature. 
+This quickstart gets you started with Apache Druid using the `micro-quickstart` startup configuration and introduces you to some Druid features, including the MSQ task engine that's part of the [multi-stage query architecture](../multi-stage-query/index.md). 
 
-Before starting, you may want to read the [general Druid overview](../design/index.md) and
-[ingestion overview](../ingestion/index.md), as the tutorials refer to concepts discussed on those pages.
+With the MSQ task engine, you can write query tasks that can reference [external data](../multi-stage-query/index.md#read-external-data) as well as perform ingestion with SQL [INSERT](../multi-stage-query/index.md#insert-data) and [REPLACE](../multi-stage-query/index.md#replace-data), eliminating the need to generate JSON-based ingestion specs.
+
+In this quickstart, you'll do the following:
+- install Druid
+- start up Druid services
+- use the MSQ task engine to ingest data
+
+Druid supports different ingestion engines. While we recommend SQL based ingestion, you can find tutorials for other modes of ingestion, such as [Load data with native batch ingestion](tutorial-batch-native.md).
 
 ## Requirements
 
 You can follow these steps on a relatively small machine, such as a laptop with around 4 CPU and 16 GiB of RAM. 
 
-Druid comes with several startup configuration profiles for a range of machine sizes. 
-The `micro-quickstart`configuration profile shown here is suitable for evaluating Druid. If you want to 
+Druid comes equipped with several startup configuration profiles for a range of machine sizes.
+
+The `micro-quickstart` configuration profile is suitable for evaluating Druid. If you want to 
 try out Druid's performance or scaling capabilities, you'll need a larger machine and configuration profile.
 
 The configuration profiles included with Druid range from the even smaller _Nano-Quickstart_ configuration (1 CPU, 4GiB RAM) 
-to the _X-Large_ configuration (64 CPU, 512GiB RAM). For more information, see 
-[Single server deployment](../operations/single-server.md). Alternatively, see [Clustered deployment](./cluster.md) for 
-information on deploying Druid services across clustered machines. 
+to the _X-Large_ configuration (64 CPU, 512GiB RAM). For more information, see [Single server deployment](../operations/single-server.md).
+For information on deploying Druid services across clustered machines, see [Clustered deployment](./cluster.md).
 
 The software requirements for the installation machine are:
 
-* Linux, Mac OS X, or other Unix-like OS (Windows is not supported).
-* Java 8, Update 92 or later (8u92+) or Java 11. See the [Java runtime](../operations/java.md) page for additional
-  information about selecting and configuring a Java runtime.
+* Linux, Mac OS X, or other Unix-like OS (Windows is not supported)
+* Java 8, Update 92 or later (8u92+) or Java 11
 
 > Druid relies on the environment variables `JAVA_HOME` or `DRUID_JAVA_HOME` to find Java on the machine. You can set
 `DRUID_JAVA_HOME` if there is more than one instance of Java. To verify Java requirements for your environment, run the 
@@ -55,26 +58,30 @@ The software requirements for the installation machine are:
 
 Before installing a production Druid instance, be sure to consider the user account on the operating system under 
 which Druid will run. This is important because any Druid console user will have, effectively, the same permissions as 
-that user. So, for example, the file browser UI will show console users the files that the underlying user can 
-access. In general, avoid running Druid as root user. Consider creating a dedicated user account for running Druid.   
+that user. For example, the file browser UI will show console users the files that the underlying user can 
+access. In general, avoid running Druid as root user. Consider creating a dedicated user account for running Druid.  
 
-## Step 1. Install Druid
+## Install Druid
 
-After confirming the [requirements](#requirements), follow these steps: 
+Download the [{{DRUIDVERSION}} release](https://www.apache.org/dyn/closer.cgi?path=/druid/{{DRUIDVERSION}}/apache-druid-{{DRUIDVERSION}}-bin.tar.gz) from Apache Druid. 
+For this quickstart, you need Druid version 24.0 or higher.
+For versions earlier than 24.0 (0.23 and below), see [Load data with native batch ingestion](tutorial-batch-native.md).
 
-1. Download
-the [{{DRUIDVERSION}} release](https://www.apache.org/dyn/closer.cgi?path=/druid/{{DRUIDVERSION}}/apache-druid-{{DRUIDVERSION}}-bin.tar.gz).
-2. In your terminal, extract Druid and change directories to the distribution directory:
+In your terminal, extract the file and change directories to the distribution directory:
 
-   ```bash
-   tar -xzf apache-druid-{{DRUIDVERSION}}-bin.tar.gz
-   cd apache-druid-{{DRUIDVERSION}}
-   ```
-In the directory, you'll find `LICENSE` and `NOTICE` files and subdirectories for executable files, configuration files, sample data and more.
+```bash
+tar -xzf apache-druid-{{DRUIDVERSION}}-bin.tar.gz
+cd apache-druid-{{DRUIDVERSION}}
+```
+
+The distribution directory contains `LICENSE` and `NOTICE` files and subdirectories for executable files, configuration files, sample data and more.
 
-## Step 2. Start up Druid services
+## Start up Druid services
 
-Start up Druid services using the `micro-quickstart` single-machine configuration. 
+Start up Druid services using the `micro-quickstart` single-machine configuration.
+This configuration includes default settings that are appropriate for this tutorial, such as loading the `druid-multi-stage-query` extension by default so that you can use the MSQ task engine.
+
+You can view that setting and others in the configuration files in the `conf/druid/single-server/micro-quickstart/`. 
 
 From the apache-druid-{{DRUIDVERSION}} package root, run the following command:
 
@@ -95,173 +102,128 @@ $ ./bin/start-micro-quickstart
 ```
 
 All persistent state, such as the cluster metadata store and segments for the services, are kept in the `var` directory under 
-the Druid root directory, apache-druid-{{DRUIDVERSION}}. Each service writes to a log file under `var/sv`, as noted in the startup script output above.
-
-At any time, you can revert Druid to its original, post-installation state by deleting the entire `var` directory. You may
-want to do this, for example, between Druid tutorials or after experimentation, to start with a fresh instance. 
+the Druid root directory, apache-druid-{{DRUIDVERSION}}. Each service writes to a log file under `var/sv`.
 
-To stop Druid at any time, use CTRL-C in the terminal. This exits the `bin/start-micro-quickstart` script and 
-terminates all Druid processes. 
+At any time, you can revert Druid to its original, post-installation state by deleting the entire `var` directory. You may want to do this, for example, between Druid tutorials or after experimentation, to start with a fresh instance. 
 
+To stop Druid at any time, use CTRL+C in the terminal. This exits the `bin/start-micro-quickstart` script and terminates all Druid processes.
 
-## Step 3. Open the Druid console 
+## Open the Druid console 
 
 After the Druid services finish startup, open the [Druid console](../operations/druid-console.md) at [http://localhost:8888](http://localhost:8888). 
 
 ![Druid console](../assets/tutorial-quickstart-01.png "Druid console")
 
-It may take a few seconds for all Druid services to finish starting, including the [Druid router](../design/router.md), which serves the console. If you attempt to open the Druid console before startup is complete, you may see errors in the browser. Wait a few moments and try again. 
-
-
-## Step 4. Load data
+It may take a few seconds for all Druid services to finish starting, including the [Druid router](../design/router.md), which serves the console. If you attempt to open the Druid console before startup is complete, you may see errors in the browser. Wait a few moments and try again.
 
+In this quickstart, you use the the Druid console to perform ingestion. The MSQ task engine specifically uses the **Query** view to edit and run SQL queries.
+For a complete walkthrough of the **Query** view as it relates to the multi-stage query architecture and the MSQ task engine, see [UI walkthrough](../operations/druid-console.md).
 
-Ingestion specs define the schema of the data Druid reads and stores. You can write ingestion specs by hand or using the _data loader_, 
-as we'll do here to perform batch file loading with Druid's native batch ingestion.
+## Load data
 
-The Druid distribution bundles sample data we can use. The sample data located in `quickstart/tutorial/wikiticker-2015-09-12-sampled.json.gz` 
-in the Druid root directory represents Wikipedia page edits for a given day. 
+The Druid distribution bundles the `wikiticker-2015-09-12-sampled.json.gz` sample dataset that you can use for testing. The sample dataset is located in the `quickstart/tutorial/` folder, accessible from the Druid root directory, and represents Wikipedia page edits for a given day. 
 
-1. Click **Load data** from the Druid console header (![Load data](../assets/tutorial-batch-data-loader-00.png)).
+Follow these steps to load the sample Wikipedia dataset:
 
-2. Select the **Local disk** tile and then click **Connect data**.
-
-   ![Data loader init](../assets/tutorial-batch-data-loader-01.png "Data loader init")
-
-3. Enter the following values: 
+1. In the **Query** view, click **Connect external data**.
+2. Select the **Local disk** tile and enter the following values:
 
    - **Base directory**: `quickstart/tutorial/`
 
    - **File filter**: `wikiticker-2015-09-12-sampled.json.gz` 
 
-   ![Data location](../assets/tutorial-batch-data-loader-015.png "Data location")
+   ![Data location](../assets/tutorial-quickstart-02.png "Data location")
 
    Entering the base directory and [wildcard file filter](https://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter.html) separately, as afforded by the UI, allows you to specify multiple files for ingestion at once.
 
-4. Click **Apply**. 
-
-   The data loader displays the raw data, giving you a chance to verify that the data 
-   appears as expected. 
-
-   ![Data loader sample](../assets/tutorial-batch-data-loader-02.png "Data loader sample")
-
-   Notice that your position in the sequence of steps to load data, **Connect** in our case, appears at the top of the console, as shown below. 
-   You can click other steps to move forward or backward in the sequence at any time.
-   
-   ![Load data](../assets/tutorial-batch-data-loader-12.png)  
-   
-
-5. Click **Next: Parse data**. 
-
-   The data loader tries to determine the parser appropriate for the data format automatically. In this case 
-   it identifies the data format as `json`, as shown in the **Input format** field at the bottom right.
-
-   ![Data loader parse data](../assets/tutorial-batch-data-loader-03.png "Data loader parse data")
-
-   Feel free to select other **Input format** options to get a sense of their configuration settings 
-   and how Druid parses other types of data.  
-
-6. With the JSON parser selected, click **Next: Parse time**. The **Parse time** settings are where you view and adjust the 
-   primary timestamp column for the data.
-
-   ![Data loader parse time](../assets/tutorial-batch-data-loader-04.png "Data loader parse time")
-
+3. Click **Connect data**. 
+4. On the **Parse** page, you can examine the raw data and perform the following optional actions before loading data into Druid: 
+   - Expand a row to see the corresponding source data.
+   - Customize how the data is handled by selecting from the **Input format** options.
+   - Adjust the primary timestamp column for the data.
    Druid requires data to have a primary timestamp column (internally stored in a column called `__time`).
-   If you do not have a timestamp in your data, select `Constant value`. In our example, the data loader 
-   determines that the `time` column is the only candidate that can be used as the primary time column.
-
-7. Click **Next: Transform**, **Next: Filter**, and then **Next: Configure schema**, skipping a few steps.
-
-   You do not need to adjust transformation or filtering settings, as applying ingestion time transforms and 
-   filters are out of scope for this tutorial.
-
-8. The Configure schema settings are where you configure what [dimensions](../ingestion/data-model.md#dimensions) 
-   and [metrics](../ingestion/data-model.md#metrics) are ingested. The outcome of this configuration represents exactly how the 
-   data will appear in Druid after ingestion. 
-
-   Since our dataset is very small, you can turn off [rollup](../ingestion/rollup.md) 
-   by unsetting the **Rollup** switch and confirming the change when prompted.
-
-   ![Data loader schema](../assets/tutorial-batch-data-loader-05.png "Data loader schema")
-
-
-10. Click **Next: Partition** to configure how the data will be split into segments. In this case, choose `DAY` as 
-    the **Segment granularity**. 
-
-    ![Data loader partition](../assets/tutorial-batch-data-loader-06.png "Data loader partition")
-
-    Since this is a small dataset, we can have just a single segment, which is what selecting `DAY` as the 
-    segment granularity gives us. 
-
-11. Click **Next: Tune** and **Next: Publish**.
-
-12. The Publish settings are where you specify the datasource name in Druid. Let's change the default name from 
-`wikiticker-2015-09-12-sampled` to `wikipedia`. 
-
-    ![Data loader publish](../assets/tutorial-batch-data-loader-07.png "Data loader publish")
-
-
-13. Click **Next: Edit spec** to review the ingestion spec we've constructed with the data loader. 
-
-    ![Data loader spec](../assets/tutorial-batch-data-loader-08.png "Data loader spec")
-
-    Feel free to go back and change settings from previous steps to see how doing so updates the spec.
-    Similarly, you can edit the spec directly and see it reflected in the previous steps. 
-
-    > For other ways to load ingestion specs in Druid, see [Tutorial: Loading a file](./tutorial-batch.md). 
-
-14. Once you are satisfied with the spec, click **Submit**.
-
-    The new task for our wikipedia datasource now appears in the Ingestion view. 
-
-    ![Tasks view](../assets/tutorial-batch-data-loader-09.png "Tasks view")
-
-    The task may take a minute or two to complete. When done, the task status should be "SUCCESS", with
-    the duration of the task indicated. Note that the view is set to automatically 
-    refresh, so you do not need to refresh the browser to see the status change.
-
-    A successful task means that one or more segments have been built and are now picked up by our data servers.
-
-
-## Step 5. Query the data 
+   If your dataset doesn't have a timestamp, Druid uses the default value of `1970-01-01 00:00:00`.
+
+   ![Data sample](../assets/tutorial-quickstart-03.png "Data sample")
+
+5. Click **Done**. You're returned to the **Query** view that displays the newly generated query.
+   The query inserts the sample data into the table named `wikiticker-2015-09-12-sampled`.
+
+   <details><summary>Show the query</summary>
+
+   ```sql
+   REPLACE INTO "wikiticker-2015-09-12-sampled" OVERWRITE ALL
+   WITH input_data AS (SELECT *
+   FROM TABLE(
+     EXTERN(
+       '{"type":"local","baseDir":"quickstart/tutorial/","filter":"wikiticker-2015-09-12-sampled.json.gz"}',
+       '{"type":"json"}',
+       '[{"name":"time","type":"string"},{"name":"channel","type":"string"},{"name":"cityName","type":"string"},{"name":"comment","type":"string"},{"name":"countryIsoCode","type":"string"},{"name":"countryName","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":"long"},{"name":"namespace","type":"string"},{"name":" [...]
+        )
+      ))
+   SELECT
+     TIME_PARSE("time") AS __time,
+     channel,
+     cityName,
+     comment,
+     countryIsoCode,
+     countryName,
+     isAnonymous,
+     isMinor,
+     isNew,
+     isRobot,
+     isUnpatrolled,
+     metroCode,
+     namespace,
+     page,
+     regionIsoCode,
+     regionName,
+     user,
+     delta,
+     added,
+     deleted
+   FROM input_data
+   PARTITIONED BY DAY
+   ```
+   </details>
 
-You can now see the data as a datasource in the console and try out a query, as follows: 
+6. Optionally, click **Preview** to see the general shape of the data before you ingest it.  
+7. Click **Run** to execute the query. The task may take a minute or two to complete. When done, the task displays its duration and the number of rows inserted into the table. The view is set to automatically refresh, so you don't need to refresh the browser to see the status change.
 
-1. Click **Datasources** from the console header. 
-  
-   If the wikipedia datasource doesn't appear, wait a few moments for the segment to finish loading. A datasource is 
-   queryable once it is shown to be "Fully available" in the **Availability** column. 
+    ![Run query](../assets/tutorial-quickstart-04.png "Run query")
 
-2. When the datasource is available, open the Actions menu (![Actions](../assets/datasources-action-button.png)) for that 
-   datasource and choose **Query with SQL**.
+   A successful task means that Druid data servers have picked up one or more segments.
 
-   ![Datasource view](../assets/tutorial-batch-data-loader-10.png "Datasource view")
+## Query data
 
-   > Notice the other actions you can perform for a datasource, including configuring retention rules, compaction, and more. 
+Once the ingestion job is complete, you can query the data. 
 
-3. Run the prepopulated query, `SELECT * FROM "wikipedia"` to see the results.
+In the **Query** view, run the following query to produce a list of top channels:
 
-   ![Query view](../assets/tutorial-batch-data-loader-11.png "Query view")
+```sql
+SELECT
+  channel,
+  COUNT(*)
+FROM "wikiticker-2015-09-12-sampled"
+GROUP BY channel
+ORDER BY COUNT(*) DESC
+```
 
-Congratulations! You've gone from downloading Druid to querying data in just one quickstart. See the following
-section for what to do next. 
+![Query view](../assets/tutorial-quickstart-05.png "Query view")
 
+Congratulations! You've gone from downloading Druid to querying data with the MSQ task engine in just one quickstart.
 
 ## Next steps
 
-After finishing the quickstart, check out the [query tutorial](../tutorials/tutorial-query.md) to further explore 
-Query features in the Druid console. 
-
-Alternatively, learn about other ways to ingest data in one of these tutorials: 
-
-- [Loading stream data from Apache Kafka](./tutorial-kafka.md) – How to load streaming data from a Kafka topic.
-- [Loading a file using Apache Hadoop](./tutorial-batch-hadoop.md) – How to perform a batch file load, using a remote Hadoop cluster.
-- [Writing your own ingestion spec](./tutorial-ingestion-spec.md) – How to write a new ingestion spec and use it to load data.
-
-
-Remember that after stopping Druid services, you can start clean next time by deleting the `var` directory from the Druid root directory and 
-running the `bin/start-micro-quickstart` script again. You will likely want to do this before taking other data ingestion tutorials, 
-since in them you will create the same wikipedia datasource. 
-
+See the following topics for more information:
 
+* [Extensions](../development/extensions.md) for details on Druid extensions.
+* [MSQ task engine query syntax](../multi-stage-query/index.md#msq-task-engine-query-syntax) to further explore queries for SQL-based ingestion.
+* [Druid SQL overview](../querying/sql.md) to learn about how to query data you ingest.
+* [Load data with native batch ingestion](tutorial-batch-native.md) to load and query data with Druid's native batch ingestion feature.
+* [Load stream data from Apache Kafka](./tutorial-kafka.md) to load streaming data from a Kafka topic.
+* [API](../multi-stage-query/msq-api.md) to submit query tasks to the MSQ task engine programmatically.
+* [Connect external data](../multi-stage-query/msq-tutorial-connect-external-data.md) to learn how to generate a query that references externally hosted data that the MSQ task engine can use to ingest data.
+* [Convert ingestion spec](../multi-stage-query/msq-tutorial-convert-ingest-spec.md) to learn how to convert an existing JSON ingestion spec to a SQL query that the MSQ task engine can use to ingest data.
 
+Remember that after stopping Druid services, you can start clean next time by deleting the `var` directory from the Druid root directory and running the `bin/start-micro-quickstart` script again. You may want to do this before taking other data ingestion tutorials, since they use the same Wikipedia datasource.
\ No newline at end of file
diff --git a/docs/tutorials/tutorial-batch-native.md b/docs/tutorials/tutorial-batch-native.md
new file mode 100644
index 0000000000..19e27dca26
--- /dev/null
+++ b/docs/tutorials/tutorial-batch-native.md
@@ -0,0 +1,155 @@
+---
+id: tutorial-batch-native
+title: "Load data with native batch ingestion"
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+
+This topic shows you how to load and query data files in Apache Druid using its native batch ingestion feature. 
+
+## Prerequisites
+
+Install Druid, start up Druid services, and open the Druid console as described in the [Druid quickstart](index.md).
+
+## Load data
+
+Ingestion specs define the schema of the data Druid reads and stores. You can write ingestion specs by hand or using the _data loader_, 
+as we'll do here to perform batch file loading with Druid's native batch ingestion.
+
+The Druid distribution bundles sample data we can use. The sample data located in `quickstart/tutorial/wikiticker-2015-09-12-sampled.json.gz` 
+in the Druid root directory represents Wikipedia page edits for a given day. 
+
+1. Click **Load data** from the Druid console header (![Load data](../assets/tutorial-batch-data-loader-00.png)).
+
+2. Select the **Local disk** tile and then click **Connect data**.
+
+   ![Data loader init](../assets/tutorial-batch-data-loader-01.png "Data loader init")
+
+3. Enter the following values: 
+
+   - **Base directory**: `quickstart/tutorial/`
+
+   - **File filter**: `wikiticker-2015-09-12-sampled.json.gz` 
+
+   ![Data location](../assets/tutorial-batch-data-loader-015.png "Data location")
+
+   Entering the base directory and [wildcard file filter](https://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter.html) separately, as afforded by the UI, allows you to specify multiple files for ingestion at once.
+
+4. Click **Apply**. 
+
+   The data loader displays the raw data, giving you a chance to verify that the data 
+   appears as expected. 
+
+   ![Data loader sample](../assets/tutorial-batch-data-loader-02.png "Data loader sample")
+
+   Notice that your position in the sequence of steps to load data, **Connect** in our case, appears at the top of the console, as shown below. 
+   You can click other steps to move forward or backward in the sequence at any time.
+
+   ![Load data](../assets/tutorial-batch-data-loader-12.png)  
+
+
+5. Click **Next: Parse data**. 
+
+   The data loader tries to determine the parser appropriate for the data format automatically. In this case 
+   it identifies the data format as `json`, as shown in the **Input format** field at the bottom right.
+
+   ![Data loader parse data](../assets/tutorial-batch-data-loader-03.png "Data loader parse data")
+
+   Feel free to select other **Input format** options to get a sense of their configuration settings 
+   and how Druid parses other types of data.  
+
+6. With the JSON parser selected, click **Next: Parse time**. The **Parse time** settings are where you view and adjust the 
+   primary timestamp column for the data.
+
+   ![Data loader parse time](../assets/tutorial-batch-data-loader-04.png "Data loader parse time")
+
+   Druid requires data to have a primary timestamp column (internally stored in a column called `__time`).
+   If you do not have a timestamp in your data, select `Constant value`. In our example, the data loader 
+   determines that the `time` column is the only candidate that can be used as the primary time column.
+
+7. Click **Next: Transform**, **Next: Filter**, and then **Next: Configure schema**, skipping a few steps.
+
+   You do not need to adjust transformation or filtering settings, as applying ingestion time transforms and 
+   filters are out of scope for this tutorial.
+
+8. The Configure schema settings are where you configure what [dimensions](../ingestion/data-model.md#dimensions) 
+   and [metrics](../ingestion/data-model.md#metrics) are ingested. The outcome of this configuration represents exactly how the 
+   data will appear in Druid after ingestion. 
+
+   Since our dataset is very small, you can turn off [rollup](../ingestion/rollup.md) 
+   by unsetting the **Rollup** switch and confirming the change when prompted.
+
+   ![Data loader schema](../assets/tutorial-batch-data-loader-05.png "Data loader schema")
+
+
+9. Click **Next: Partition** to configure how the data will be split into segments. In this case, choose `DAY` as the **Segment granularity**. 
+
+    ![Data loader partition](../assets/tutorial-batch-data-loader-06.png "Data loader partition")
+
+    Since this is a small dataset, we can have just a single segment, which is what selecting `DAY` as the 
+    segment granularity gives us. 
+
+10. Click **Next: Tune** and **Next: Publish**.
+
+11. The Publish settings are where you specify the datasource name in Druid. Let's change the default name from  `wikiticker-2015-09-12-sampled` to `wikipedia`. 
+
+    ![Data loader publish](../assets/tutorial-batch-data-loader-07.png "Data loader publish")
+
+12. Click **Next: Edit spec** to review the ingestion spec we've constructed with the data loader. 
+
+    ![Data loader spec](../assets/tutorial-batch-data-loader-08.png "Data loader spec")
+
+    Feel free to go back and change settings from previous steps to see how doing so updates the spec.
+    Similarly, you can edit the spec directly and see it reflected in the previous steps. 
+
+    For other ways to load ingestion specs in Druid, see [Tutorial: Loading a file](./tutorial-batch.md). 
+13. Once you are satisfied with the spec, click **Submit**.
+
+
+    The new task for our wikipedia datasource now appears in the Ingestion view. 
+
+    ![Tasks view](../assets/tutorial-batch-data-loader-09.png "Tasks view")
+
+    The task may take a minute or two to complete. When done, the task status should be "SUCCESS", with
+    the duration of the task indicated. Note that the view is set to automatically 
+    refresh, so you do not need to refresh the browser to see the status change.
+
+    A successful task means that one or more segments have been built and are now picked up by our data servers.
+
+
+## Query the data 
+
+You can now see the data as a datasource in the console and try out a query, as follows: 
+
+1. Click **Datasources** from the console header. 
+
+   If the wikipedia datasource doesn't appear, wait a few moments for the segment to finish loading. A datasource is 
+   queryable once it is shown to be "Fully available" in the **Availability** column. 
+
+2. When the datasource is available, open the Actions menu (![Actions](../assets/datasources-action-button.png)) for that 
+   datasource and choose **Query with SQL**.
+
+   ![Datasource view](../assets/tutorial-batch-data-loader-10.png "Datasource view")
+
+   > Notice the other actions you can perform for a datasource, including configuring retention rules, compaction, and more. 
+3. Run the prepopulated query, `SELECT * FROM "wikipedia"` to see the results.
+
+   ![Query view](../assets/tutorial-batch-data-loader-11.png "Query view")
diff --git a/docs/tutorials/tutorial-msq-connect-extern.md b/docs/tutorials/tutorial-msq-connect-extern.md
new file mode 100644
index 0000000000..7d548b8874
--- /dev/null
+++ b/docs/tutorials/tutorial-msq-connect-extern.md
@@ -0,0 +1,44 @@
+---
+id: tutorial-msq-external-data
+title: "Connect external data"
+sidebar_label: "Connect external data"
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+<!DOCTYPE html>
+<!--This redirects to the Multi-Stage Query tutorial. This redirect file exists cause duplicate entries in the left nav aren't allowed-->
+<html lang="en-US">
+  <head>
+    <meta charset="UTF-8" />
+    <meta
+      http-equiv="refresh"
+      content="0; url=/docs/multi-stage-query/connect-external-data.html"
+    />
+    <script type="text/javascript">
+      window.location.href = '/docs/multi-stage-query/connect-external-data.html';
+    </script>
+    <title>About the Druid documentation</title>
+  </head>
+  <body>
+    If you are not redirected automatically, follow this
+    <a href="/docs/multi-stage-query/connect-external-data.html">link</a>.
+  </body>
+</html>
\ No newline at end of file
diff --git a/docs/tutorials/tutorial-msq-convert-json.md b/docs/tutorials/tutorial-msq-convert-json.md
new file mode 100644
index 0000000000..9353edb24f
--- /dev/null
+++ b/docs/tutorials/tutorial-msq-convert-json.md
@@ -0,0 +1,44 @@
+---
+id: tutorial-msq-convert-json
+title: "Convert JSON ingestion spec to SQL"
+sidebar_label: "Convert JSON ingestion spec"
+---
+
+<!--
+  ~ Licensed to the Apache Software Foundation (ASF) under one
+  ~ or more contributor license agreements.  See the NOTICE file
+  ~ distributed with this work for additional information
+  ~ regarding copyright ownership.  The ASF licenses this file
+  ~ to you under the Apache License, Version 2.0 (the
+  ~ "License"); you may not use this file except in compliance
+  ~ with the License.  You may obtain a copy of the License at
+  ~
+  ~   http://www.apache.org/licenses/LICENSE-2.0
+  ~
+  ~ Unless required by applicable law or agreed to in writing,
+  ~ software distributed under the License is distributed on an
+  ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~ KIND, either express or implied.  See the License for the
+  ~ specific language governing permissions and limitations
+  ~ under the License.
+  -->
+
+<!DOCTYPE html>
+<!--This redirects to the Multi-Stage Query tutorial. This redirect file exists cause duplicate entries in the left nav aren't allowed-->
+<html lang="en-US">
+  <head>
+    <meta charset="UTF-8" />
+    <meta
+      http-equiv="refresh"
+      content="0; url=/docs/multi-stage-query/convert-json-spec.html"
+    />
+    <script type="text/javascript">
+      window.location.href = '/docs/multi-stage-query/convert-json-spec.html';
+    </script>
+    <title>About the Druid documentation</title>
+  </head>
+  <body>
+    If you are not redirected automatically, follow this
+    <a href="/docs/multi-stage-query/convert-json-spec.html">link</a>.
+  </body>
+</html>
\ No newline at end of file
diff --git a/website/.spelling b/website/.spelling
index 2482bafbff..20565e9363 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -18,6 +18,7 @@
 # global dictionary is at the start, file overrides afterwards
 # one word per line, to define a file override use ' - filename'
 # where filename is relative to this configuration file
+1M
 32-bit
 500MiB
 64-bit
@@ -38,6 +39,7 @@ BCP
 Base64
 Base64-encoded
 ByteBuffer
+bottlenecked
 concat
 CIDR
 CORS
@@ -60,6 +62,7 @@ DimensionSpec
 DimensionSpecs
 Dockerfile
 DogStatsD
+DOCTYPE
 Double.NEGATIVE_INFINITY
 Double.NEGATIVE_INFINITY.
 Double.POSITIVE_INFINITY
@@ -95,6 +98,7 @@ HDFSFirehose
 HLL
 HashSet
 Homebrew
+html
 HyperLogLog
 IAM
 IANA
@@ -153,6 +157,7 @@ MariaDB
 MiddleManager
 MiddleManagers
 Montréal
+MSQ
 Murmur3
 MVCC
 NFS
@@ -301,6 +306,7 @@ https
 idempotency
 i.e.
 influxdb
+ingestions
 ingestionSpec
 injective
 inlined
@@ -330,6 +336,7 @@ lookback
 lookups
 mapreduce
 masse
+maxBytes
 maxNumericInFilters
 maxNumFiles
 maxNumSegments
@@ -349,6 +356,7 @@ namespaced
 namespaces
 natively
 netflow
+nonfinalized
 non-nullable
 noop
 numerics
@@ -468,6 +476,8 @@ unioned
 unmergeable
 unmerged
 UNNEST
+unnest
+unnests
 unparseable
 unparsed
 unsetting
@@ -490,6 +500,7 @@ vectorizeVirtualColumns
 versioning
 virtualColumns
 w.r.t.
+walkthrough
 whitelist
 whitelisted
 whitespace
@@ -527,6 +538,8 @@ NUMERIC_PRECISION_RADIX
 NUMERIC_SCALE
 ORDINAL_POSITION
 POSIX
+P1M
+P1Y
 PT1M
 PT5M
 SCHEMA_NAME
@@ -581,6 +594,7 @@ segment_id
 server_type
 shard_spec
 sqlTimeZone
+sql-msq-task
 supervisor_id
 sys
 sys.segments
@@ -592,10 +606,96 @@ useApproximateCountDistinct
 useGroupingSetForExactDistinct
 useApproximateTopN
 wikipedia
+your-table
 enableTimeBoundaryPlanning
 TimeBoundary
 druid.query.default.context.enableTimeBoundaryPlanning
 IEC
+# MSQ general
+SegmentGenerator
+granularity_string
+QueryKit
+# MSQ report fields
+taskId
+multiStageQuery.taskId
+multiStageQuery.payload.status
+multiStageQuery.payload.status.status
+multiStageQuery.payload.status.startTime 
+multiStageQuery.payload.status.durationMs
+multiStageQuery.payload.status.errorReport
+multiStageQuery.payload.status.errorReport.taskId
+multiStageQuery.payload.status.errorReport.host
+multiStageQuery.payload.status.errorReport.stageNumber
+multiStageQuery.payload.status.errorReport.error
+multiStageQuery.payload.status.errorReport.error.errorCode
+multiStageQuery.payload.status.errorReport.error.errorMessage
+multiStageQuery.payload.status.errorReport.exceptionStackTrace
+multiStageQuery.payload.stages stages
+multiStageQuery.payload.stages[].stageNumber
+definition.id
+definition.input
+definition.broadcast
+definition.processor
+definition.signature
+stageNumber
+startTime
+multiStageQuery.payload.stages
+READING_INPUT
+POST_READING
+RESULTS_COMPLETE
+workerCount
+partitionCount
+startCount
+# MSQ errors and limits
+BroadcastTablesTooLarge  
+CannotParseExternalData 
+ColumnNameRestricted
+ColumnTypeNotSupported
+DurableStorageConfiguration  
+ColumnTypeNotSupported 
+InsertCannotAllocateSegment
+InsertCannotBeEmpty 
+InsertCannotOrderByDescending  
+InsertCannotReplaceExistingSegment 
+InsertLockPreempted  
+InsertTimeNull
+CURRENT_TIMESTAMP  
+InsertTimeOutOfBounds
+UnknownError
+TaskStartTimeout
+OutOfMemoryError
+SegmentGenerator
+maxFrameSize
+InvalidNullByte  
+QueryNotSupported   
+QueryNotSupported   
+RowTooLarge  
+TooManyBuckets  
+TooManyInputFiles 
+TooManyPartitions   
+TooManyColumns 
+TooManyWarnings 
+TooManyWorkers 
+NotEnoughMemory  
+WorkerFailed  
+WorkerRpcFailed  
+# MSQ context parameters
+maxNumTasks 
+taskAssignment 
+finalizeAggregations 
+rowsInMemory 
+segmentSortOrder 
+rowsPerSegment 
+durableShuffleStorage
+# Aggregations
+groupByEnableMultiValueUnnesting
+APPROX_COUNT_DISTINCT_DS_HLL
+APPROX_COUNT_DISTINCT_DS_THETA
+APPROX_QUANTILE_DS
+DS_QUANTILES_SKETCH
+APPROX_QUANTILE_FIXED_BUCKETS
+
+# File specific overrides
  - ../docs/comparisons/druid-vs-elasticsearch.md
 100x
  - ../docs/configuration/logging.md
diff --git a/website/sidebars.json b/website/sidebars.json
index c414181d61..c69ae15e34 100644
--- a/website/sidebars.json
+++ b/website/sidebars.json
@@ -13,14 +13,16 @@
       "tutorials/tutorial-batch-hadoop",
       "tutorials/tutorial-query",
       "tutorials/tutorial-rollup",
-      "tutorials/tutorial-sketches-theta",
+      "tutorials/tutorial-sketches-theta",      
       "tutorials/tutorial-retention",
       "tutorials/tutorial-update-data",
       "tutorials/tutorial-compaction",
       "tutorials/tutorial-delete-data",
       "tutorials/tutorial-ingestion-spec",
       "tutorials/tutorial-transform-spec",
-      "tutorials/tutorial-kerberos-hadoop"
+      "tutorials/tutorial-kerberos-hadoop",
+      "tutorials/tutorial-msq-external-data",
+      "tutorials/tutorial-msq-convert-json"
     ],
     "Design": [
       "design/architecture",
@@ -66,6 +68,17 @@
       "ingestion/tasks",
       "ingestion/faq"
     ],
+  "SQL-based ingestion": [
+      "multi-stage-query/index",
+      "multi-stage-query/concepts",
+      "multi-stage-query/connect-external-data",
+      "multi-stage-query/convert-json-spec",
+      "multi-stage-query/examples",
+      "multi-stage-query/api",
+      "multi-stage-query/security",
+      "multi-stage-query/reference",
+      "multi-stage-query/known-issues"
+    ],
     "Querying": [
       {
         "type": "subcategory",
@@ -102,7 +115,7 @@
           "querying/query-context"
         ]
       },
-      
+
       {
         "type": "subcategory",
         "label": "Native query types",
@@ -158,7 +171,7 @@
           "operations/tls-support"
 
         ]
-      }, 
+      },
       {
         "type": "subcategory",
         "label": "Performance tuning",
@@ -169,7 +182,7 @@
           "operations/http-compression",
           "operations/clean-metadata-store"
         ]
-      },       
+      },
       {
         "type": "subcategory",
         "label": "Monitoring",
@@ -178,7 +191,7 @@
           "operations/metrics",
           "operations/alerts"
         ]
-      },       
+      },
       "operations/api-reference",
       "operations/high-availability",
       "operations/rolling-updates",


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