SQL-based ingestion reference

This page describes SQL-based batch ingestion using the druid-multi-stage-query extension, new in Druid 24.0. Refer to the ingestion methods table to determine which ingestion method is right for you.

SQL reference

This topic is a reference guide for the multi-stage query architecture in Apache Druid. For examples of real-world usage, refer to the Examples page.

EXTERN

Use the EXTERN function to read external data.

Function format:

  1. SELECT
  2. <column>
  3. FROM TABLE(
  4. EXTERN(
  5. '<Druid input source>',
  6. '<Druid input format>',
  7. '<row signature>'
  8. )
  9. )

EXTERN consists of the following parts:

  1. Any Druid input source as a JSON-encoded string.
  2. Any Druid input format as a JSON-encoded string.
  3. A row signature, as a JSON-encoded array of column descriptors. Each column descriptor must have a name and a type. The type can be string, long, double, or float. This row signature is used to map the external data into the SQL layer.

For more information, see Read external data with EXTERN.

INSERT

Use the INSERT statement to insert data.

Unlike standard SQL, INSERT loads data into the target table according to column name, not positionally. If necessary, use AS in your SELECT column list to assign the correct names. Do not rely on their positions within the SELECT clause.

Statement format:

  1. INSERT INTO <table name>
  2. < SELECT query >
  3. PARTITIONED BY <time frame>
  4. [ CLUSTERED BY <column list> ]

INSERT consists of the following parts:

  1. Optional context parameters.
  2. An INSERT INTO <dataSource> clause at the start of your query, such as INSERT INTO your-table.
  3. A clause for the data you want to insert, such as SELECT ... FROM .... You can use EXTERN to reference external tables using FROM TABLE(EXTERN(...)).
  4. A PARTITIONED BY clause, such as PARTITIONED BY DAY.
  5. An optional CLUSTERED BY clause.

For more information, see Load data with INSERT.

REPLACE

You can use the REPLACE function to replace all or some of the data.

Unlike standard SQL, REPLACE loads data into the target table according to column name, not positionally. If necessary, use AS in your SELECT column list to assign the correct names. Do not rely on their positions within the SELECT clause.

REPLACE all data

Function format to replace all data:

  1. REPLACE INTO <target table>
  2. OVERWRITE ALL
  3. < SELECT query >
  4. PARTITIONED BY <time granularity>
  5. [ CLUSTERED BY <column list> ]

REPLACE specific time ranges

Function format to replace specific time ranges:

  1. REPLACE INTO <target table>
  2. OVERWRITE WHERE __time >= TIMESTAMP '<lower bound>' AND __time < TIMESTAMP '<upper bound>'
  3. < SELECT query >
  4. PARTITIONED BY <time granularity>
  5. [ CLUSTERED BY <column list> ]

REPLACE consists of the following parts:

  1. Optional context parameters.
  2. A REPLACE INTO <dataSource> clause at the start of your query, such as REPLACE INTO "your-table".
  3. An OVERWRITE clause after the datasource, either OVERWRITE ALL or OVERWRITE WHERE:
  • OVERWRITE ALL replaces the entire existing datasource with the results of the query.
  • OVERWRITE WHERE drops the time segments that match the condition you set. Conditions are based on the __time column and use the format __time [< > = <= >=] TIMESTAMP. Use them with AND, OR, and NOT between them, inclusive of the timestamps specified. No other expressions or functions are valid in OVERWRITE.
  1. A clause for the actual data you want to use for the replacement.
  2. A PARTITIONED BY clause, such as PARTITIONED BY DAY.
  3. An optional CLUSTERED BY clause.

For more information, see Overwrite data with REPLACE.

PARTITIONED BY

The PARTITIONED BY <time granularity> clause is required for INSERT and REPLACE. See Partitioning for details.

The following granularity arguments are accepted:

  • Time unit: HOUR, DAY, MONTH, or YEAR. Equivalent to FLOOR(__time TO TimeUnit).
  • TIME_FLOOR(__time, 'granularity_string'), where granularity_string is one of the ISO 8601 periods listed below. The first argument must be __time.
  • FLOOR(__time TO TimeUnit), where TimeUnit is any unit supported by the FLOOR function. The first argument must be __time.
  • ALL or ALL TIME, which effectively disables time partitioning by placing all data in a single time chunk. To use LIMIT or OFFSET at the outer level of your INSERT or REPLACE query, you must set PARTITIONED BY to ALL or ALL TIME.

The following ISO 8601 periods are supported for TIME_FLOOR:

  • PT1S
  • PT1M
  • PT5M
  • PT10M
  • PT15M
  • PT30M
  • PT1H
  • PT6H
  • P1D
  • P1W
  • P1M
  • P3M
  • P1Y

For more information about partitioning, see Partitioning.

CLUSTERED BY

The CLUSTERED BY <column list> clause is optional for INSERT and REPLACE. It accepts a list of column names or expressions.

For more information about clustering, see Clustering.

Context parameters

In addition to the Druid SQL context parameters, the multi-stage query task engine accepts certain context parameters that are specific to it.

Use context parameters alongside your queries to customize the behavior of the query. If you’re using the API, include the context parameters in the query context when you submit a query:

  1. {
  2. "query": "SELECT 1 + 1",
  3. "context": {
  4. "<key>": "<value>",
  5. "maxNumTasks": 3
  6. }
  7. }

If you’re using the web console, you can specify the context parameters through various UI options.

The following table lists the context parameters for the MSQ task engine:

ParameterDescriptionDefault value
maxNumTasksSELECT, INSERT, REPLACE

The maximum total number of tasks to launch, including the controller task. The lowest possible value for this setting is 2: one controller and one worker. All tasks must be able to launch simultaneously. If they cannot, the query returns a TaskStartTimeout error code after approximately 10 minutes.

May also be provided as numTasks. If both are present, maxNumTasks takes priority.
2
taskAssignmentSELECT, INSERT, REPLACE

Determines how many tasks to use. Possible values include:
  • max: Uses as many tasks as possible, up to maxNumTasks.
  • auto: When file sizes can be determined through directory listing (for example: local files, S3, GCS, HDFS) uses as few tasks as possible without exceeding 10 GiB or 10,000 files per task, unless exceeding these limits is necessary to stay within maxNumTasks. When file sizes cannot be determined through directory listing (for example: http), behaves the same as max.
max
finalizeAggregationsSELECT, INSERT, REPLACE

Determines the type of aggregation to return. If true, Druid finalizes the results of complex aggregations that directly appear in query results. If false, Druid returns the aggregation’s intermediate type rather than finalized type. This parameter is useful during ingestion, where it enables storing sketches directly in Druid tables. For more information about aggregations, see SQL aggregation functions.
true
rowsInMemoryINSERT or REPLACE

Maximum number of rows to store in memory at once before flushing to disk during the segment generation process. Ignored for non-INSERT queries. In most cases, use the default value. You may need to override the default if you run into one of the known issues around memory usage.
100,000
segmentSortOrderINSERT or REPLACE

Normally, Druid sorts rows in individual segments using time first, followed by the CLUSTERED BY clause. When you set segmentSortOrder, Druid sorts rows in segments using this column list first, followed by the CLUSTERED BY order.

You provide the column list as comma-separated values or as a JSON array in string form. If your query includes
time, then this list must begin with time. For example, consider an INSERT query that uses CLUSTERED BY country and has segmentSortOrder set to time,city. Within each time chunk, Druid assigns rows to segments based on country, and then within each of those segments, Druid sorts those rows by __time first, then city, then country.
empty list
maxParseExceptionsSELECT, INSERT, REPLACE

Maximum number of parse exceptions that are ignored while executing the query before it stops with TooManyWarningsFault. To ignore all the parse exceptions, set the value to -1.
0
rowsPerSegmentINSERT or REPLACE

The number of rows per segment to target. The actual number of rows per segment may be somewhat higher or lower than this number. In most cases, use the default. For general information about sizing rows per segment, see Segment Size Optimization.
3,000,000
sqlTimeZoneSets the time zone for this connection, which affects how time functions and timestamp literals behave. Use a time zone name like “America/Los_Angeles” or offset like “-08:00”.druid.sql.planner.sqlTimeZone on the Broker (default: UTC)
useApproximateCountDistinctWhether to use an approximate cardinality algorithm for COUNT(DISTINCT foo).druid.sql.planner.useApproximateCountDistinct on the Broker (default: true)

Limits

Knowing the limits for the MSQ task engine can help you troubleshoot any errors that you encounter. Many of the errors occur as a result of reaching a limit.

The following table lists query limits:

LimitValueError if exceeded
Size of an individual row written to a frame. Row size when written to a frame may differ from the original row size.1 MBRowTooLarge
Number of segment-granular time chunks encountered during ingestion.5,000TooManyBuckets
Number of input files/segments per worker.10,000TooManyInputFiles
Number of output partitions for any one stage. Number of segments generated during ingestion.25,000TooManyPartitions
Number of output columns for any one stage.2,000TooManyColumns
Number of workers for any one stage.Hard limit is 1,000. Memory-dependent soft limit may be lower.TooManyWorkers
Maximum memory occupied by broadcasted tables.30% of each processor memory bundle.BroadcastTablesTooLarge

Error codes

The following table describes error codes you may encounter in the multiStageQuery.payload.status.errorReport.error.errorCode field:

CodeMeaningAdditional fields
BroadcastTablesTooLargeThe size of the broadcast tables, used in right hand side of the joins, exceeded the memory reserved for them in a worker task.maxBroadcastTablesSize: Memory reserved for the broadcast tables, measured in bytes.
CanceledThe query was canceled. Common reasons for cancellation:

  • User-initiated shutdown of the controller task via the /druid/indexer/v1/task/{taskId}/shutdown API.
  • Restart or failure of the server process that was running the controller task.
CannotParseExternalDataA worker task could not parse data from an external datasource.
ColumnNameRestrictedThe query uses a restricted column name.
ColumnTypeNotSupportedSupport for writing or reading from a particular column type is not supported.
ColumnTypeNotSupportedThe query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.columnName

columnType
InsertCannotAllocateSegmentThe controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:

  • Attempting to mix different granularities in the same intervals of the same datasource.
  • Prior ingestions that used non-extendable shard specs.
dataSource

interval: The interval for the attempted new segment allocation.
InsertCannotBeEmptyAn INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with PARTITIONED BY set to something other than ALL or ALL TIME.dataSource
InsertCannotOrderByDescendingAn INSERT query contained a CLUSTERED BY expression in descending order. Druid’s segment generation code only supports ascending order.columnName
InsertCannotReplaceExistingSegmentA REPLACE query cannot proceed because an existing segment partially overlaps those bounds, and the portion within the bounds is not fully overshadowed by query results.

There are two ways to address this without modifying your query:
  • Shrink the OVERLAP filter to match the query results.
  • Expand the OVERLAP filter to fully contain the existing segment.
segmentId: The existing segment
InsertLockPreemptedAn INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task.
InsertTimeNullAn INSERT or REPLACE query encountered a null timestamp in the time field.

This can happen due to using an expression like TIME_PARSE(timestamp) AS
time with a timestamp that cannot be parsed. (TIME_PARSE returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern.

If your timestamps may genuinely be null, consider using COALESCE to provide a default value. One option is CURRENT_TIMESTAMP, which represents the start time of the job.
InsertTimeOutOfBoundsA REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause.

To avoid this error, verify that the you specified is valid.
interval: time chunk interval corresponding to the out-of-bounds timestamp
InvalidNullByteA string column included a null byte. Null bytes in strings are not permitted.column: The column that included the null byte
QueryNotSupportedQueryKit could not translate the provided native query to a multi-stage query.

This can happen if the query uses features that aren’t supported, like GROUPING SETS.
RowTooLargeThe query tried to process a row that was too large to write to a single frame. See the Limits table for the specific limit on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing.maxFrameSize: The limit on the frame size.
TaskStartTimeoutUnable to launch all the worker tasks in time.

There might be insufficient available slots to start all the worker tasks simultaneously.

Try splitting up the query into smaller chunks with lesser maxNumTasks number. Another option is to increase capacity.
TooManyBucketsExceeded the number of partition buckets for a stage. Partition buckets are only used for segmentGranularity during INSERT queries. The most common reason for this error is that your segmentGranularity is too narrow relative to the data. See the Limits table for the specific limit.maxBuckets: The limit on buckets.
TooManyInputFilesExceeded the number of input files/segments per worker. See the Limits table for the specific limit.umInputFiles: The total number of input files/segments for the stage.

maxInputFiles: The maximum number of input files/segments per worker per stage.

minNumWorker: The minimum number of workers required for a successful run.
TooManyPartitionsExceeded the number of partitions for a stage. The most common reason for this is that the final stage of an INSERT or REPLACE query generated too many segments. See the Limits table for the specific limit.maxPartitions: The limit on partitions which was exceeded
TooManyColumnsExceeded the number of columns for a stage. See the Limits table for the specific limit.maxColumns: The limit on columns which was exceeded.
TooManyWarningsExceeded the allowed number of warnings of a particular type.rootErrorCode: The error code corresponding to the exception that exceeded the required limit.

maxWarnings: Maximum number of warnings that are allowed for the corresponding rootErrorCode.
TooManyWorkersExceeded the supported number of workers running simultaneously. See the Limits table for the specific limit.workers: The number of simultaneously running workers that exceeded a hard or soft limit. This may be larger than the number of workers in any one stage if multiple stages are running simultaneously.

maxWorkers: The hard or soft limit on workers that was exceeded.
NotEnoughMemoryInsufficient memory to launch a stage.serverMemory: The amount of memory available to a single process.

serverWorkers: The number of workers running in a single process.

serverThreads: The number of threads in a single process.
WorkerFailedA worker task failed unexpectedly.workerTaskId: The ID of the worker task.
WorkerRpcFailedA remote procedure call to a worker task failed and could not recover.workerTaskId: the id of the worker task
UnknownErrorAll other errors.