The upgrade notes assume that you are upgrading from the Druid version that immediately precedes your target version. If you are upgrading across multiple versions, make sure you read the upgrade notes for all the intermediate versions.
For the full release notes for a specific version, see the releases page.
Front-coded dictionaries reduce storage and improve performance by optimizing for strings where the front part looks similar.
Once this feature is on, you cannot easily downgrade to an earlier version that does not support the feature.
For more information, see Migration guide: front-coded dictionaries.
If you‘re already using this feature, you don’t need to take any action.
Hadoop-based ingestion has been deprecated since Druid 32.0 and is scheduled to be removed in Druid 37.0.0.
We recommend one of Druid's other supported ingestion methods, such as SQL-based ingestion or MiddleManager-less ingestion using Kubernetes.
As part of this change, you must now opt-in to using the deprecated index_hadoop
task type. If you don't do this, your Hadoop-based ingestion tasks will fail.
To opt-in, set druid.indexer.task.allowHadoopTaskExecution
to true
in your common.runtime.properties
file. #18239
groupBy
and topN
queriesDruid now uses the groupBy
native query type, rather than topN
, for SQL queries that group by and order by the same column, have LIMIT
, and don't have HAVING
. This speeds up execution of such queries since groupBy
is vectorized while topN
is not.
You can restore the previous behavior by setting the query context parameter useLexicographicTopN
to true
. Behavior for useApproximateTopN
is unchanged, and the default remains true
.
IS_INCREMENTAL_HANDOFF_SUPPORTED
config removedRemoved the IS_INCREMENTAL_HANDOFF_SUPPORTED
context reference from supervisors, as incremental publishing has been the default behavior since version 0.16.0. This context was originally introduced to support rollback to LegacyKafkaIndexTaskRunner
in versions earlier than 0.16.0, which has since been removed.
useMaxMemoryEstimates
config removedRemoved the useMaxMemoryEstimates
config. When set to false, Druid used a much more accurate memory estimate that was introduced in Druid 0.23.0. That more accurate method is the only available method now. The config has defaulted to false for several releases.
useMaxMemoryEstimates
useMaxMemoryEstimates
is now set to false for MSQ task engine tasks. Additionally, the property has been deprecated and will be removed in a future release. Setting this to false allows for better on-heap memory estimation.
Automatic kill tasks are now limited to 30 days or fewer worth of segments per task.
The previous behavior (no limit on interval per kill task) can be restored by setting druid.coordinator.kill.maxInterval = P0D
.
By default, the Docker image now uses the canonical hostname if you're running Druid in Kubernetes. Otherwise, it uses the IP address otherwise #17697
Various configs were deprecated in a previous release and have now been removed. The following table lists the removed configs and their replacements:
Removed config | Replacement config |
---|---|
druid.processing.merge.task.initialYieldNumRows | druid.processing.merge.initialYieldNumRows |
druid.processing.merge.task.targetRunTimeMillis | druid.processing.merge.targetRunTimeMillis |
druid.processing.merge.task.smallBatchNumRows | druid.processing.merge.smallBatchNumRows |
druid.processing.merge.pool.awaitShutdownMillis | |
druid.processing.merge.awaitShutdownMillis | |
druid.processing.merge.pool.parallelism | druid.processing.merge.parallelism |
druid.processing.merge.pool.defaultMaxQueryParallelism | druid.processing.merge.defaultMaxQueryParallelism |
If you need to downgrade to a version where Druid doesn't support the segment metadata cache, you must set the druid.manager.segments.useCache
config to false or remove it prior to the upgrade.
This feature is introduced in Druid 33.0.
Support for the configs that let you maintain older behavior that wasn't ANSI-SQL compliant have been removed:
druid.generic.useDefaultValueForNull=true
druid.expressions.useStrictBooleans=false
druid.generic.useThreeValueLogicForNativeFilters=false
They no longer affect your query results. Only SQL-compliant non-legacy behavior is supported now.
If the configs are set to the legacy behavior, Druid services will fail to start.
If you want to continue to get the same results without these settings, you must update your queries or your results will be incorrect after you upgrade.
For more information about how to update your queries, see the migration guide.
Java support in Druid has been updated:
We recommend that you upgrade to Java 17.
Hadoop-based ingestion is now deprecated. We recommend that you migrate to SQL-based ingestion.
The SQL-based ingestion query context flag arrayIngestMode
now defaults to array
instead of mvd
. This means that SQL VARCHAR ARRAY
types is no longer implicitly translated and stored in VARCHAR
columns, but is instead stored as VARCHAR ARRAY
. This change permits other array types such as BIGINT ARRAY
and DOUBLE ARRAY
to be inserted with MSQ task engine into their respective array column types instead of failing as they do in mvd
mode.
To continue to store multi-value strings, modify any insert/replace queries to wrap the array types with the ARRAY_TO_MV
operator.
Validation is in place to prevent mixing VARCHAR
and VARCHAR ARRAY
columns in the same table, so any ingestions affected by this change will fail and provide a descriptive error message instead of exhibiting unexpected behavior.
The arrayIngestMode
option of none
has been removed. It was introduced prior to the table validation logic as a means for cluster operators to force query writers to explicitly set array
or mvd
on their query contexts, but provides little utility in Druid 31.
See the following topics for more information:
The deprecated task action audit logging has been removed. This change includes the following updates:
/indexer/v1/task/{taskId}/segments
is no longer supported.druid_taskLog
.druid.indexer.auditlog.enabled
.task/action/log/time
.These changes are backward compatible with all existing metadata storage extensions.
Removed Firehose and FirehoseFactory and remaining implementations. Apache deprecated support for Druid firehoses in version 0.17. Support for firehose ingestion was removed in version 26.0.
The native scan query legacy mode has been removed. It was introduced in Druid 0.11 to maintain compatibility during an upgrade from older versions of Druid where the scan query was part of a contrib
extension.
Hard-coded "legacy":false
following removal of the legacy mode to prevent error during rolling upgrades or downgrades.
ZK-based segment loading is now disabled. ZK servedSegmentsPath
was deprecated in Druid 0.7.1. This legacy path has been replaced by liveSegmentsPath
.
Segment-serving processes such as Peons, Historicals and Indexers no longer create ZK loadQueuePath
entries. The druid.zk.paths.loadQueuePath
and druid.zk.paths.servedSegmentsPath
properties are no longer used.
Move to HTTP-based segment loading first and then perform the version upgrade.
The append
function for JsonPath for ORC format now fails with an exception. Previously, it would run but not append anything.
The following properties have been deprecated as part of simplifying the memory tuning for Kinesis ingestion:
recordBufferSize
, use recordBufferSizeBytes
insteadmaxRecordsPerPoll
, use maxBytesPerPoll
insteadThe stopTaskCount
config now prioritizes stopping older tasks first. As part of this change, you must also explicitly set a value for stopTaskCount
. It no longer defaults to the same value as taskCount
.
The following are the changes to the default values for the Coordinator service:
druid.coordinator.kill.period
(if unspecified) has changed from P1D
to the value of druid.coordinator.period.indexingPeriod
. Operators can choose to override druid.coordinator.kill.period
and that takes precedence over the default behavior.killTaskSlotRatio
has been updated from 1.0
to 0.1
. This ensures that kill tasks take up only one task slot by default instead of consuming all available task slots.GoogleTaskLogs
upload buffer sizeChanged the upload buffer size in GoogleTaskLogs
to 1 MB instead of 15 MB to allow more uploads in parallel and prevent the Middle Manager service from running out of memory.
targetDataSource
in EXPLAIN queriesDruid 30.0.0 includes a breaking change that restores the behavior for targetDataSource
to its 28.0.0 and earlier state, different from Druid 29.0.0 and only 29.0.0. In 29.0.0, targetDataSource
returns a JSON object that includes the datasource name. In all other versions, targetDataSource
returns a string containing the name of the datasource.
If you're upgrading from any version other than 29.0.0, there is no change in behavior.
If you are upgrading from 29.0.0, this is an incompatible change.
ZooKeeper-based segment loading is being removed due to known issues. It has been deprecated for several releases. Recent improvements to the Druid Coordinator have significantly enhanced performance with HTTP-based segment loading.
Removed the following Coordinator configs:
druid.coordinator.load.timeout
: Not needed as the default value of this parameter (15 minutes) is known to work well for all clusters.druid.coordinator.loadqueuepeon.type
: Not needed as this value is always http
.druid.coordinator.curator.loadqueuepeon.numCallbackThreads
: Not needed as ZooKeeper(curator)-based segment loading isn't an option anymore.Auto-cleanup of compaction configs of inactive datasources is now enabled by default.
useMaxMemoryEstimates
for Hadoop jobsThe default value of the useMaxMemoryEstimates
parameter for Hadoop jobs is now false
.
targetDataSource
in EXPLAIN queriesDruid 29.0.1 includes a breaking change that restores the behavior for targetDataSource
to its 28.0.0 and earlier state, different from Druid 29.0.0 and only 29.0.0. In 29.0.0, targetDataSource
returns a JSON object that includes the datasource name. In all other versions, targetDataSource
returns a string containing the name of the datasource.
If you're upgrading from any version other than 29.0.0, there is no change in behavior.
If you are upgrading from 29.0.0, this is an incompatible change.
equals
filter for native queriesThe equality filter on mixed type auto
columns that contain arrays must now be filtered as their presenting type. This means that if any rows are arrays (for example, the segment metadata and information_schema
reports the type as some array type), then the native queries must also filter as if they are some array type.
This change impacts mixed type auto
columns that contain both scalars and arrays. It doesn't impact SQL, which already has this limitation due to how the type presents itself.
arrayIngestMode
for MSQ queriesDruid console now configures the arrayIngestMode
parameter in the data loading flow, and its value can persist across the SQL tab unless manually updated. When loading multi-value dimensions or arrays in the Druid console, note the value of the arrayIngestMode
parameter to prevent mixing multi-value dimensions and arrays in the same column of a data source.
You no longer have to manually determine the task lock type for concurrent append and replace (experimental) with the taskLockType
task context. Instead, Druid can now determine it automatically for you. You can use the context parameter "useConcurrentLocks": true
for individual tasks and datasources or enable concurrent append and replace at a cluster level using druid.indexer.task.default.context
.
The MSQ task engine now allows empty ingest queries by default. For queries that don't generate any output rows, the MSQ task engine reports zero values for numTotalRows
and totalSizeInBytes
instead of null. Previously, ingest queries that produced no data would fail with the InsertCannotBeEmpty
MSQ fault.
To revert to the original behavior, set the MSQ query parameter failOnEmptyInsert
to true
.
When query scheduler threads are less than server HTTP threads, total laning turns on. This reserves some HTTP threads for non-query requests such as health checks. The total laning previously would reject any query request that exceeds the lane capacity. Now, excess requests will instead be queued with a timeout equal to MIN(Integer.MAX_VALUE, druid.server.http.maxQueryTimeout)
.
Columns ingested with the auto column indexer that contain only empty or null arrays are now stored as ARRAY\<LONG\>
instead of COMPLEX<json\>
.
When the requested granularity is a month or larger but a segment can‘t be allocated, Druid resorts to day partitioning. Unless explicitly specified, Druid skips week-granularity segments for data partitioning because these segments don’t align with the end of the month or more coarse-grained intervals.
Previously, if Druid couldn't allocate segments by month, it tried allocating them by week next. In the new behavior, Druid skips partitioning by week and goes directly to day. Week segments can only be allocated if the chosen partitioning in the append task is WEEK.
auto
search strategyRemoved the auto
search strategy from the native search query. Setting searchStrategy
to auto
is now equivalent to useIndexes
.
Druid 28.0.0 adds a new column to the Druid metadata table that requires an update to the table.
If druid.metadata.storage.connector.createTables
is set to true
and the metadata store user has DDL privileges, the segments table gets automatically updated at startup to include the new used_status_last_updated
column. No additional work is needed for the upgrade.
If either of those requirements are not met, pre-upgrade steps are required. You must make these updates before you upgrade to Druid 28.0.0, or the Coordinator and Overlord processes fail.
Although you can manually alter your table to add the new used_status_last_updated
column, Druid also provides a CLI tool to do it.
In the example commands below:
lib
is the Druid lib directoryextensions
is the Druid extensions directorybase
corresponds to the value of druid.metadata.storage.tables.base
in the configuration, druid
by default.--connectURI
parameter corresponds to the value of druid.metadata.storage.connector.connectURI
.--user
parameter corresponds to the value of druid.metadata.storage.connector.user
.--password
parameter corresponds to the value of druid.metadata.storage.connector.password
.--action
parameter corresponds to the update action you are executing. In this case, it is add-last-used-to-segments
cd ${DRUID_ROOT} java -classpath "lib/*" -Dlog4j.configurationFile=conf/druid/cluster/_common/log4j2.xml -Ddruid.extensions.directory="extensions" -Ddruid.extensions.loadList=[\"mysql-metadata-storage\"] -Ddruid.metadata.storage.type=mysql org.apache.druid.cli.Main tools metadata-update --connectURI="<mysql-uri>" --user USER --password PASSWORD --base druid --action add-used-flag-last-updated-to-segments
cd ${DRUID_ROOT} java -classpath "lib/*" -Dlog4j.configurationFile=conf/druid/cluster/_common/log4j2.xml -Ddruid.extensions.directory="extensions" -Ddruid.extensions.loadList=[\"postgresql-metadata-storage\"] -Ddruid.metadata.storage.type=postgresql org.apache.druid.cli.Main tools metadata-update --connectURI="<postgresql-uri>" --user USER --password PASSWORD --base druid --action add-used-flag-last-updated-to-segments
ALTER TABLE druid_segments ADD used_status_last_updated varchar(255);
The recommended syntax for SQL UNNEST has changed. We recommend using CROSS JOIN instead of commas for most queries to prevent issues with precedence. For example, use:
SELECT column_alias_name1 FROM datasource CROSS JOIN UNNEST(source_expression1) AS table_alias_name1(column_alias_name1) CROSS JOIN UNNEST(source_expression2) AS table_alias_name2(column_alias_name2), ...
Do not use:
SELECT column_alias_name FROM datasource, UNNEST(source_expression1) AS table_alias_name1(column_alias_name1), UNNEST(source_expression2) AS table_alias_name2(column_alias_name2), ...
The Apache Calcite version has been upgraded from 1.21 to 1.35. As part of the Calcite upgrade, the behavior of type inference for dynamic parameters has changed. To avoid any type interference issues, explicitly CAST
all dynamic parameters as a specific data type in SQL queries. For example, use:
SELECT (1 * CAST (? as DOUBLE))/2 as tmp
Do not use:
SELECT (1 * ?)/2 as tmp
json
type columns created with Druid 28.0.0 are not backwards compatible with Druid versions older than 26.0.0. If you are upgrading from a version prior to Druid 26.0.0 and you use json
columns, upgrade to Druid 26.0.0 before you upgrade to Druid 28.0.0. Additionally, to downgrade to a version older than Druid 26.0.0, any new segments created in Druid 28.0.0 should be re-ingested using Druid 26.0.0 or 27.0.0 prior to further downgrading.
When upgrading from a previous version, you can continue to write nested columns in a backwards compatible format (version 4).
In a classic batch ingestion job, include formatVersion
in the dimensions
list of the dimensionsSpec
property. For example:
"dimensionsSpec": { "dimensions": [ "product", "department", { "type": "json", "name": "shipTo", "formatVersion": 4 } ] },
To set the default nested column version, set the desired format version in the common runtime properties. For example:
druid.indexing.formats.nestedColumnFormatVersion=4
Starting with Druid 28.0.0, the default way Druid treats nulls and booleans has changed.
For nulls, Druid now differentiates between an empty string and a record with no data as well as between an empty numerical record and 0
.
You can revert to the previous behavior by setting druid.generic.useDefaultValueForNull
to true
.
This property affects both storage and querying, and must be set on all Druid service types to be available at both ingestion time and query time. Reverting this setting to the old value restores the previous behavior without reingestion.
For booleans, Druid now strictly uses 1
(true) or 0
(false). Previously, true and false could be represented either as true
and false
as well as 1
and 0
, respectively. In addition, Druid now returns a null value for boolean comparisons like True && NULL
.
You can revert to the previous behavior by setting druid.expressions.useStrictBooleans
to false
. This property affects both storage and querying, and must be set on all Druid service types to be available at both ingestion time and query time. Reverting this setting to the old value restores the previous behavior without reingestion.
The following table illustrates some example scenarios and the impact of the changes.
Query | Druid 27.0.0 and earlier | Druid 28.0.0 and later |
---|---|---|
Query empty string | Empty string ('' ) or null | Empty string ('' ) |
Query null string | Null or empty | Null |
COUNT(*) | All rows, including nulls | All rows, including nulls |
COUNT(column) | All rows excluding empty strings | All rows including empty strings but excluding nulls |
Expression 100 && 11 | 11 | 1 |
Expression 100 || 11 | 100 | 1 |
Null FLOAT/DOUBLE column | 0.0 | Null |
Null LONG column | 0 | Null |
Null __time column | 0, meaning 1970-01-01 00:00:00 UTC | 1970-01-01 00:00:00 UTC |
Null MVD column | '' | Null |
ARRAY | Null | Null |
COMPLEX | none | Null |
Before upgrading to Druid 28.0.0, update your queries to account for the changed behavior as described in the following sections.
If your queries use NULL in the filter condition to match both nulls and empty strings, you should add an explicit filter clause for empty strings. For example, update s IS NULL
to s IS NULL OR s = ''
.
COUNT(column)
now counts empty strings. If you want to continue excluding empty strings from the count, replace COUNT(column)
with COUNT(column) FILTER(WHERE column <> '')
.
GroupBy queries on columns containing null values can now have additional entries as nulls can co-exist with empty strings.
If you have added supervisors that ingest from multiple Kafka topics in Druid 28.0.0 or later, stop those supervisors before downgrading to a version prior to Druid 28.0.0 because the supervisors will fail in versions prior to Druid 28.0.0.
lenientAggregatorMerge
deprecatedlenientAggregatorMerge
property in segment metadata queries has been deprecated. It will be removed in future releases. Use aggregatorMergeStrategy
instead. aggregatorMergeStrategy
also supports the latest
and earliest
strategies in addition to strict
and lenient
strategies from lenientAggregatorMerge
.
The paths for druid.processing.merge.pool.*
and druid.processing.merge.task.*
have been flattened to use druid.processing.merge.*
instead. The legacy paths for the configs are now deprecated and will be removed in a future release. Migrate your settings to use the new paths because the old paths will be ignored in the future.
Starting with Druid 28.0.0, the MSQ task engine can detect and ingest arrays as ARRAY typed columns when you set the query context parameter arrayIngestMode
to array
. The arrayIngestMode
context parameter controls how ARRAY type values are stored in Druid segments.
When you set arrayIngestMode
to array
(recommended for SQL compliance), the MSQ task engine stores all ARRAY typed values in ARRAY typed columns and supports storing both VARCHAR and numeric typed arrays.
For backwards compatibility, arrayIngestMode
defaults to mvd
. When "arrayIngestMode":"mvd"
, Druid only supports VARCHAR typed arrays and stores them as multi-value string columns.
When you set arrayIngestMode
to none
, Druid throws an exception when trying to store any type of arrays.
For more information on how to ingest ARRAY
typed columns with SQL-based ingestion, see SQL data types and Array columns.
Support for Hadoop 2 has been removed. Migrate to SQL-based ingestion or JSON-based batch ingestion if you are using Hadoop 2.x for ingestion today. If migrating to Druid's built-in ingestion is not possible, you must upgrade your Hadoop infrastructure to 3.x+ before upgrading to Druid 28.0.0.
The GroupBy v1 engine has been removed. Use the GroupBy v2 engine instead, which has been the default GroupBy engine for several releases. There should be no impact on your queries.
Additionally, AggregatorFactory.getRequiredColumns
has been deprecated and will be removed in a future release. If you have an extension that implements AggregatorFactory
, then this method should be removed from your implementation.
The decommissioningMaxPercentOfMaxSegmentsToMove
config has been removed. The use case for this config is handled by smart segment loading now, which is enabled by default.
cachingCost
strategyThe cachingCost
strategy for segment loading has been removed. Use cost
instead, which has the same benefits as cachingCost
.
If you have cachingCost
set, the system ignores this setting and automatically uses cost
.
InsertCannotOrderByDescending
The deprecated MSQ fault InsertCannotOrderByDescending
has been removed.
The backward compatibility code for the Handoff API in CoordinatorBasedSegmentHandoffNotifier
has been removed. If you are upgrading from a Druid version older than 0.14.0, upgrade to a newer version of Druid before upgrading to Druid 28.0.0.
The maximum input bytes for each worker for SQL-based ingestion is now 512 MiB (previously 10 GiB).
When using the built-in FileConfigProvider
for Kafka, interpolations are now intercepted by the JsonConfigurator instead of being passed down to the Kafka provider. This breaks existing deployments.
For more information, see KIP-297.
Many of the important dependent libraries that Druid uses no longer support Hadoop 2. In order for Druid to stay current and have pathways to mitigate security vulnerabilities, the community has decided to deprecate support for Hadoop 2.x releases starting this release. Starting with Druid 28.x, Hadoop 3.x is the only supported Hadoop version.
Consider migrating to SQL-based ingestion or native ingestion if you are using Hadoop 2.x for ingestion today. If migrating to Druid ingestion is not possible, plan to upgrade your Hadoop infrastructure before upgrading to the next Druid release.
GroupBy queries using the v1 legacy engine has been deprecated. It will be removed in future releases. Use v2 instead. Note that v2 has been the default GroupBy engine.
For more information, see GroupBy queries.
Support for push-based real-time ingestion has been deprecated. It will be removed in future releases.
cachingCost
segment balancing strategy deprecatedThe cachingCost
strategy has been deprecated and will be removed in future releases. Use an alternate segment balancing strategy instead, such as cost
.
The following segment related configs are now deprecated and will be removed in future releases:
maxSegmentsInNodeLoadingQueue
maxSegmentsToMove
replicationThrottleLimit
useRoundRobinSegmentAssignment
replicantLifetime
maxNonPrimaryReplicantsToLoad
decommissioningMaxPercentOfMaxSegmentsToMove
Use smartSegmentLoading
mode instead, which calculates values for these variables automatically.
Additionally, the defaults for the following Coordinator dynamic configs have changed:
maxsegmentsInNodeLoadingQueue
: 500, previously 100maxSegmentsToMove
: 100, previously 5replicationThrottleLimit
: 500, previously 10These new defaults can improve performance for most use cases.
SysMonitor
support deprecatedSwitch to OshiSysMonitor
as SysMonitor
is now deprecated and will be removed in future releases.
druid.processing.columnCache.sizeBytes
has been removed since it provided limited utility after a number of internal changes. Leaving this config is harmless, but it does nothing.
The following Coordinator dynamic configs have been removed:
emitBalancingStats
: Stats for errors encountered while balancing will always be emitted. Other debugging stats will not be emitted but can be logged by setting the appropriate debugDimensions
.useBatchedSegmentSampler
and percentOfSegmentsToConsiderPerMove
: Batched segment sampling is now the standard and will always be on.Use the new smart segment loading mode instead.
Optimized query performance by lowering the default maxRowsInMemory for real-time ingestion, which might lower overall ingestion throughput.
The firehose/parser specification used by legacy Druid streaming formats is removed. Firehose ingestion was deprecated in version 0.17, and support for this ingestion was removed in version 24.0.0.
The Druid system table (INFORMATION_SCHEMA
) now uses SQL types instead of Druid types for columns. This change makes the INFORMATION_SCHEMA
table behave more like standard SQL. You may need to update your queries in the following scenarios in order to avoid unexpected results if you depend either of the following:
frontCoded
segment format changeThe frontCoded
type of stringEncodingStrategy
on indexSpec
with a new segment format version, which typically has faster read speeds and reduced segment size. This improvement is backwards incompatible with Druid 25.0.0.
The default segment discovery method now uses HTTP instead of ZooKeeper.
This update changes the defaults for the following properties:
Property | New default | Previous default |
---|---|---|
druid.serverview.type for segment management | http | batch |
druid.coordinator.loadqueuepeon.type for segment management | http | curator |
druid.indexer.runner.type for the Overlord | httpRemote | local |
To use ZooKeeper instead of HTTP, change the values for the properties back to the previous defaults. ZooKeeper-based implementations for these properties are deprecated and will be removed in a subsequent release.
The aggregation functions for HLL and quantiles sketches returned sketches or numbers when they are finalized depending on where they were in the native query plan.
Druid no longer finalizes aggregators in the following two cases:
This change aligns the behavior of HLL and quantiles sketches with theta sketches.
To restore old behavior, you can set sqlFinalizeOuterSketches=true
in the query context.
When you issue a kill task, Druid marks the underlying segments as unused only if explicitly specified. For more information, see the API reference.
Apache Curator upgraded to the latest version, 5.3.0. This version drops support for ZooKeeper 3.4 but Druid has already officially dropped support in 0.22. In 5.3.0, Curator has removed support for Exhibitor so all related configurations and tests have been removed.
The behavior of the parquet reader for lists of structured objects has been changed to be consistent with other parquet logical list conversions. The data is now fetched directly, more closely matching its expected structure.
To read external data using the multi-stage query task engine, you must have READ permissions for the EXTERNAL resource type. Users without the correct permission encounter a 403 error when trying to run SQL queries that include EXTERN.
The way you assign the permission depends on your authorizer. For example, with basic security in Druid, add the EXTERNAL READ permission by sending a POST request to the roles API.
The example adds permissions for users with the admin role using a basic authorizer named MyBasicMetadataAuthorizer. The following permissions are granted:
curl --location --request POST 'http://localhost:8081/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/admin/permissions' \ --header 'Content-Type: application/json' \ --data-raw '[ { "resource": { "name": ".*", "type": "DATASOURCE" }, "action": "READ" }, { "resource": { "name": ".*", "type": "DATASOURCE" }, "action": "WRITE" }, { "resource": { "name": ".*", "type": "CONFIG" }, "action": "READ" }, { "resource": { "name": ".*", "type": "CONFIG" }, "action": "WRITE" }, { "resource": { "name": ".*", "type": "STATE" }, "action": "READ" }, { "resource": { "name": ".*", "type": "STATE" }, "action": "WRITE" }, { "resource": { "name": "EXTERNAL", "type": "EXTERNAL" }, "action": "READ" } ]'
Druid automatically retains any segments marked as unused. Previously, Druid permanently deleted unused segments from metadata store and deep storage after their duration to retain passed. This behavior was reverted from 0.23.0.
druid.processing.fifo
The default for druid.processing.fifo
is now true. This means that tasks of equal priority are treated in a FIFO manner. For most use cases, this change can improve performance on heavily loaded clusters.
In previous releases, Druid automatically closed the JDBC Statement when the ResultSet was closed. Druid closed the ResultSet on EOF. Druid closed the statement on any exception. This behavior is, however, non-standard. In this release, Druid's JDBC driver follows the JDBC standards more closely: The ResultSet closes automatically on EOF, but does not close the Statement or PreparedStatement. Your code must close these statements, perhaps by using a try-with-resources block. The PreparedStatement can now be used multiple times with different parameters. (Previously this was not true since closing the ResultSet closed the PreparedStatement.) If any call to a Statement or PreparedStatement raises an error, the client code must still explicitly close the statement. According to the JDBC standards, statements are not closed automatically on errors. This allows you to obtain information about a failed statement before closing it. If you have code that depended on the old behavior, you may have to change your code to add the required close statement.
In 0.23.0, Auto killing of segments is now enabled by default (#12187). The new defaults should kill all unused segments older than 90 days. If users do not want this behavior on an upgrade, they should explicitly disable the behavior. This is a risky change since depending on the interval, segments will be killed immediately after being marked unused. this behavior will be reverted or changed in the next druid release. Please see (#12693) for more details.
listShards
API access on the stream.percentOfSegmentsToConsiderPerMove
has been deprecated and will be removed in a future release of Druid. It is being replaced by a new segment picking strategy introduced in (#11257). This new strategy is currently toggled off by default, but can be toggled on if you set the dynamic coordinator config useBatchedSegmentSampler
to true. Setting this as such, will disable the use of the deprecated percentOfSegmentsToConsiderPerMove
. In a future release, useBatchedSegmentSampler
will become permanently true. (#11960)Following up to 0.21, which officially deprecated support for ZooKeeper 3.4, which has been end-of-life for a while, support for ZooKeeper 3.4 is now removed in 0.22.0. Be sure to upgrade your ZooKeeper cluster prior to upgrading your Druid cluster to 0.22.0.
Druid 0.22.0 includes an important bug-fix in native batch indexing where transient failures of indexing sub-tasks can result in non-contiguous partitions in the result segments, which will never become queryable due to logic which checks for the ‘complete’ set. This issue has been resolved in the latest version of Druid, but required a change in the protocol which batch tasks use to allocate segments, and this change can cause issues during rolling downgrades if you decide to roll back from Druid 0.22.0 to an earlier version.
To avoid task failure during a rolling-downgrade, set
druid.indexer.task.default.context={ "useLineageBasedSegmentAllocation" : false }
in the overlord runtime properties, and wait for all tasks which have useLineageBasedSegmentAllocation
set to true to complete before initiating the downgrade. After these tasks have all completed the downgrade shouldn't have any further issue and the setting can be removed from the overlord configuration (recommended, as you will want this setting enabled if you are running Druid 0.22.0 or newer).
Prior to Druid 0.22, an SQL group by query which is using a single universal grouping key (e.g. only aggregators) such as SELECT COUNT(*), SUM(x) FROM y WHERE z = 'someval'
would produce an empty result set instead of [0, null]
that might be expected from this query matching no results. This was because underneath this would plan into a timeseries query with ‘ALL’ granularity, and skipEmptyBuckets set to true in the query context. This latter option caused the results of such a query to return no results, as there are no buckets with values to aggregate and so they are skipped, making an empty result set instead of a ‘nil’ result set. This behavior has been changed to behave in line with other SQL implementations, but the previous behavior can be obtained by explicitly setting skipEmptyBuckets
on the query context.
Batch tasks using a ‘Druid’ input source to reingest segment data will no longer accept the ‘dimensions’ and ‘metrics’ sections of their task spec, and now will internally use a new columns filter to specify which columns from the original segment should be retained. Additionally, timestampSpec is no longer ignored, allowing the __time column to be modified or replaced with a different column. These changes additionally fix a bug where transformed columns would be ignored and unavailable on the new segments.
Some things might still work, but it is no longer officially supported so that newer Javascript features can be used to develop the web-console.
Druid coordinator maxSegmentsInNodeLoadingQueue
dynamic configuration has been changed from unlimited (0
) to 100
. This should make the coordinator behave in a much more relaxed manner during periods of cluster volatility, such as a rolling upgrade, but caps the total number of segments that will be loaded in any given coordinator cycle to 100 per server, which can slow down the speed at which a completely stopped cluster is started and loaded from deep storage.
Before this release, Druid returned the “internal error (500)” for most of the query errors. Now Druid returns different error codes based on their cause. The following table lists the errors and their corresponding codes that has changed:
Exception | Description | Old code | New code |
---|---|---|---|
SqlParseException and ValidationException from Calcite | Query planning failed | 500 | 400 |
QueryTimeoutException | Query execution didn't finish in timeout | 500 | 504 |
ResourceLimitExceededException | Query asked more resources than configured threshold | 500 | 400 |
InsufficientResourceException | Query failed to schedule because of lack of merge buffers available at the time when it was submitted | 500 | 429, merged to QueryCapacityExceededException |
QueryUnsupportedException | Unsupported functionality | 400 | 501 |
query/interrupted/count
no longer counts the queries that timed out. These queries are counted by query/timeout/count
.
context
is now a default dimension emitted for all query metrics. context
is a JSON-formatted string containing the query context for the query that the emitted metric refers to. The addition of a dimension that was not previously alters some metrics emitted by Druid. You should plan to handle this new context
dimension in your metrics pipeline. Since the dimension is a JSON-formatted string, a common solution is to parse the dimension and either flatten it or extract the bits you want and discard the full JSON-formatted string blob.
As ZooKeeper 3.4 has been end-of-life for a while, support for ZooKeeper 3.4 is deprecated in 0.21.0 and will be removed in the near future.
All columns in the sys.segments
table are now serialized in the JSON format to make them consistent with other system tables. Column names now use the same “snake case” convention.