Skip to main content

SQL metadata tables

info

Apache Druid supports two query languages: Druid SQL and native queries. This document describes the SQL language.

Druid Brokers infer table and column metadata for each datasource from segments loaded in the cluster, and use this to plan SQL queries. This metadata is cached on Broker startup and also updated periodically in the background through SegmentMetadata queries. Background metadata refreshing is triggered by segments entering and exiting the cluster, and can also be throttled through configuration.

Druid exposes system information through special system tables. There are two such schemas available: Information Schema and Sys Schema. Information schema provides details about table and column types. The "sys" schema provides information about Druid internals like segments/tasks/servers.

INFORMATION SCHEMA

You can access table and column metadata through JDBC using connection.getMetaData(), or through the INFORMATION_SCHEMA tables described below. For example, to retrieve metadata for the Druid datasource "foo", use the query:

SELECT *
FROM INFORMATION_SCHEMA.COLUMNS
WHERE "TABLE_SCHEMA" = 'druid' AND "TABLE_NAME" = 'foo'
info

Note: INFORMATION_SCHEMA tables do not currently support Druid-specific functions like TIME_PARSE and APPROX_QUANTILE_DS. Only standard SQL functions can be used.

SCHEMATA table

INFORMATION_SCHEMA.SCHEMATA provides a list of all known schemas, which include druid for standard Druid Table datasources, lookup for Lookups, sys for the virtual System metadata tables, and INFORMATION_SCHEMA for these virtual tables. Tables are allowed to have the same name across different schemas, so the schema may be included in an SQL statement to distinguish them, e.g. lookup.table vs druid.table.

ColumnTypeNotes
CATALOG_NAMEVARCHARAlways set as druid
SCHEMA_NAMEVARCHARdruid, lookup, sys, or INFORMATION_SCHEMA
SCHEMA_OWNERVARCHARUnused
DEFAULT_CHARACTER_SET_CATALOGVARCHARUnused
DEFAULT_CHARACTER_SET_SCHEMAVARCHARUnused
DEFAULT_CHARACTER_SET_NAMEVARCHARUnused
SQL_PATHVARCHARUnused

TABLES table

INFORMATION_SCHEMA.TABLES provides a list of all known tables and schemas.

ColumnTypeNotes
TABLE_CATALOGVARCHARAlways set as druid
TABLE_SCHEMAVARCHARThe 'schema' which the table falls under, see SCHEMATA table for details
TABLE_NAMEVARCHARTable name. For the druid schema, this is the dataSource.
TABLE_TYPEVARCHAR"TABLE" or "SYSTEM_TABLE"
IS_JOINABLEVARCHARIf a table is directly joinable if on the right hand side of a JOIN statement, without performing a subquery, this value will be set to YES, otherwise NO. Lookups are always joinable because they are globally distributed among Druid query processing nodes, but Druid datasources are not, and will use a less efficient subquery join.
IS_BROADCASTVARCHARIf a table is 'broadcast' and distributed among all Druid query processing nodes, this value will be set to YES, such as lookups and Druid datasources which have a 'broadcast' load rule, else NO.

COLUMNS table

INFORMATION_SCHEMA.COLUMNS provides a list of all known columns across all tables and schema.

ColumnTypeNotes
TABLE_CATALOGVARCHARAlways set as druid
TABLE_SCHEMAVARCHARThe 'schema' which the table column falls under, see SCHEMATA table for details
TABLE_NAMEVARCHARThe 'table' which the column belongs to, see TABLES table for details
COLUMN_NAMEVARCHARThe column name
ORDINAL_POSITIONBIGINTThe order in which the column is stored in a table
COLUMN_DEFAULTVARCHARUnused
IS_NULLABLEVARCHAR
DATA_TYPEVARCHAR
CHARACTER_MAXIMUM_LENGTHBIGINTUnused
CHARACTER_OCTET_LENGTHBIGINTUnused
NUMERIC_PRECISIONBIGINT
NUMERIC_PRECISION_RADIXBIGINT
NUMERIC_SCALEBIGINT
DATETIME_PRECISIONBIGINT
CHARACTER_SET_NAMEVARCHAR
COLLATION_NAMEVARCHAR
JDBC_TYPEBIGINTType code from java.sql.Types (Druid extension)

For example, this query returns data type information for columns in the foo table:

SELECT "ORDINAL_POSITION", "COLUMN_NAME", "IS_NULLABLE", "DATA_TYPE", "JDBC_TYPE"
FROM INFORMATION_SCHEMA.COLUMNS
WHERE "TABLE_NAME" = 'foo'

ROUTINES table

INFORMATION_SCHEMA.ROUTINES provides a list of all known functions.

ColumnTypeNotes
ROUTINE_CATALOGVARCHARThe catalog that contains the routine. Always set as druid
ROUTINE_SCHEMAVARCHARThe schema that contains the routine. Always set as INFORMATION_SCHEMA
ROUTINE_NAMEVARCHARTHe routine name
ROUTINE_TYPEVARCHARThe routine type. Always set as FUNCTION
IS_AGGREGATORVARCHARIf a routine is an aggregator function, then the value will be set to YES, else NO
SIGNATURESVARCHAROne or more routine signatures

For example, this query returns information about all the aggregator functions:

SELECT "ROUTINE_CATALOG", "ROUTINE_SCHEMA", "ROUTINE_NAME", "ROUTINE_TYPE", "IS_AGGREGATOR", "SIGNATURES"
FROM "INFORMATION_SCHEMA"."ROUTINES"
WHERE "IS_AGGREGATOR" = 'YES'

SYSTEM SCHEMA

The "sys" schema provides visibility into Druid segments, servers and tasks.

info

Note: "sys" tables do not currently support Druid-specific functions like TIME_PARSE and APPROX_QUANTILE_DS. Only standard SQL functions can be used.

SEGMENTS table

Segments table provides details on all Druid segments, whether they are published yet or not.

ColumnTypeNotes
segment_idVARCHARUnique segment identifier
datasourceVARCHARName of datasource
startVARCHARInterval start time (in ISO 8601 format)
endVARCHARInterval end time (in ISO 8601 format)
sizeBIGINTSize of segment in bytes
versionVARCHARVersion string (generally an ISO8601 timestamp corresponding to when the segment set was first started). Higher version means the more recently created segment. Version comparing is based on string comparison.
partition_numBIGINTPartition number (an integer, unique within a datasource+interval+version; may not necessarily be contiguous)
num_replicasBIGINTNumber of replicas of this segment currently being served
num_rowsBIGINTNumber of rows in this segment, or zero if the number of rows is not known.

This row count is gathered by the Broker in the background. It will be zero if the Broker has not gathered a row count for this segment yet. For segments ingested from streams, the reported row count may lag behind the result of a count(*) query because the cached num_rows on the Broker may be out of date. This will settle shortly after new rows stop being written to that particular segment.
is_activeBIGINTTrue for segments that represent the latest state of a datasource.

Equivalent to (is_published = 1 AND is_overshadowed = 0) OR is_realtime = 1. In steady state, when no ingestion or data management operations are happening, is_active will be equivalent to is_available. However, they may differ from each other when ingestion or data management operations have executed recently. In these cases, Druid will load and unload segments appropriately to bring actual availability in line with the expected state given by is_active.
is_publishedBIGINTBoolean represented as long type where 1 = true, 0 = false. 1 if this segment has been published to the metadata store and is marked as used. See the segment lifecycle documentation for more details.
is_availableBIGINTBoolean represented as long type where 1 = true, 0 = false. 1 if this segment is currently being served by any data serving process, like a Historical or a realtime ingestion task. See the segment lifecycle documentation for more details.
is_realtimeBIGINTBoolean represented as long type where 1 = true, 0 = false. 1 if this segment is only served by realtime tasks, and 0 if any Historical process is serving this segment.
is_overshadowedBIGINTBoolean represented as long type where 1 = true, 0 = false. 1 if this segment is published and is fully overshadowed by some other published segments. Currently, is_overshadowed is always 0 for unpublished segments, although this may change in the future. You can filter for segments that "should be published" by filtering for is_published = 1 AND is_overshadowed = 0. Segments can briefly be both published and overshadowed if they were recently replaced, but have not been unpublished yet. See the segment lifecycle documentation for more details.
shard_specVARCHARJSON-serialized form of the segment ShardSpec
dimensionsVARCHARJSON-serialized form of the segment dimensions
metricsVARCHARJSON-serialized form of the segment metrics
last_compaction_stateVARCHARJSON-serialized form of the compaction task's config (compaction task which created this segment). May be null if segment was not created by compaction task.
replication_factorBIGINTTotal number of replicas of the segment that are required to be loaded across all historical tiers, based on the load rule that currently applies to this segment. If this value is 0, the segment is not assigned to any historical and will not be loaded. This value is -1 if load rules for the segment have not been evaluated yet.

For example, to retrieve all currently active segments for datasource "wikipedia", use the query:

SELECT * FROM sys.segments
WHERE datasource = 'wikipedia'
AND is_active = 1

Another example to retrieve segments total_size, avg_size, avg_num_rows and num_segments per datasource:

SELECT
datasource,
SUM("size") AS total_size,
CASE WHEN SUM("size") = 0 THEN 0 ELSE SUM("size") / (COUNT(*) FILTER(WHERE "size" > 0)) END AS avg_size,
CASE WHEN SUM(num_rows) = 0 THEN 0 ELSE SUM("num_rows") / (COUNT(*) FILTER(WHERE num_rows > 0)) END AS avg_num_rows,
COUNT(*) AS num_segments
FROM sys.segments
WHERE is_active = 1
GROUP BY 1
ORDER BY 2 DESC

This query goes a step further and shows the overall profile of available, non-realtime segments across buckets of 1 million rows each for the foo datasource:

SELECT ABS("num_rows" /  1000000) as "bucket",
COUNT(*) as segments,
SUM("size") / 1048576 as totalSizeMiB,
MIN("size") / 1048576 as minSizeMiB,
AVG("size") / 1048576 as averageSizeMiB,
MAX("size") / 1048576 as maxSizeMiB,
SUM("num_rows") as totalRows,
MIN("num_rows") as minRows,
AVG("num_rows") as averageRows,
MAX("num_rows") as maxRows,
(AVG("size") / AVG("num_rows")) as avgRowSizeB
FROM sys.segments
WHERE is_available = 1 AND is_realtime = 0 AND "datasource" = `foo`
GROUP BY 1
ORDER BY 1

If you want to retrieve segment that was compacted (ANY compaction):

SELECT * FROM sys.segments WHERE is_active = 1 AND last_compaction_state IS NOT NULL

or if you want to retrieve segment that was compacted only by a particular compaction spec (such as that of the auto compaction):

SELECT * FROM sys.segments WHERE is_active = 1 AND last_compaction_state = 'CompactionState{partitionsSpec=DynamicPartitionsSpec{maxRowsPerSegment=5000000, maxTotalRows=9223372036854775807}, indexSpec={bitmap={type=roaring}, dimensionCompression=lz4, metricCompression=lz4, longEncoding=longs, segmentLoader=null}}'

SERVERS table

Servers table lists all discovered servers in the cluster.

ColumnTypeNotes
serverVARCHARServer name in the form host:port
hostVARCHARHostname of the server
plaintext_portBIGINTUnsecured port of the server, or -1 if plaintext traffic is disabled
tls_portBIGINTTLS port of the server, or -1 if TLS is disabled
server_typeVARCHARType of Druid service. Possible values include: COORDINATOR, OVERLORD, BROKER, ROUTER, HISTORICAL, MIDDLE_MANAGER or PEON.
tierVARCHARDistribution tier see druid.server.tier. Only valid for HISTORICAL type, for other types it's null
current_sizeBIGINTCurrent size of segments in bytes on this server. Only valid for HISTORICAL type, for other types it's 0
max_sizeBIGINTMax size in bytes this server recommends to assign to segments see druid.server.maxSize. Only valid for HISTORICAL type, for other types it's 0
is_leaderBIGINT1 if the server is currently the 'leader' (for services which have the concept of leadership), otherwise 0 if the server is not the leader, or the default long value (null or zero depending on druid.generic.useDefaultValueForNull) if the server type does not have the concept of leadership
start_timeSTRINGTimestamp in ISO8601 format when the server was announced in the cluster

To retrieve information about all servers, use the query:

SELECT * FROM sys.servers;

SERVER_SEGMENTS table

SERVER_SEGMENTS is used to join servers with segments table

ColumnTypeNotes
serverVARCHARServer name in format host:port (Primary key of servers table)
segment_idVARCHARSegment identifier (Primary key of segments table)

JOIN between "servers" and "segments" can be used to query the number of segments for a specific datasource, grouped by server, example query:

SELECT count(segments.segment_id) as num_segments from sys.segments as segments
INNER JOIN sys.server_segments as server_segments
ON segments.segment_id = server_segments.segment_id
INNER JOIN sys.servers as servers
ON servers.server = server_segments.server
WHERE segments.datasource = 'wikipedia'
GROUP BY servers.server;

TASKS table

The tasks table provides information about active and recently-completed indexing tasks. For more information check out the documentation for ingestion tasks.

ColumnTypeNotes
task_idVARCHARUnique task identifier
group_idVARCHARTask group ID for this task, the value depends on the task type. For example, for native index tasks, it's same as task_id, for sub tasks, this value is the parent task's ID
typeVARCHARTask type, for example this value is "index" for indexing tasks. See tasks-overview
datasourceVARCHARDatasource name being indexed
created_timeVARCHARTimestamp in ISO8601 format corresponding to when the ingestion task was created. Note that this value is populated for completed and waiting tasks. For running and pending tasks this value is set to 1970-01-01T00:00:00Z
queue_insertion_timeVARCHARTimestamp in ISO8601 format corresponding to when this task was added to the queue on the Overlord
statusVARCHARStatus of a task can be RUNNING, FAILED, SUCCESS
runner_statusVARCHARRunner status of a completed task would be NONE, for in-progress tasks this can be RUNNING, WAITING, PENDING
durationBIGINTTime it took to finish the task in milliseconds, this value is present only for completed tasks
locationVARCHARServer name where this task is running in the format host:port, this information is present only for RUNNING tasks
hostVARCHARHostname of the server where task is running
plaintext_portBIGINTUnsecured port of the server, or -1 if plaintext traffic is disabled
tls_portBIGINTTLS port of the server, or -1 if TLS is disabled
error_msgVARCHARDetailed error message in case of FAILED tasks

For example, to retrieve tasks information filtered by status, use the query

SELECT * FROM sys.tasks WHERE status='FAILED';

SUPERVISORS table

The supervisors table provides information about supervisors.

ColumnTypeNotes
supervisor_idVARCHARSupervisor task identifier
stateVARCHARBasic state of the supervisor. Available states: UNHEALTHY_SUPERVISOR, UNHEALTHY_TASKS, PENDING, RUNNING, SUSPENDED, STOPPING. See Supervisor reference for more information.
detailed_stateVARCHARSupervisor specific state. See documentation of the specific supervisor for details: Kafka or Kinesis.
healthyBIGINTBoolean represented as long type where 1 = true, 0 = false. 1 indicates a healthy supervisor
typeVARCHARType of supervisor, e.g. kafka, kinesis or materialized_view
sourceVARCHARSource of the supervisor, e.g. Kafka topic or Kinesis stream
suspendedBIGINTBoolean represented as long type where 1 = true, 0 = false. 1 indicates supervisor is in suspended state
specVARCHARJSON-serialized supervisor spec

For example, to retrieve supervisor tasks information filtered by health status, use the query

SELECT * FROM sys.supervisors WHERE healthy=0;