Apache Druid supports the following types of native batch indexing tasks:
- Parallel task indexing (`index_parallel`) that can run multiple indexing tasks concurrently. Parallel task works well for production ingestion tasks.
- Simple task indexing (`index`) that run a single indexing task at a time. Simple task indexing is suitable for development and test environments.
To run either kind of native batch indexing task you can:
- Use the **Load Data** UI in the Druid console to define and submit an ingestion spec.
- Define an ingestion spec in JSON based upon the [examples](#parallel-indexing-example) and reference topics for batch indexing. Then POST the ingestion spec to the [Indexer API endpoint](../operations/api-reference.md#tasks),
`/druid/indexer/v1/task`, the Overlord service. Alternatively you can use the indexing script included with Druid at `bin/post-index-task`.
The parallel task type `index_parallel` is a task for multi-threaded batch indexing. Parallel task indexing only relies on Druid resources. It does not depend on other external systems like Hadoop.
Druid issues the worker tasks to the Overlord. The overlord schedules and runs the workers on MiddleManagers or Indexers. After a worker task successfully processes the assigned input portion, it reports the resulting segment list to the supervisor task.
The supervisor task periodically checks the status of worker tasks. If a task fails, the supervisor retries the task until the number of retries reaches the configured limit. If all worker tasks succeed, it publishes the reported segments at once and finalizes ingestion.
- a splittable [`inputSource`](#splittable-input-sources) in the `ioConfig`. For a list of supported splittable input formats, see [Splittable input sources](#splittable-input-sources).
- the `maxNumConcurrentSubTasks` greater than 1 in the `tuningConfig`. Otherwise tasks run sequentially. The `index_parallel` task reads each input file one by one and creates segments by itself.
This section covers implementation details to consider when you implement parallel task ingestion.
#### Volume control for worker tasks
You can control the amount of input data each worker task processes using different configurations depending on the phase in parallel ingestion.
- See [`partitionsSpec`](#partitionsspec) for details about how partitioning affects data volume for tasks.
- For the tasks that read data from the `inputSource`, you can set the [Split hint spec](#split-hint-spec) in the `tuningConfig`.
- For the task that merge shuffled segments, you can set the `totalNumMergeTasks` in the `tuningConfig`.
#### Number of running tasks
The `maxNumConcurrentSubTasks` in the `tuningConfig` determines the number of concurrent worker tasks that run in parallel. The supervisor task checks the number of current running worker tasks and creates more if it's smaller than `maxNumConcurrentSubTasks` regardless of the number of available task slots. This may affect to other ingestion performance. See [Capacity planning](#capacity-planning) section for more details.
#### Replacing or appending data
By default, batch ingestion replaces all data in the intervals in your `granularitySpec`' for any segment that it writes to. If you want to add to the segment instead, set the `appendToExisting` flag in the `ioConfig`. Batch ingestion only replaces data in segments where it actively adds data. If there are segments in the intervals for your `granularitySpec` that have do not have data from a task, they remain unchanged. If any existing segments partially overlap with the intervals in the `granularitySpec`, the portion of those segments outside the interval for the new spec remain visible.
#### Fully replacing existing segments using tombstones
You can set `dropExisting` flag in the `ioConfig` to true if you want the ingestion task to replace all existing segments that start and end within the intervals for your `granularitySpec`. This applies whether or not the new data covers all existing segments. `dropExisting` only applies when `appendToExisting` is false and the `granularitySpec` contains an `interval`. WARNING: this functionality is still in beta.
Consider an existing segment with an interval of 2020-01-01 to 2021-01-01 and `YEAR``segmentGranularity`. You want to overwrite the whole interval of 2020-01-01 to 2021-01-01 with new data using the finer segmentGranularity of `MONTH`. If the replacement data does not have a record within every months from 2020-01-01 to 2021-01-01 Druid cannot drop the original `YEAR` segment even if it does include all the replacement data. Set `dropExisting` to true in this case to replace the original segment at `YEAR``segmentGranularity` since you no longer need it.<br><br>
Imagine you want to re-ingest or overwrite a datasource and the new data does not contain some time intervals that exist in the datasource. For example, a datasource contains the following data at `MONTH` segmentGranularity:
- **January**: 1 record
- **February**: 10 records
- **March**: 10 records
You want to re-ingest and overwrite with new data as follows:
- **January**: 0 records
- **February**: 10 records
- **March**: 9 records
Unless you set `dropExisting` to true, the result after ingestion with overwrite using the same MONTH segmentGranularity would be:
This may not be what it is expected since the new data has 0 records for January. Set `dropExisting` to true to replace the unneeded January segment with a tombstone.
This field is required. In general, it defines the way that Druid will store your data: the primary timestamp column, the dimensions, metrics, and any transformations. For an overview, see [Ingestion Spec DataSchema](../ingestion/ingestion-spec.md#dataschema).
When defining the `granularitySpec` for index parallel, consider the defining `intervals` explicitly if you know the time range of the data. This way locking failure happens faster and Druid won't accidentally replace data outside the interval range some rows contain unexpected timestamps. The reasoning is as follows:
- If you explicitly define `intervals`, batch ingestion locks all intervals specified when it starts up. Problems with locking become evident quickly when multiple ingestion or indexing tasks try to obtain a lock on the same interval. For example, if a Kafka ingestion task tries to obtain a lock on a locked interval causing the ingestion task fail. Furthermore, if there are rows outside the specified intervals, Druid drops them, avoiding conflict with unexpected intervals.
- If you do not define `intervals`, batch ingestion locks each interval when the interval is discovered. In this case if the task overlaps with a higher-priority task, issues with conflicting locks occur later in the ingestion process. Also if the source data includes rows with unexpected timestamps, they may caused unexpected locking of intervals.
|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This means that you can append new segments to any datasource regardless of its original partitioning scheme. You must use the `dynamic` partitioning type for the appended segments. If you specify a different partitioning type, the task fails with an error.|false|no|
|dropExisting|If `true` and `appendToExisting` is `false` and the `granularitySpec` contains an`interval`, then the ingestion task replaces all existing segments fully contained by the specified `interval` when the task publishes new segments. If ingestion fails, Druid does not change any existing segment. In the case of misconfiguration where either `appendToExisting` is `true` or `interval` is not specified in `granularitySpec`, Druid does not replace any segments even if `dropExisting` is `true`. WARNING: this functionality is still in beta.|false|no|
|maxRowsInMemory|Determines when Druid should perform intermediate persists to disk. Normally you do not need to set this. Depending on the nature of your data, if rows are short in terms of bytes. For example, you may not want to store a million rows in memory. In this case, set this value.|1000000|no|
|maxBytesInMemory|Use to determine when Druid should perform intermediate persists to disk. Normally Druid computes this internally and you do not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists). Note that `maxBytesInMemory` also includes heap usage of artifacts created from intermediary persists. This means that after every persist, the amount of `maxBytesInMemory` until next persist will decrease. Tasks fail when the sum of bytes of all intermediary persisted artifacts exceeds `maxBytesInMemory`.|1/6 of max JVM memory|no|
|maxColumnsToMerge|Limit of the number of segments to merge in a single phase when merging segments for publishing. This limit affects the total number of columns present in a set of segments to merge. If the limit is exceeded, segment merging occurs in multiple phases. Druid merges at least 2 segments per phase, regardless of this setting.|-1 (unlimited)|no|
|maxTotalRows|Deprecated. Use `partitionsSpec` instead. Total number of rows in segments waiting to be pushed. Used to determine when intermediate pushing should occur.|20000000|no|
|numShards|Deprecated. Use `partitionsSpec` instead. Directly specify the number of shards to create when using a `hashed``partitionsSpec`. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. `numShards` cannot be specified if `maxRowsPerSegment` is set.|null|no|
|splitHintSpec|Hint to control the amount of data that each first phase task reads. Druid may ignore the hint depending on the implementation of the input source. See [Split hint spec](#split-hint-spec) for more details.|size-based split hint spec|no|
|partitionsSpec|Defines how to partition data in each timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic` if `forceGuaranteedRollup` = false, `hashed` or `single_dim` if `forceGuaranteedRollup` = true|no|
|indexSpecForIntermediatePersists|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use this configuration to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, if you disable compression on intermediate segments, page cache use my increase while intermediate segments are used before Druid merges them to the final published segment published. See [IndexSpec](./ingestion-spec.md#indexspec) for possible values.|same as indexSpec|no|
|maxPendingPersists|Maximum number of pending persists that remain not started. If a new intermediate persist exceeds this limit, ingestion blocks until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
|forceGuaranteedRollup|Forces [perfect rollup](rollup.md). The perfect rollup optimizes the total size of generated segments and querying time but increases indexing time. If true, specify `intervals` in the `granularitySpec` and use either `hashed` or `single_dim` for the `partitionsSpec`. You cannot use this flag in conjunction with `appendToExisting` of IOConfig. For more details, see [Segment pushing modes](#segment-pushing-modes).|false|no|
|reportParseExceptions|If true, Druid throws exceptions encountered during parsing and halts ingestion. If false, Druid skips unparseable rows and fields.|false|no|
|pushTimeout|Milliseconds to wait to push segments. Must be >= 0, where 0 means to wait forever.|0|no|
|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [SegmentWriteOutMediumFactory](./native-batch-simple-task.md#segmentwriteoutmediumfactory).|If not specified, uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` |no|
|maxNumConcurrentSubTasks|Maximum number of worker tasks that can be run in parallel at the same time. The supervisor task spawns worker tasks up to `maxNumConcurrentSubTasks` regardless of the current available task slots. If this value is 1, the supervisor task processes data ingestion on its own instead of spawning worker tasks. If this value is set to too large, the supervisor may create too many worker tasks that block other ingestion tasks. See [Capacity planning](#capacity-planning) for more details.|1|no|
|maxNumSegmentsToMerge|Max limit for the number of segments that a single task can merge at the same time in the second phase. Used only when `forceGuaranteedRollup` is true.|100|no|
|totalNumMergeTasks|Total number of tasks that merge segments in the merge phase when `partitionsSpec` is set to `hashed` or `single_dim`.|10|no|
|awaitSegmentAvailabilityTimeoutMillis|Long|Milliseconds to wait for the newly indexed segments to become available for query after ingestion completes. If `<= 0`, no wait occurs. If `> 0`, the task waits for the Coordinator to indicate that the new segments are available for querying. If the timeout expires, the task exits as successful, but the segments are not confirmed as available for query.|no (default = 0)|
The split hint spec is used to help the supervisor task divide input sources. Each worker task processes a single input division. You can control the amount of data each worker task reads during the first phase.
|maxSplitSize|Maximum number of bytes of input files to process in a single subtask. If a single file is larger than the limit, Druid processes the file alone in a single subtask. Druid does not split files across tasks. One subtask will not process more files than `maxNumFiles` even when their total size is smaller than `maxSplitSize`. [Human-readable format](../configuration/human-readable-byte.md) is supported.|1GiB|no|
|maxNumFiles|Maximum number of input files to process in a single subtask. This limit avoids task failures when the ingestion spec is too long. There are two known limits on the max size of serialized ingestion spec: the max ZNode size in ZooKeeper (`jute.maxbuffer`) and the max packet size in MySQL (`max_allowed_packet`). These limits can cause ingestion tasks fail if the serialized ingestion spec size hits one of them. One subtask will not process more data than `maxSplitSize` even when the total number of files is smaller than `maxNumFiles`.|1000|no|
|maxInputSegmentBytesPerTask|Maximum number of bytes of input segments to process in a single subtask. If a single segment is larger than this number, Druid processes the segment alone in a single subtask. Druid never splits input segments across tasks. A single subtask will not process more segments than `maxNumSegments` even when their total size is smaller than `maxInputSegmentBytesPerTask`. [Human-readable format](../configuration/human-readable-byte.md) is supported.|1GiB|no|
|maxNumSegments|Maximum number of input segments to process in a single subtask. This limit avoids failures due to the the ingestion spec being too long. There are two known limits on the max size of serialized ingestion spec: the max ZNode size in ZooKeeper (`jute.maxbuffer`) and the max packet size in MySQL (`max_allowed_packet`). These limits can make ingestion tasks fail when the serialized ingestion spec size hits one of them. A single subtask will not process more data than `maxInputSegmentBytesPerTask` even when the total number of segments is smaller than `maxNumSegments`.|1000|no|
The primary partition for Druid is time. You can define a secondary partitioning method in the partitions spec. Use the `partitionsSpec` type that applies for your [rollup](rollup.md) method. For perfect rollup, you can use:
-`hashed` partitioning based on the hash value of specified dimensions for each row
-`single_dim` based on ranges of values for a single dimension
-`range` based on ranges of values of multiple dimensions.
| `hashed` | Moderate | Multiple dimension [hash-based partitioning](#hash-based-partitioning) may reduce both your datasource size and query latency by improving data locality. See [Partitioning](./partitioning.md) for more details. | Perfect rollup | The broker can use the partition information to prune segments early to speed up queries. Since the broker knows how to hash `partitionDimensions` values to locate a segment, given a query including a filter on all the `partitionDimensions`, the broker can pick up only the segments holding the rows satisfying the filter on `partitionDimensions` for query processing.<br/><br/>Note that `partitionDimensions` must be set at ingestion time to enable secondary partition pruning at query time.|
| `single_dim` | Slower | Single dimension [range partitioning](#single-dimension-range-partitioning) may reduce your datasource size and query latency by improving data locality. See [Partitioning](./partitioning.md) for more details. | Perfect rollup | The broker can use the partition information to prune segments early to speed up queries. Since the broker knows the range of `partitionDimension` values in each segment, given a query including a filter on the `partitionDimension`, the broker can pick up only the segments holding the rows satisfying the filter on `partitionDimension` for query processing. |
| `range` | Slowest | Multiple dimension [range partitioning](#multi-dimension-range-partitioning) may reduce your datasource size and query latency by improving data locality. See [Partitioning](./partitioning.md) for more details. | Perfect rollup | The broker can use the partition information to prune segments early to speed up queries. Since the broker knows the range of `partitionDimensions` values within each segment, given a query including a filter on the first of the `partitionDimensions`, the broker can pick up only the segments holding the rows satisfying the filter on the first partition dimension for query processing. |
|maxRowsPerSegment|Used in sharding. Determines how many rows are in each segment.|5000000|no|
|maxTotalRows|Total number of rows across all segments waiting for being pushed. Used in determining when intermediate segment push should occur.|20000000|no|
With the Dynamic partitioning, the parallel index task runs in a single phase:
- When the total number of rows in all segments across all time chunks reaches to `maxTotalRows`. At this point the task pushes all segments created so far to the deep storage and creates new ones.
|numShards|Directly specify the number of shards to create. If this is specified and `intervals` is specified in the `granularitySpec`, the index task can skip the determine intervals/partitions pass through the data. This property and `targetRowsPerSegment` cannot both be set.|none|no|
|targetRowsPerSegment|A target row count for each partition. If `numShards` is left unspecified, the Parallel task will determine a partition count automatically such that each partition has a row count close to the target, assuming evenly distributed keys in the input data. A target per-segment row count of 5 million is used if both `numShards` and `targetRowsPerSegment` are null. |null (or 5,000,000 if both `numShards` and `targetRowsPerSegment` are null)|no|
and assigns each split to a worker task. Each worker task (type `partial_index_generate`) reads the assigned split, and partitions rows by the time chunk from `segmentGranularity` (primary partition key) in the `granularitySpec`
The Parallel task spawns a new set of worker tasks (type `partial_index_generic_merge`) to merge the partitioned data created in the previous phase. Here, the partitioned data is shuffled based on
the time chunk and the hash value of `partitionDimensions` to be merged; each worker task reads the data falling in the same time chunk and the same hash value from multiple MiddleManager/Indexer processes and merges them to create the final segments. Finally, they push the final segments to the deep storage at once.
In hash partitioning, the partition function is used to compute hash of partition dimensions. The partition dimension values are first serialized into a byte array as a whole, and then the partition function is applied to compute hash of the byte array. Druid currently supports only one partition function.
|`murmur3_32_abs`|Applies an absolute value function to the result of [`murmur3_32`](https://guava.dev/releases/16.0/api/docs/com/google/common/hash/Hashing.html#murmur3_32()).|
When you use this technique to partition your data, segment sizes may be unequally distributed if the data in your `partitionDimension` is also unequally distributed. Therefore, to avoid imbalance in data layout, review the distribution of values in your source data before deciding on a partitioning strategy.
|targetRowsPerSegment|Target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|none|either this or `maxRowsPerSegment`|
|assumeGrouped|Assume that input data has already been grouped on time and dimensions. Ingestion will run faster, but may choose sub-optimal partitions if this assumption is violated.|false|no|
the assigned split and builds a histogram for `partitionDimension`.
The Parallel task collects those histograms from worker tasks and finds
the best range partitioning based on `partitionDimension` to evenly
distribute rows across partitions. Note that either `targetRowsPerSegment`
or `maxRowsPerSegment` will be used to find the best partitioning.
- In the `partial segment generation` phase, the Parallel task spawns new worker tasks (type `partial_range_index_generate`)
to create partitioned data. Each worker task reads a split created as in the previous phase,
partitions rows by the time chunk from the `segmentGranularity` (primary partition key) in the `granularitySpec`
and then by the range partitioning found in the previous phase.
The partitioned data is stored in local storage of
the [middleManager](../design/middlemanager.md) or the [indexer](../design/indexer.md).
- In the `partial segment merge` phase, the parallel index task spawns a new set of worker tasks (type `partial_index_generic_merge`) to merge the partitioned
data created in the previous phase. Here, the partitioned data is shuffled based on
the time chunk and the value of `partitionDimension`; each worker task reads the segments
falling in the same partition of the same range from multiple MiddleManager/Indexer processes and merges
them to create the final segments. Finally, they push the final segments to the deep storage.
> Because the task with single-dimension range partitioning makes two passes over the input
> in `partial dimension distribution` and `partial segment generation` phases,
> the task may fail if the input changes in between the two passes.
> Multiple dimension (multi-dimension) range partitioning is an experimental feature. Multi-dimension range partitioning is currently not supported in the sequential mode of the Parallel task.
When you use multi-dimension partitioning for your data, Druid is able to distribute segment sizes more evenly than with single dimension partitioning.
For segment pruning to be effective and translate into better query performance, you must include the first of your `partitionDimensions` in the `WHERE` clause at query time. For example, given the following `partitionDimensions`:
```
"partitionsSpec": {
"type": "range",
"partitionDimensions":["coutryName","cityName"],
"targetRowsPerSegment" : 5000
}
```
Use "countryName" or both "countryName" and "cityName" in the `WHERE` clause of your query to take advantage of the performance benefits from multi-dimension partitioning.
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|Set the value to `range`.|none|yes|
|partitionDimensions|An array of dimensions to partition on. Order the dimensions from most frequently queried to least frequently queried. For best results, limit your number of dimensions to between three and five dimensions.|none|yes|
|targetRowsPerSegment|Target number of rows to include in a partition, should be a number that targets segments of 500MB\~1GB.|none|either this or `maxRowsPerSegment`|
|maxRowsPerSegment|Soft max for the number of rows to include in a partition.|none|either this or `targetRowsPerSegment`|
|assumeGrouped|Assume that input data has already been grouped on time and dimensions. Ingestion will run faster, but may choose sub-optimal partitions if this assumption is violated.|false|no|
Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
While ingesting data using the parallel task indexing, Druid creates segments from the input data and pushes them. For segment pushing,
the parallel task index supports the following segment pushing modes based upon your type of [rollup](./rollup.md):
- Bulk pushing mode: Used for perfect rollup. Druid pushes every segment at the very end of the index task. Until then, Druid stores created segments in memory and local storage of the service running the index task. This mode can cause problems if you have limited storage capacity, and is not recommended to use in production.
To enable bulk pushing mode, set `forceGuaranteedRollup` in your TuningConfig. You cannot use bulk pushing with `appendToExisting` in your IOConfig.
- Incremental pushing mode: Used for best-effort rollup. Druid pushes segments are incrementally during the course of the indexing task. The index task collects data and stores created segments in the memory and disks of the services running the task until the total number of collected rows exceeds `maxTotalRows`. At that point the index task immediately pushes all segments created up until that moment, cleans up pushed segments, and continues to ingest the remaining data.
Use the `inputSource` object to define the location where your index can read data. Only the native parallel task and simple task support the input source.