druid/docs/querying/sql-query-context.md

88 lines
9.5 KiB
Markdown
Raw Normal View History

---
id: sql-query-context
title: "SQL query context"
sidebar_label: "SQL query context"
---
<!--
~ Licensed to the Apache Software Foundation (ASF) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. The ASF licenses this file
~ to you under the Apache License, Version 2.0 (the
~ "License"); you may not use this file except in compliance
~ with the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing,
~ software distributed under the License is distributed on an
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
~ KIND, either express or implied. See the License for the
~ specific language governing permissions and limitations
~ under the License.
-->
:::info
Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
This document describes the SQL language.
:::
Druid supports query context parameters which affect [SQL query](./sql.md) planning.
See [Query context](query-context.md) for general query context parameters for all query types.
## SQL query context parameters
Configure Druid SQL query planning using the parameters in the table below.
|Parameter|Description|Default value|
|---------|-----------|-------------|
|`sqlQueryId`|Unique identifier given to this SQL query. For HTTP client, it will be returned in `X-Druid-SQL-Query-Id` header.<br/><br/>To specify a unique identifier for SQL query, use `sqlQueryId` instead of [`queryId`](query-context.md). Setting `queryId` for a SQL request has no effect. All native queries underlying SQL use an auto-generated `queryId`.|auto-generated|
Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH. (#13247) * Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH. These aggregation functions are documented as creating sketches. However, they are planned into native aggregators that include finalization logic to convert the sketch to a number of some sort. This creates an inconsistency: the functions sometimes return sketches, and sometimes return numbers, depending on where they lie in the native query plan. This patch changes these SQL aggregators to _never_ finalize, by using the "shouldFinalize" feature of the native aggregators. It already existed for theta sketches. This patch adds the feature for hll and quantiles sketches. As to impact, Druid finalizes aggregators in two cases: - When they appear in the outer level of a query (not a subquery). - When they are used as input to an expression or finalizing-field-access post-aggregator (not any other kind of post-aggregator). With this patch, the functions will no longer be finalized in these cases. The second item is not likely to matter much. The SQL functions all declare return type OTHER, which would be usable as an input to any other function that makes sense and that would be planned into an expression. So, the main effect of this patch is the first item. To provide backwards compatibility with anyone that was depending on the old behavior, the patch adds a "sqlFinalizeOuterSketches" query context parameter that restores the old behavior. Other changes: 1) Move various argument-checking logic from runtime to planning time in DoublesSketchListArgBaseOperatorConversion, by adding an OperandTypeChecker. 2) Add various JsonIgnores to the sketches to simplify their JSON representations. 3) Allow chaining of ExpressionPostAggregators and other PostAggregators in the SQL layer. 4) Avoid unnecessary FieldAccessPostAggregator wrapping in the SQL layer, now that expressions can operate on complex inputs. 5) Adjust return type to thetaSketch (instead of OTHER) in ThetaSketchSetBaseOperatorConversion. * Fix benchmark class. * Fix compilation error. * Fix ThetaSketchSqlAggregatorTest. * Hopefully fix ITAutoCompactionTest. * Adjustment to ITAutoCompactionTest.
2022-11-03 12:43:00 -04:00
|`sqlTimeZone`|Sets the time zone for this connection, which will affect how time functions and timestamp literals behave. Should be a time zone name like "America/Los_Angeles" or offset like "-08:00".|`druid.sql.planner.sqlTimeZone` on the Broker (default: UTC)|
|`sqlStringifyArrays`|When set to true, result columns which return array values will be serialized into a JSON string in the response instead of as an array|true, except for JDBC connections, where it is always false|
|`useApproximateCountDistinct`|Whether to use an approximate cardinality algorithm for `COUNT(DISTINCT foo)`.|`druid.sql.planner.useApproximateCountDistinct` on the Broker (default: true)|
|`useGroupingSetForExactDistinct`|Whether to use grouping sets to execute queries with multiple exact distinct aggregations.|`druid.sql.planner.useGroupingSetForExactDistinct` on the Broker (default: false)|
|`useApproximateTopN`|Whether to use approximate [TopN queries](topnquery.md) when a SQL query could be expressed as such. If false, exact [GroupBy queries](groupbyquery.md) will be used instead.|`druid.sql.planner.useApproximateTopN` on the Broker (default: true)|
|`enableTimeBoundaryPlanning`|If true, SQL queries will get converted to TimeBoundary queries wherever possible. TimeBoundary queries are very efficient for min-max calculation on `__time` column in a datasource |`druid.query.default.context.enableTimeBoundaryPlanning` on the Broker (default: false)|
Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH. (#13247) * Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH. These aggregation functions are documented as creating sketches. However, they are planned into native aggregators that include finalization logic to convert the sketch to a number of some sort. This creates an inconsistency: the functions sometimes return sketches, and sometimes return numbers, depending on where they lie in the native query plan. This patch changes these SQL aggregators to _never_ finalize, by using the "shouldFinalize" feature of the native aggregators. It already existed for theta sketches. This patch adds the feature for hll and quantiles sketches. As to impact, Druid finalizes aggregators in two cases: - When they appear in the outer level of a query (not a subquery). - When they are used as input to an expression or finalizing-field-access post-aggregator (not any other kind of post-aggregator). With this patch, the functions will no longer be finalized in these cases. The second item is not likely to matter much. The SQL functions all declare return type OTHER, which would be usable as an input to any other function that makes sense and that would be planned into an expression. So, the main effect of this patch is the first item. To provide backwards compatibility with anyone that was depending on the old behavior, the patch adds a "sqlFinalizeOuterSketches" query context parameter that restores the old behavior. Other changes: 1) Move various argument-checking logic from runtime to planning time in DoublesSketchListArgBaseOperatorConversion, by adding an OperandTypeChecker. 2) Add various JsonIgnores to the sketches to simplify their JSON representations. 3) Allow chaining of ExpressionPostAggregators and other PostAggregators in the SQL layer. 4) Avoid unnecessary FieldAccessPostAggregator wrapping in the SQL layer, now that expressions can operate on complex inputs. 5) Adjust return type to thetaSketch (instead of OTHER) in ThetaSketchSetBaseOperatorConversion. * Fix benchmark class. * Fix compilation error. * Fix ThetaSketchSqlAggregatorTest. * Hopefully fix ITAutoCompactionTest. * Adjustment to ITAutoCompactionTest.
2022-11-03 12:43:00 -04:00
|`useNativeQueryExplain`|If true, `EXPLAIN PLAN FOR` will return the explain plan as a JSON representation of equivalent native query(s), else it will return the original version of explain plan generated by Calcite.<br /><br />This property is provided for backwards compatibility. It is not recommended to use this parameter unless you were depending on the older behavior.|`druid.sql.planner.useNativeQueryExplain` on the Broker (default: true)|
|`sqlFinalizeOuterSketches`|If false (default behavior in Druid 25.0.0 and later), `DS_HLL`, `DS_THETA`, and `DS_QUANTILES_SKETCH` return sketches in query results, as documented. If true (default behavior in Druid 24.0.1 and earlier), sketches from these functions are finalized when they appear in query results.<br /><br />This property is provided for backwards compatibility with behavior in Druid 24.0.1 and earlier. It is not recommended to use this parameter unless you were depending on the older behavior. Instead, use a function that does not return a sketch, such as `APPROX_COUNT_DISTINCT_DS_HLL`, `APPROX_COUNT_DISTINCT_DS_THETA`, `APPROX_QUANTILE_DS`, `DS_THETA_ESTIMATE`, or `DS_GET_QUANTILE`.|`druid.query.default.context.sqlFinalizeOuterSketches` on the Broker (default: false)|
|`sqlUseBoundAndSelectors`|If false (default behavior if `druid.generic.useDefaultValueForNull=false` in Druid 27.0.0 and later), the SQL planner will use [equality](./filters.md#equality-filter), [null](./filters.md#null-filter), and [range](./filters.md#range-filter) filters instead of [selector](./filters.md#selector-filter) and [bounds](./filters.md#bound-filter). This value must be set to `false` for correct behavior for filtering `ARRAY` typed values. | Defaults to same value as `druid.generic.useDefaultValueForNull`, which is `false`|
|`sqlReverseLookup`|Whether to consider the [reverse-lookup rewrite](lookups.md#reverse-lookup) of the `LOOKUP` function during SQL planning.<br /><br />Calls to `LOOKUP` are only reversed when the number of matching keys is lower than both `inSubQueryThreshold` and `sqlReverseLookupThreshold`.|true|
|`sqlReverseLookupThreshold`|Maximum size of `IN` filter to create when applying a [reverse-lookup rewrite](lookups.md#reverse-lookup). If a `LOOKUP` call matches more keys than this threshold, it is left as-is.<br /><br />If `inSubQueryThreshold` is lower than `sqlReverseLookupThreshold`, the `inSubQueryThreshold` is used as the threshold instead.|10000|
|`sqlPullUpLookup`|Whether to consider the [pull-up rewrite](lookups.md#pull-up) of the `LOOKUP` function during SQL planning.|true|
Segments sorted by non-time columns. (#16849) * Segments primarily sorted by non-time columns. Currently, segments are always sorted by __time, followed by the sort order provided by the user via dimensionsSpec or CLUSTERED BY. Sorting by __time enables efficient execution of queries involving time-ordering or granularity. Time-ordering is a simple matter of reading the rows in stored order, and granular cursors can be generated in streaming fashion. However, for various workloads, it's better for storage footprint and query performance to sort by arbitrary orders that do not start with __time. With this patch, users can sort segments by such orders. For spec-based ingestion, users add "useExplicitSegmentSortOrder: true" to dimensionsSpec. The "dimensions" list determines the sort order. To define a sort order that includes "__time", users explicitly include a dimension named "__time". For SQL-based ingestion, users set the context parameter "useExplicitSegmentSortOrder: true". The CLUSTERED BY clause is then used as the explicit segment sort order. In both cases, when the new "useExplicitSegmentSortOrder" parameter is false (the default), __time is implicitly prepended to the sort order, as it always was prior to this patch. The new parameter is experimental for two main reasons. First, such segments can cause errors when loaded by older servers, due to violating their expectations that timestamps are always monotonically increasing. Second, even on newer servers, not all queries can run on non-time-sorted segments. Scan queries involving time-ordering and any query involving granularity will not run. (To partially mitigate this, a currently-undocumented SQL feature "sqlUseGranularity" is provided. When set to false the SQL planner avoids using "granularity".) Changes on the write path: 1) DimensionsSpec can now optionally contain a __time dimension, which controls the placement of __time in the sort order. If not present, __time is considered to be first in the sort order, as it has always been. 2) IncrementalIndex and IndexMerger are updated to sort facts more flexibly; not always by time first. 3) Metadata (stored in metadata.drd) gains a "sortOrder" field. 4) MSQ can generate range-based shard specs even when not all columns are singly-valued strings. It merely stops accepting new clustering key fields when it encounters the first one that isn't a singly-valued string. This is useful because it enables range shard specs on "someDim" to be created for clauses like "CLUSTERED BY someDim, __time". Changes on the read path: 1) Add StorageAdapter#getSortOrder so query engines can tell how a segment is sorted. 2) Update QueryableIndexStorageAdapter, IncrementalIndexStorageAdapter, and VectorCursorGranularizer to throw errors when using granularities on non-time-ordered segments. 3) Update ScanQueryEngine to throw an error when using the time-ordering "order" parameter on non-time-ordered segments. 4) Update TimeBoundaryQueryRunnerFactory to perform a segment scan when running on a non-time-ordered segment. 5) Add "sqlUseGranularity" context parameter that causes the SQL planner to avoid using granularities other than ALL. Other changes: 1) Rename DimensionsSpec "hasCustomDimensions" to "hasFixedDimensions" and change the meaning subtly: it now returns true if the DimensionsSpec represents an unchanging list of dimensions, or false if there is some discovery happening. This is what call sites had expected anyway. * Fixups from CI. * Fixes. * Fix missing arg. * Additional changes. * Fix logic. * Fixes. * Fix test. * Adjust test. * Remove throws. * Fix styles. * Fix javadocs. * Cleanup. * Smoother handling of null ordering. * Fix tests. * Missed a spot on the merge. * Fixups. * Avoid needless Filters.and. * Add timeBoundaryInspector to test. * Fix tests. * Fix FrameStorageAdapterTest. * Fix various tests. * Use forceSegmentSortByTime instead of useExplicitSegmentSortOrder. * Pom fix. * Fix doc.
2024-08-23 11:24:43 -04:00
|`enableJoinLeftTableScanDirect`|This flag applies to queries which have joins. For joins, where left child is a simple scan with a filter, by default, druid will run the scan as a query and the join the results to the right child on broker. Setting this flag to true overrides that behavior and druid will attempt to push the join to data servers instead. Please note that the flag could be applicable to queries even if there is no explicit join. since queries can internally translated into a join by the SQL planner.|`false`|
|`maxNumericInFilters`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates only to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this parameter to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (123, 456)`. This value cannot exceed the set system configuration `druid.sql.planner.maxNumericInFilters`. This value is ignored if `druid.sql.planner.maxNumericInFilters` is not set explicitly.|`-1`|
|`inFunctionThreshold`| At or beyond this threshold number of values, SQL `IN` is converted to [`SCALAR_IN_ARRAY`](sql-functions.md#scalar_in_array). A threshold of 0 forces this conversion in all cases. A threshold of [Integer.MAX_VALUE] disables this conversion. The converted function is eligible for fewer planning-time optimizations, which speeds up planning, but may prevent certain planning-time optimizations.| `100`|
|`inFunctionExprThreshold`|At or beyond this threshold number of values, SQL `IN` is eligible for execution using the native function `scalar_in_array` rather than an <code>&#124;&#124;</code> of `==`, even if the number of values is below `inFunctionThreshold`. This property only affects translation of SQL `IN` to a [native expression](math-expr.md). It does not affect translation of SQL `IN` to a [native filter](filters.md). This property is provided for backwards compatibility purposes, and may be removed in a future release.|`2`|
|`inSubQueryThreshold`|At or beyond this threshold number of values, SQL `IN` is converted to `JOIN` on an inline table. `inFunctionThreshold` takes priority over this setting. A threshold of 0 forces usage of an inline table in all cases where the size of a SQL `IN` is larger than `inFunctionThreshold`. A threshold of `2147483647` disables the rewrite of SQL `IN` to `JOIN`. |`2147483647`|
## Setting the query context
The query context parameters can be specified as a "context" object in the [JSON API](../api-reference/sql-api.md) or as a [JDBC connection properties object](../api-reference/sql-jdbc.md).
See examples for each option below.
### Example using JSON API
```
{
"query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar' AND __time > TIMESTAMP '2000-01-01 00:00:00'",
"context" : {
"sqlTimeZone" : "America/Los_Angeles"
}
}
```
### Example using JDBC
```java
String url = "jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica/";
// Set any query context parameters you need here.
Properties connectionProperties = new Properties();
connectionProperties.setProperty("sqlTimeZone", "America/Los_Angeles");
connectionProperties.setProperty("useCache", "false");
try (Connection connection = DriverManager.getConnection(url, connectionProperties)) {
// create and execute statements, process result sets, etc
}
```