You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/03/13 00:44:08 UTC

[GitHub] [druid] maytasm commented on a change in pull request #10935: First refactor of compaction

maytasm commented on a change in pull request #10935:
URL: https://github.com/apache/druid/pull/10935#discussion_r592796705



##########
File path: docs/configuration/index.md
##########
@@ -842,24 +842,24 @@ A description of the compaction config is:
 |`taskPriority`|[Priority](../ingestion/tasks.md#priority) of compaction task.|no (default = 25)|
 |`inputSegmentSizeBytes`|Maximum number of total segment bytes processed per compaction task. Since a time chunk must be processed in its entirety, if the segments for a particular time chunk have a total size in bytes greater than this parameter, compaction will not run for that time chunk. Because each compaction task runs with a single thread, setting this value too far above 1–2GB will result in compaction tasks taking an excessive amount of time.|no (default = 419430400)|
 |`maxRowsPerSegment`|Max number of rows per segment after compaction.|no|
-|`skipOffsetFromLatest`|The offset for searching segments to be compacted. Strongly recommended to set for realtime dataSources. |no (default = "P1D")|
+|`skipOffsetFromLatest`|The offset for searching segments to be compacted in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Strongly recommended to set for realtime dataSources. See [Data handling with compaction](../ingestion/compaction.md#data-handling-with-compaction)|no (default = "P1D")|
 |`tuningConfig`|Tuning config for compaction tasks. See below [Compaction Task TuningConfig](#compaction-tuningconfig).|no|
 |`taskContext`|[Task context](../ingestion/tasks.md#context) for compaction tasks.|no|
+|`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` for the compacted segments.|No|

Review comment:
       Is there a section describing the keys (and what the keys are for) of this custom `granularitySpec`? Would be good to explicitly point out what is and isn't supported compare to a regular `granularitySpec`

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,207 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+
+Compaction in Apache Druid is a strategy to optimize segment size. Compaction tasks read an existing set of segments for a given time range and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Compaction can sometimes increase performance because it reduces the number of segments and, consequently, the per-segment processing and the memory overhead required for ingestion and for querying paths.
+
+As a strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. For example if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction. See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your case.
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- Compacting recent data before older data suboptimal is suboptimal in your environment.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.

Review comment:
       Actually I think @techdocsmith clarification here is nice and clear.
   1. segment 1 with DAY granularity interval of 01/01/2020-01/02/2020, segment 2 with MONTH granularity interval of 01/01/2020-02/01/2020. The two segments overlapped.
   2. Druid attempt to combine and compacted the overlapped segments. For the case above, the earliest start time of the two segments above is 01/01/2020 and the latest end time of the two segments above is 02/01/2020. Druid will then compacted these two segments together despite they have different segment granularity. 
   3. For the example above, since the earliest start time of the two segments above is 01/01/2020 and  the latest end time of the two segments above is 02/01/2020, Druid will determine that the segmentGranularity it should use is MONTH although segment 1's original segmentGranularity is DAY

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.

Review comment:
       This can goes both ways. Too larger to smaller and too small to larger. 

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.
+
+### Query granularity handling
+
+Unless you modify the query granularity in the [granularity spec](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
+
+> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+
+If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
+
+### Dimension handling
+Apache Druid supports schema changes. Therefore, dimensions can be different across segments even if they are a part of the same data source. See [Different schemas among segments](../design/segments.md#different-schemas-among-segments). If the input segments have different dimensions, the resulting compacted segment includes all dimensions of the input segments. 
+
+Even when the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. The dimensions of recent segments precede that of old segments in terms of data types and the ordering because more recent segments are more likely to have the preferred order and data types.
+
+If you want to use your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec.

Review comment:
       I would say "If you want to force the ordering and types" or "If you want to ensure ordering and types is of a certain values" instead

##########
File path: docs/ingestion/data-management.md
##########
@@ -21,173 +21,9 @@ title: "Data management"
   ~ specific language governing permissions and limitations
   ~ under the License.
   -->
+Within the context of this topic Data management refers to Apache Druid's data maintenance capabilities for existing datasources. There are several options to help you keep your data relevant and to help your Druid cluster remain performant. For example updating, reingesting, adding lookups, reindexing, or deleting data.
 
-
-
-
-## Schema changes
-
-Schemas for datasources can change at any time and Apache Druid supports different schemas among segments.
-
-### Replacing segments

Review comment:
       Why is this section removed? 

##########
File path: docs/ingestion/data-management.md
##########
@@ -21,173 +21,9 @@ title: "Data management"
   ~ specific language governing permissions and limitations
   ~ under the License.
   -->
+Within the context of this topic Data management refers to Apache Druid's data maintenance capabilities for existing datasources. There are several options to help you keep your data relevant and to help your Druid cluster remain performant. For example updating, reingesting, adding lookups, reindexing, or deleting data.
 
-
-
-
-## Schema changes

Review comment:
       Why is this section removed? 

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.

Review comment:
       You can also change the priority for compaction task so that the compaction task supersedes the ingestion task

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.
+
+### Query granularity handling
+
+Unless you modify the query granularity in the [granularity spec](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
+
+> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+
+If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
+
+### Dimension handling
+Apache Druid supports schema changes. Therefore, dimensions can be different across segments even if they are a part of the same data source. See [Different schemas among segments](../design/segments.md#different-schemas-among-segments). If the input segments have different dimensions, the resulting compacted segment includes all dimensions of the input segments. 
+
+Even when the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. The dimensions of recent segments precede that of old segments in terms of data types and the ordering because more recent segments are more likely to have the preferred order and data types.
+
+If you want to use your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec.
+
+### Rollup
+Druid only rolls up the output segment when `rollup` is set for all input segments. See [Roll-up](../ingestion/index.md#rollup) for more details.
+You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.md#analysistypes).
+
+## Setting up manual compaction
+
+To perform a manual compaction, you submit a compaction task. Compaction tasks merge all segments for the defined interval according to the following syntax:
+
+```json
+{
+    "type": "compact",
+    "id": <task_id>,
+    "dataSource": <task_datasource>,
+    "ioConfig": <IO config>,
+    "dimensionsSpec" <custom dimensionsSpec>,
+    "metricsSpec" <custom metricsSpec>,
+    "tuningConfig" <parallel indexing task tuningConfig>,
+    "context": <task context>
+}
+```
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `compact`|Yes|
+|`id`|Task id|No|
+|`dataSource`|Data source name to compact|Yes|
+|`ioConfig`|I/O configuration for compaction task. See [Compaction I/O configuration](#compaction-io-configuration) for details.|Yes|
+|`dimensionsSpec`|Custom dimensions spec. The compaction task uses the specified dimensions spec if it exists instead of generating one.|No|
+|`metricsSpec`|Custom metrics spec. The compaction task uses the specified metrics spec rather than generating one.|No|
+|`segmentGranularity`|When set, the compaction task changes the segment granularity for the given interval.  Deprecated. Use `granularitySpec`. |No.|
+|`tuningConfig`|[Parallel indexing task tuningConfig](./native-batch.md#tuningconfig)|No|
+|`context`|[Task context](./tasks.md#context)|No|
+|`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` and `queryGranularity` for the compacted segments. See [Compaction granularitySpec](#compaction-granularity-spec).|No|
+
+To control the number of result segments per time chunk, you can set [maxRowsPerSegment](../configuration/index.md#compaction-dynamic-configuration) or [numShards](../ingestion/native-batch.md#tuningconfig).
+
+> You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals.
+
+A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [DruidInputSource](../native-batch.md#druid-input-source), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default.
+
+Compaction tasks would exit without doing anything and issue a failure status code:
+- if the interval you specify has no data segments loaded<br>
+OR
+- if the interval you specify is empty.
+
+Note that the metadata between input segments and the resulting compacted segments may differ if the metadata among the input segments differs as well. If all input segments have the same metadata, however, the resulting output segment will have the same metadata as all input segments.
+
+
+### Example compaction task
+The following JSON illustrates a compaction task to compact _all segments_ within the interval `2017-01-01/2018-01-01` and create new segments:
+
+```json
+{
+  "type" : "compact",
+  "dataSource" : "wikipedia",
+  "ioConfig" : {
+    "type": "compact",
+    "inputSpec": {
+      "type": "interval",
+      "interval": "2017-01-01/2018-01-01",
+    }
+  }
+}
+```
+
+This task doesn't specify a `granularitySpec` so Druid retains the original segment granularity unchanged when compaction is complete.
+
+### Compaction I/O configuration
+
+The compaction `ioConfig` requires specifying `inputSpec` as follows:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `compact`|Yes|
+|`inputSpec`|Input specification|Yes|
+
+There are two supported `inputSpec`s for now.
+
+The interval `inputSpec` is:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `interval`|Yes|
+|`interval`|Interval to compact|Yes|
+
+The segments `inputSpec` is:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `segments`|Yes|
+|`segments`|A list of segment IDs|Yes|
+
+### Compaction granularity spec
+
+You can optionally use the `granularitySpec` object to configure the segment granularity and the query granularity of the compacted segments. Their syntax is as follows:
+```json
+    "type": "compact",
+    "id": <task_id>,
+    "dataSource": <task_datasource>,
+    ...
+    ,
+    "granularitySpec": {
+      "segmentGranularity": <time_period>,
+      "queryGranularity": <time_period>
+    }
+    ...
+```
+
+`granularitySpec` takes the following keys:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`segmentGranularity`|Time chunking period for the segment granularity. Defaults to 'null'. Accepts all [Query granularities](../querying/granularities.md).|No|

Review comment:
       Maybe explain the behavior of null here too

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.

Review comment:
       The speed of a compaction task submitted manually and via auto compaction completing will be the same. I think what we need to make clear here is that automatic compaction has an assigned number of task slot it can used. If this is reached, then it will wait until previously submitted auto compaction tasks are done. Manual compaction can use all available task slots and you can get "faster" by manually submitting more tasks (for more intervals) concurrently. 

##########
File path: docs/configuration/index.md
##########
@@ -842,24 +842,24 @@ A description of the compaction config is:
 |`taskPriority`|[Priority](../ingestion/tasks.md#priority) of compaction task.|no (default = 25)|
 |`inputSegmentSizeBytes`|Maximum number of total segment bytes processed per compaction task. Since a time chunk must be processed in its entirety, if the segments for a particular time chunk have a total size in bytes greater than this parameter, compaction will not run for that time chunk. Because each compaction task runs with a single thread, setting this value too far above 1–2GB will result in compaction tasks taking an excessive amount of time.|no (default = 419430400)|
 |`maxRowsPerSegment`|Max number of rows per segment after compaction.|no|
-|`skipOffsetFromLatest`|The offset for searching segments to be compacted. Strongly recommended to set for realtime dataSources. |no (default = "P1D")|
+|`skipOffsetFromLatest`|The offset for searching segments to be compacted in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Strongly recommended to set for realtime dataSources. See [Data handling with compaction](../ingestion/compaction.md#data-handling-with-compaction)|no (default = "P1D")|
 |`tuningConfig`|Tuning config for compaction tasks. See below [Compaction Task TuningConfig](#compaction-tuningconfig).|no|
 |`taskContext`|[Task context](../ingestion/tasks.md#context) for compaction tasks.|no|
+|`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` for the compacted segments.|No|
 
 An example of compaction config is:
 
 ```json
 {
-  "dataSource": "wikiticker"
+  "dataSource": "wikiticker",

Review comment:
       Not have to be in this PR, but we should have a separate page dedicated to auto-compaction. This will allow us to go into more detail on why/how to use auto-compaction.
   
   

##########
File path: docs/ingestion/data-management.md
##########
@@ -21,173 +21,9 @@ title: "Data management"
   ~ specific language governing permissions and limitations
   ~ under the License.
   -->
+Within the context of this topic Data management refers to Apache Druid's data maintenance capabilities for existing datasources. There are several options to help you keep your data relevant and to help your Druid cluster remain performant. For example updating, reingesting, adding lookups, reindexing, or deleting data.
 
-
-
-
-## Schema changes
-
-Schemas for datasources can change at any time and Apache Druid supports different schemas among segments.
-
-### Replacing segments
-
-Druid uniquely
-identifies segments using the datasource, interval, version, and partition number. The partition number is only visible in the segment id if
-there are multiple segments created for some granularity of time. For example, if you have hourly segments, but you
-have more data in an hour than a single segment can hold, you can create multiple segments for the same hour. These segments will share
-the same datasource, interval, and version, but have linearly increasing partition numbers.
-
-```
-foo_2015-01-01/2015-01-02_v1_0
-foo_2015-01-01/2015-01-02_v1_1
-foo_2015-01-01/2015-01-02_v1_2
-```
-
-In the example segments above, the dataSource = foo, interval = 2015-01-01/2015-01-02, version = v1, partitionNum = 0.
-If at some later point in time, you reindex the data with a new schema, the newly created segments will have a higher version id.
-
-```
-foo_2015-01-01/2015-01-02_v2_0
-foo_2015-01-01/2015-01-02_v2_1
-foo_2015-01-01/2015-01-02_v2_2
-```
-
-Druid batch indexing (either Hadoop-based or IndexTask-based) guarantees atomic updates on an interval-by-interval basis.
-In our example, until all `v2` segments for `2015-01-01/2015-01-02` are loaded in a Druid cluster, queries exclusively use `v1` segments.
-Once all `v2` segments are loaded and queryable, all queries ignore `v1` segments and switch to the `v2` segments.
-Shortly afterwards, the `v1` segments are unloaded from the cluster.
-
-Note that updates that span multiple segment intervals are only atomic within each interval. They are not atomic across the entire update.
-For example, you have segments such as the following:
-
-```
-foo_2015-01-01/2015-01-02_v1_0
-foo_2015-01-02/2015-01-03_v1_1
-foo_2015-01-03/2015-01-04_v1_2
-```
-
-`v2` segments will be loaded into the cluster as soon as they are built and replace `v1` segments for the period of time the
-segments overlap. Before v2 segments are completely loaded, your cluster may have a mixture of `v1` and `v2` segments.
-
-```
-foo_2015-01-01/2015-01-02_v1_0
-foo_2015-01-02/2015-01-03_v2_1
-foo_2015-01-03/2015-01-04_v1_2
-```
-
-In this case, queries may hit a mixture of `v1` and `v2` segments.
-
-### Different schemas among segments

Review comment:
       Why is this section removed? 

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 

Review comment:
       "When segments can benefit from compaction" -> is a little vague. Auto compaction looks for segments that is either not compacted or was compacted with a different spec (there was a change in spec i.e. change in partitioning) and submit compaction task for those and only those. 

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.
+
+### Query granularity handling
+
+Unless you modify the query granularity in the [granularity spec](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
+
+> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+
+If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
+
+### Dimension handling
+Apache Druid supports schema changes. Therefore, dimensions can be different across segments even if they are a part of the same data source. See [Different schemas among segments](../design/segments.md#different-schemas-among-segments). If the input segments have different dimensions, the resulting compacted segment includes all dimensions of the input segments. 
+
+Even when the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. The dimensions of recent segments precede that of old segments in terms of data types and the ordering because more recent segments are more likely to have the preferred order and data types.
+
+If you want to use your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec.
+
+### Rollup
+Druid only rolls up the output segment when `rollup` is set for all input segments. See [Roll-up](../ingestion/index.md#rollup) for more details.
+You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.md#analysistypes).
+
+## Setting up manual compaction
+
+To perform a manual compaction, you submit a compaction task. Compaction tasks merge all segments for the defined interval according to the following syntax:
+
+```json

Review comment:
       granularitySpec is missing from this example 

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.

Review comment:
       You can also add and remove dimensions with compaction task. That can change your data too. 

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.
+
+### Query granularity handling
+
+Unless you modify the query granularity in the [granularity spec](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
+
+> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+
+If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
+
+### Dimension handling
+Apache Druid supports schema changes. Therefore, dimensions can be different across segments even if they are a part of the same data source. See [Different schemas among segments](../design/segments.md#different-schemas-among-segments). If the input segments have different dimensions, the resulting compacted segment includes all dimensions of the input segments. 
+
+Even when the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. The dimensions of recent segments precede that of old segments in terms of data types and the ordering because more recent segments are more likely to have the preferred order and data types.
+
+If you want to use your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec.
+
+### Rollup
+Druid only rolls up the output segment when `rollup` is set for all input segments. See [Roll-up](../ingestion/index.md#rollup) for more details.
+You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.md#analysistypes).
+
+## Setting up manual compaction
+
+To perform a manual compaction, you submit a compaction task. Compaction tasks merge all segments for the defined interval according to the following syntax:
+
+```json
+{
+    "type": "compact",
+    "id": <task_id>,
+    "dataSource": <task_datasource>,
+    "ioConfig": <IO config>,
+    "dimensionsSpec" <custom dimensionsSpec>,
+    "metricsSpec" <custom metricsSpec>,
+    "tuningConfig" <parallel indexing task tuningConfig>,
+    "context": <task context>
+}
+```
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `compact`|Yes|
+|`id`|Task id|No|
+|`dataSource`|Data source name to compact|Yes|
+|`ioConfig`|I/O configuration for compaction task. See [Compaction I/O configuration](#compaction-io-configuration) for details.|Yes|
+|`dimensionsSpec`|Custom dimensions spec. The compaction task uses the specified dimensions spec if it exists instead of generating one.|No|
+|`metricsSpec`|Custom metrics spec. The compaction task uses the specified metrics spec rather than generating one.|No|
+|`segmentGranularity`|When set, the compaction task changes the segment granularity for the given interval.  Deprecated. Use `granularitySpec`. |No.|
+|`tuningConfig`|[Parallel indexing task tuningConfig](./native-batch.md#tuningconfig)|No|
+|`context`|[Task context](./tasks.md#context)|No|
+|`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` and `queryGranularity` for the compacted segments. See [Compaction granularitySpec](#compaction-granularity-spec).|No|
+
+To control the number of result segments per time chunk, you can set [maxRowsPerSegment](../configuration/index.md#compaction-dynamic-configuration) or [numShards](../ingestion/native-batch.md#tuningconfig).
+
+> You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals.
+
+A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [DruidInputSource](../native-batch.md#druid-input-source), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default.

Review comment:
       I would change from "...dimensions and metrics of the input segments by default." to something like "...dimensions and metrics of the input segments when not given in the compaction spec".

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.
+
+### Query granularity handling
+
+Unless you modify the query granularity in the [granularity spec](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
+
+> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+
+If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
+
+### Dimension handling
+Apache Druid supports schema changes. Therefore, dimensions can be different across segments even if they are a part of the same data source. See [Different schemas among segments](../design/segments.md#different-schemas-among-segments). If the input segments have different dimensions, the resulting compacted segment includes all dimensions of the input segments. 
+
+Even when the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. The dimensions of recent segments precede that of old segments in terms of data types and the ordering because more recent segments are more likely to have the preferred order and data types.
+
+If you want to use your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec.
+
+### Rollup
+Druid only rolls up the output segment when `rollup` is set for all input segments. See [Roll-up](../ingestion/index.md#rollup) for more details.
+You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.md#analysistypes).
+

Review comment:
       You can't change rollup via compaction task right now.
   
   To add to Suneet comment, for dimension order - you can change it to add or remove columns too. For example, for old data, you want no longer need x,y,z columns and wish to remove them or maybe you want to add a new aggregation on old data.

##########
File path: docs/configuration/index.md
##########
@@ -842,24 +842,24 @@ A description of the compaction config is:
 |`taskPriority`|[Priority](../ingestion/tasks.md#priority) of compaction task.|no (default = 25)|
 |`inputSegmentSizeBytes`|Maximum number of total segment bytes processed per compaction task. Since a time chunk must be processed in its entirety, if the segments for a particular time chunk have a total size in bytes greater than this parameter, compaction will not run for that time chunk. Because each compaction task runs with a single thread, setting this value too far above 1–2GB will result in compaction tasks taking an excessive amount of time.|no (default = 419430400)|
 |`maxRowsPerSegment`|Max number of rows per segment after compaction.|no|
-|`skipOffsetFromLatest`|The offset for searching segments to be compacted. Strongly recommended to set for realtime dataSources. |no (default = "P1D")|
+|`skipOffsetFromLatest`|The offset for searching segments to be compacted in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Strongly recommended to set for realtime dataSources. See [Data handling with compaction](../ingestion/compaction.md#data-handling-with-compaction)|no (default = "P1D")|
 |`tuningConfig`|Tuning config for compaction tasks. See below [Compaction Task TuningConfig](#compaction-tuningconfig).|no|
 |`taskContext`|[Task context](../ingestion/tasks.md#context) for compaction tasks.|no|
+|`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` for the compacted segments.|No|

Review comment:
       Note that this is also different from manual compaction's granularitySpec described in #compaction-granularity-spec since the one here does not support `queryGranularity`

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.

Review comment:
       You may also want to use compaction to change from dynamic partitioning (best effort roll up) to hash/range (perfect roll up)

##########
File path: docs/ingestion/compaction.md
##########
@@ -0,0 +1,210 @@
+---
+id: compaction
+title: "Compaction"
+description: "Defines compaction and automatic compaction (auto-compaction or autocompaction) as a strategy for segment optimization. Use cases for compaction. Describes compaction task configuration."
+---
+
+<!--
+  ~ 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.
+  -->
+Query performance in Apache Druid depends on optimally sized segments. Compaction is one strategy you can use to optimize segment size for your Druid database. Compaction tasks read an existing set of segments for a given time interval and combine the data into a new "compacted" set of segments. The compacted segments are generally larger, but there are fewer of them. Here compaction increases performance because fewer segments require less the per-segment processing and the memory overhead for ingestion and for querying paths.
+
+As a general strategy, compaction is effective when you have data arriving out of chronological order resulting in lots of small segments. This often happens, for example, if you are appending data using `appendToExisting` for [native batch](./native_batch.md) ingestion. Conversely, if you are rewriting your data with each ingestion task, you don't need to use compaction.
+
+In some cases you can use compaction to reduce segment size. For example, if a misconfigured ingestion task creates oversized segments, you can create a compaction task to split the segment files into smaller, more optimally sized ones.
+
+See [Segment optimization](../operations/segment-optimization.md) for guidance to determine if compaction will help in your environment.
+
+
+## Types of segment compaction
+You can configure the Druid Coordinator to perform automatic compaction, also called auto-compaction, for a datasource. Using a segment search policy, the coordinator periodically identifies segments for compaction starting with the newest to oldest. When segments can benefit from compaction, the coordinator automatically submits a compaction task. 
+
+Automatic compaction works in most use cases and should be your first option. To learn more about automatic compaction, see [Compacting Segments](../design/coordinator.md#compacting-segments).
+
+In cases where you require more control over compaction, you can manually submit compaction tasks. For example:
+- Automatic compaction is too slow.
+- You want to force compaction for a specific time range.
+- You want to compact data out of chronological order.
+
+See [Setting up a manual compaction task](#setting-up-manual-compaction) for more about manual compaction tasks.
+
+
+## Data handling with compaction
+During compaction, Druid overwrites the original set of segments with the compacted set without modifying the data. During compaction Druid locks the segments for the time interval being compacted to ensure data consistency.
+
+If an ingestion task needs to write data to a segment for a time interval locked for compaction, the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information.
+
+### Segment granularity handling
+
+Unless you modify the segment granularity in the [granularity spec](#compaction-granularity-spec), Druid attempts to retain the granularity for the compacted segments. When segments have different segment granularities with no overlap in interval Druid creates a separate compaction task for each to retain the segment granularity in the compacted segment. If segments have different segment granularities before compaction but there is some overlap in interval, Druid attempts find start and end of the overlapping interval and uses the closest segment granularity level for the compacted segment.
+
+### Query granularity handling
+
+Unless you modify the query granularity in the [granularity spec](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
+
+> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+
+If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
+
+### Dimension handling
+Apache Druid supports schema changes. Therefore, dimensions can be different across segments even if they are a part of the same data source. See [Different schemas among segments](../design/segments.md#different-schemas-among-segments). If the input segments have different dimensions, the resulting compacted segment includes all dimensions of the input segments. 
+
+Even when the input segments have the same set of dimensions, the dimension order or the data type of dimensions can be different. The dimensions of recent segments precede that of old segments in terms of data types and the ordering because more recent segments are more likely to have the preferred order and data types.
+
+If you want to use your own ordering and types, you can specify a custom `dimensionsSpec` in the compaction task spec.
+
+### Rollup
+Druid only rolls up the output segment when `rollup` is set for all input segments. See [Roll-up](../ingestion/index.md#rollup) for more details.
+You can check that your segments are rolled up or not by using [Segment Metadata Queries](../querying/segmentmetadataquery.md#analysistypes).
+
+## Setting up manual compaction
+
+To perform a manual compaction, you submit a compaction task. Compaction tasks merge all segments for the defined interval according to the following syntax:
+
+```json
+{
+    "type": "compact",
+    "id": <task_id>,
+    "dataSource": <task_datasource>,
+    "ioConfig": <IO config>,
+    "dimensionsSpec" <custom dimensionsSpec>,
+    "metricsSpec" <custom metricsSpec>,
+    "tuningConfig" <parallel indexing task tuningConfig>,
+    "context": <task context>
+}
+```
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `compact`|Yes|
+|`id`|Task id|No|
+|`dataSource`|Data source name to compact|Yes|
+|`ioConfig`|I/O configuration for compaction task. See [Compaction I/O configuration](#compaction-io-configuration) for details.|Yes|
+|`dimensionsSpec`|Custom dimensions spec. The compaction task uses the specified dimensions spec if it exists instead of generating one.|No|
+|`metricsSpec`|Custom metrics spec. The compaction task uses the specified metrics spec rather than generating one.|No|
+|`segmentGranularity`|When set, the compaction task changes the segment granularity for the given interval.  Deprecated. Use `granularitySpec`. |No.|
+|`tuningConfig`|[Parallel indexing task tuningConfig](./native-batch.md#tuningconfig)|No|
+|`context`|[Task context](./tasks.md#context)|No|
+|`granularitySpec`|Custom `granularitySpec` to describe the `segmentGranularity` and `queryGranularity` for the compacted segments. See [Compaction granularitySpec](#compaction-granularity-spec).|No|
+
+To control the number of result segments per time chunk, you can set [maxRowsPerSegment](../configuration/index.md#compaction-dynamic-configuration) or [numShards](../ingestion/native-batch.md#tuningconfig).
+
+> You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals.
+
+A compaction task internally generates an `index` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [DruidInputSource](../native-batch.md#druid-input-source), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default.
+
+Compaction tasks would exit without doing anything and issue a failure status code:
+- if the interval you specify has no data segments loaded<br>
+OR
+- if the interval you specify is empty.
+
+Note that the metadata between input segments and the resulting compacted segments may differ if the metadata among the input segments differs as well. If all input segments have the same metadata, however, the resulting output segment will have the same metadata as all input segments.
+
+
+### Example compaction task
+The following JSON illustrates a compaction task to compact _all segments_ within the interval `2017-01-01/2018-01-01` and create new segments:
+
+```json
+{
+  "type" : "compact",
+  "dataSource" : "wikipedia",
+  "ioConfig" : {
+    "type": "compact",
+    "inputSpec": {
+      "type": "interval",
+      "interval": "2017-01-01/2018-01-01",
+    }
+  }
+}
+```
+
+This task doesn't specify a `granularitySpec` so Druid retains the original segment granularity unchanged when compaction is complete.
+
+### Compaction I/O configuration
+
+The compaction `ioConfig` requires specifying `inputSpec` as follows:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `compact`|Yes|
+|`inputSpec`|Input specification|Yes|
+
+There are two supported `inputSpec`s for now.
+
+The interval `inputSpec` is:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `interval`|Yes|
+|`interval`|Interval to compact|Yes|
+
+The segments `inputSpec` is:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`type`|Task type. Should be `segments`|Yes|
+|`segments`|A list of segment IDs|Yes|
+
+### Compaction granularity spec
+
+You can optionally use the `granularitySpec` object to configure the segment granularity and the query granularity of the compacted segments. Their syntax is as follows:
+```json
+    "type": "compact",
+    "id": <task_id>,
+    "dataSource": <task_datasource>,
+    ...
+    ,
+    "granularitySpec": {
+      "segmentGranularity": <time_period>,
+      "queryGranularity": <time_period>
+    }
+    ...
+```
+
+`granularitySpec` takes the following keys:
+
+|Field|Description|Required|
+|-----|-----------|--------|
+|`segmentGranularity`|Time chunking period for the segment granularity. Defaults to 'null'. Accepts all [Query granularities](../querying/granularities.md).|No|
+|`queryGranularity`|Time chunking period for the query granularity. Defaults to 'null'. Accepts all [Query granularities](../querying/granularities.md). Not supported for automatic compaction.|No|

Review comment:
       Maybe explain the behavior of null here too




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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