This commit contains the cleanup needed for the new integration test framework.
Changes:
- Fix log lines, misspellings, docs, etc.
- Allow the use of some of Druid's "JSON config" objects in tests
- Fix minor bug in `BaseNodeRoleWatcher`
SQL expressions such as those containing `MV_FILTER_ONLY` and `MV_FILTER_NONE`
are planned as specialized virtual columns instead of the default `expression`-type virtual columns.
This commit adds a new context parameter to force the `expression`-type virtual columns.
Changes
- Add query context param `forceExpressionVirtualColumns`
- Use context param to determine if specialized virtual columns should be used or not
- Moved some tests into `CalciteExplainQueryTest`
* Add TIME_IN_INTERVAL SQL operator.
The operator is implemented as a convertlet rather than an
OperatorConversion, because this allows it to be equivalent to using
the >= and < operators directly.
* SqlParserPos cannot be null here.
* Remove unused import.
* Doc updates.
* Add words to dictionary.
True, false, and null have different meanings: true/false mean "legacy"
and "not legacy"; null means use the default set by ScanQueryConfig.
So, we need to respect this in the JsonIgnore setup.
* Remove null and empty fields from native queries
* Test fixes
* Attempted IT fix.
* Revisions from review comments
* Build fixes resulting from changes suggested by reviews
* IT fix for changed segment size
Fixes an issue where sql query request logs do not include the default query context
values set via `druid.query.default.context.xyz` runtime properties.
# Change summary
* Inject `DefaultQueryConfig` into `SqlLifecycleFactory`
* Add params from `DefaultQueryConfig` to the query context in `SqlLifecycle`
# Description
- This change does not affect query execution. This is because the
`DefaultQueryConfig` was already being used in `QueryLifecycle`,
which is initialized when the SQL is translated to a native query.
- This also handles any potential use case where a context parameter should be
handled at the SQL stage itself.
RowBasedColumnSelectorFactory inherited strange behavior from
Rows.objectToStrings for nulls that appear in lists: instead of being
left as a null, it is replaced with the string "null". Some callers may
need compatibility with this strange behavior, but it should be opt-in.
Query-time call sites are changed to opt-out of this behavior, since it
is not consistent with query-time expectations. The IncrementalIndex
ingestion-time call site retains the old behavior, as this is traditionally
when Rows.objectToStrings would be used.
Description
Fixes a bug when running q's like
SELECT cntarray,
Count(*)
FROM (SELECT dim1,
dim2,
Array_agg(cnt) AS cntarray
FROM (SELECT dim1,
dim2,
dim3,
Count(*) AS cnt
FROM foo
GROUP BY 1,
2,
3)
GROUP BY 1,
2)
GROUP BY 1
This generates an error:
org.apache.druid.java.util.common.ISE: Unable to convert type [Ljava.lang.Object; to org.apache.druid.segment.data.ComparableList
at org.apache.druid.segment.DimensionHandlerUtils.convertToList(DimensionHandlerUtils.java:405) ~[druid-xx]
Because it's an array of numbers it looks like it does the convertToList call, which looks like:
@Nullable
public static ComparableList convertToList(Object obj)
{
if (obj == null) {
return null;
}
if (obj instanceof List) {
return new ComparableList((List) obj);
}
if (obj instanceof ComparableList) {
return (ComparableList) obj;
}
throw new ISE("Unable to convert type %s to %s", obj.getClass().getName(), ComparableList.class.getName());
}
I.e. it doesn't know about arrays. Added the array handling as part of this PR.
In the case that the clustered by is before the partitioned by for an sql query, the error message is a bit confusing.
insert into foo select * from bar clustered by dim1 partitioned by all
Error: SQL parse failed
Encountered "PARTITIONED" at line 1, column 88.
Was expecting one of: <EOF> "," ... "ASC" ... "DESC" ... "NULLS" ... "." ... "NOT" ... "IN" ... "<" ... "<=" ... ">" ... ">=" ... "=" ... "<>" ... "!=" ... "BETWEEN" ... "LIKE" ... "SIMILAR" ... "+" ... "-" ... "*" ... "/" ... "%" ... "||" ... "AND" ... "OR" ... "IS" ... "MEMBER" ... "SUBMULTISET" ... "CONTAINS" ... "OVERLAPS" ... "EQUALS" ... "PRECEDES" ... "SUCCEEDS" ... "IMMEDIATELY" ... "MULTISET" ... "[" ... "FORMAT" ... "(" ... Less...
org.apache.calcite.sql.parser.SqlParseException
This is a bit confusing and adding a check could be added to throw a more user friendly message stating that the order should be reversed.
Add error message for incorrectly ordered clause in sql.
* Direct UTF-8 access for "in" filters.
Directly related:
1) InDimFilter: Store stored Strings (in ValuesSet) plus sorted UTF-8
ByteBuffers (in valuesUtf8). Use valuesUtf8 whenever possible. If
necessary, the input set is copied into a ValuesSet. Much logic is
simplified, because we always know what type the values set will be.
I think that there won't even be an efficiency loss in most cases.
InDimFilter is most frequently created by deserialization, and this
patch updates the JsonCreator constructor to deserialize
directly into a ValuesSet.
2) Add Utf8ValueSetIndex, which InDimFilter uses to avoid UTF-8 decodes
during index lookups.
3) Add unsigned comparator to ByteBufferUtils and use it in
GenericIndexed.BYTE_BUFFER_STRATEGY. This is important because UTF-8
bytes can be compared as bytes if, and only if, the comparison
is unsigned.
4) Add specialization to GenericIndexed.singleThreaded().indexOf that
avoids needless ByteBuffer allocations.
5) Clarify that objects returned by ColumnIndexSupplier.as are not
thread-safe. DictionaryEncodedStringIndexSupplier now calls
singleThreaded() on all relevant GenericIndexed objects, saving
a ByteBuffer allocation per access.
Also:
1) Fix performance regression in LikeFilter: since #12315, it applied
the suffix matcher to all values in range even for type MATCH_ALL.
2) Add ObjectStrategy.canCompare() method. This fixes LikeFilterBenchmark,
which was broken due to calls to strategy.compare in
GenericIndexed.fromIterable.
* Add like-filter implementation tests.
* Add in-filter implementation tests.
* Add tests, fix issues.
* Fix style.
* Adjustments from review.
* SQL: Add is_active to sys.segments, update examples and docs.
is_active is short for:
(is_published = 1 AND is_overshadowed = 0) OR is_realtime = 1
It's important because this represents "all the segments that should
be queryable, whether or not they actually are right now". Most of the
time, this is the set of segments that people will want to look at.
The web console already adds this filter to a lot of its queries,
proving its usefulness.
This patch also reworks the caveat at the bottom of the sys.segments
section, so its information is mixed into the description of each result
field. This should make it more likely for people to see the information.
* Wording updates.
* Adjustments for spellcheck.
* Adjust IT.
- Add user friendly error messages for missing or incorrect OVERWRITE clause for REPLACE SQL query
- Move validation of missing OVERWRITE clause at code level instead of parser for custom error message
Relevant Issue: #11929
- Add custom replace statement to Druid SQL parser.
- Edit DruidPlanner to convert relevant fields to Query Context.
- Refactor common code with INSERT statements to reuse them for REPLACE where possible.
Following up on #12315, which pushed most of the logic of building ImmutableBitmap into BitmapIndex in order to hide the details of how column indexes are implemented from the Filter implementations, this PR totally refashions how Filter consume indexes. The end result, while a rather dramatic reshuffling of the existing code, should be extraordinarily flexible, eventually allowing us to model any type of index we can imagine, and providing the machinery to build the filters that use them, while also allowing for other column implementations to implement the built-in index types to provide adapters to make use indexing in the current set filters that Druid provides.
* Add feature flag for sql planning of TimeBoundary queries
* fixup! Add feature flag for sql planning of TimeBoundary queries
* Add documentation for enableTimeBoundaryPlanning
* fixup! Add documentation for enableTimeBoundaryPlanning
* Vectorized version of string last aggregator
* Updating string last and adding testcases
* Updating code and adding testcases for serializable pairs
* Addressing review comments
* Reduce allocations due to Jackson serialization.
This patch attacks two sources of allocations during Jackson
serialization:
1) ObjectMapper.writeValue and JsonGenerator.writeObject create a new
DefaultSerializerProvider instance for each call. It has lots of
fields and creates pressure on the garbage collector. So, this patch
adds helper functions in JacksonUtils that enable reuse of
SerializerProvider objects and updates various call sites to make
use of this.
2) GroupByQueryToolChest copies the ObjectMapper for every query to
install a special module that supports backwards compatibility with
map-based rows. This isn't needed if resultAsArray is set and
all servers are running Druid 0.16.0 or later. This release was a
while ago. So, this patch disables backwards compatibility by default,
which eliminates the need to copy the heavyweight ObjectMapper. The
patch also introduces a configuration option that allows admins to
explicitly enable backwards compatibility.
* Add test.
* Update additional call sites and add to forbidden APIs.
* SQL: Create millisecond precision timestamp literals.
Fixes a bug where implicit casts of strings to timestamps would use seconds
precision rather than milliseconds. The new test case
testCountStarWithBetweenTimeFilterUsingMillisecondsInStringLiterals
exercises this.
* Update sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java
Co-authored-by: Frank Chen <frankchen@apache.org>
* Correct precision handling.
- Set default precision to 3 (millis) for things involving timestamps.
- Respect precision specified in types when available.
* Silence, checkstyle.
Co-authored-by: Frank Chen <frankchen@apache.org>
Unnamed columns in the select part of insert SQL statements currently create a table with the column name such as "EXPR$3". This PR adds a check for this.
* Vectorizing Latest aggregator Part 1
* Updating benchmark tests
* Changing appropriate logic for vectors for null handling
* Introducing an abstract class and moving the commonalities there
* Adding vectorization for StringLast aggregator (initial version)
* Updated bufferized version of numeric aggregators
* Adding some javadocs
* Making sure this PR vectorizes numeric latest agg only
* Adding another benchmarking test
* Fixing intellij inspections
* Adding tests for double
* Adding test cases for long and float
* Updating testcases
* Checkstyle oops..
* One tiny change in test case
* Fixing spotbug and rhs not being used
* Support array based results in timeBoundary query
* Fix bug with query interval in timeBoundary
* Convert min(__time) and max(__time) SQL queries to timeBoundary
* Add tests for timeBoundary backed SQL queries
* Fix query plans for existing tests
* fixup! Convert min(__time) and max(__time) SQL queries to timeBoundary
* fixup! Add tests for timeBoundary backed SQL queries
* fixup! Fix bug with query interval in timeBoundary
The query context is a way that the user gives a hint to the Druid query engine, so that they enforce a certain behavior or at least let the query engine prefer a certain plan during query planning. Today, there are 3 types of query context params as below.
Default context params. They are set via druid.query.default.context in runtime properties. Any user context params can be default params.
User context params. They are set in the user query request. See https://druid.apache.org/docs/latest/querying/query-context.html for parameters.
System context params. They are set by the Druid query engine during query processing. These params override other context params.
Today, any context params are allowed to users. This can cause
1) a bad UX if the context param is not matured yet or
2) even query failure or system fault in the worst case if a sensitive param is abused, ex) maxSubqueryRows.
This PR adds an ability to limit context params per user role. That means, a query will fail if you have a context param set in the query that is not allowed to you. To do that, this PR adds a new built-in resource type, QUERY_CONTEXT. The resource to authorize has a name of the context param (such as maxSubqueryRows) and the type of QUERY_CONTEXT. To allow a certain context param for a user, the user should be granted WRITE permission on the context param resource. Here is an example of the permission.
{
"resourceAction" : {
"resource" : {
"name" : "maxSubqueryRows",
"type" : "QUERY_CONTEXT"
},
"action" : "WRITE"
},
"resourceNamePattern" : "maxSubqueryRows"
}
Each role can have multiple permissions for context params. Each permission should be set for different context params.
When a query is issued with a query context X, the query will fail if the user who issued the query does not have WRITE permission on the query context X. In this case,
HTTP endpoints will return 403 response code.
JDBC will throw ForbiddenException.
Note: there is a context param called brokerService that is used only by the router. This param is used to pin your query to run it in a specific broker. Because the authorization is done not in the router, but in the broker, if you have brokerService set in your query without a proper permission, your query will fail in the broker after routing is done. Technically, this is not right because the authorization is checked after the context param takes effect. However, this should not cause any user-facing issue and thus should be OK. The query will still fail if the user doesn’t have permission for brokerService.
The context param authorization can be enabled using druid.auth.authorizeQueryContextParams. This is disabled by default to avoid any hassle when someone upgrades his cluster blindly without reading release notes.
For a query like
INSERT INTO tablename SELECT channel, added as count FROM wikipedia the error message is Encountered "as count". However, for the insert statement
INSERT INTO t SELECT channel, added as count FROM wikipedia PARTITIONED BY ALL
returns INSERT statements must specify PARTITIONED BY clause explictly (incorrectly). This PR corrects this.
Add EOF to end of Druid SQL Insert statements
Rename SQL Insert statements in the parser to reflect the behaviour change
Added Calcites InQueryThreshold as a query context parameter. Setting this parameter appropriately reduces the time taken for queries with large number of values in their IN conditions.
* Fix error message for groupByEnableMultiValueUnnesting.
It referred to the incorrect context parameter.
Also, create a dedicated exception class, to allow easier detection of this
specific error.
* Fix other test.
* More better error messages.
* Test getDimensionName method.
* upgrade Airline to Airline 2
https://github.com/airlift/airline is no longer maintained, updating to
https://github.com/rvesse/airline (Airline 2) to use an actively
maintained version, while minimizing breaking changes.
Note, this is a backwards incompatible change, and extensions relying on
the CliCommandCreator extension point will also need to be updated.
* fix dependency checks where jakarta.inject is now resolved first instead
of javax.inject, due to Airline 2 using jakarta
As part of #12078 one of the followup's was to have a specific config which does not allow accidental unnesting of multi value columns if such columns become part of the grouping key.
Added a config groupByEnableMultiValueUnnesting which can be set in the query context.
The default value of groupByEnableMultiValueUnnesting is true, therefore it does not change the current engine behavior.
If groupByEnableMultiValueUnnesting is set to false, the query will fail if it encounters a multi-value column in the grouping key.
* Moving in filter check to broker
* Adding more unit tests, making error message meaningful
* Spelling and doc changes
* Updating default to -1 and making this feature hide by default. The number of IN filters can grow upto a max limit of 100
* Removing upper limit of 100, updated docs
* Making documentation more meaningful
* Moving check outside to PlannerConfig, updating test cases and adding back max limit
* Updated with some additional code comments
* Missed removing one line during the checkin
* Addressing doc changes and one forbidden API correction
* Final doc change
* Adding a speling exception, correcting a testcase
* Reading entire filter tree to address combinations of ANDs and ORs
* Specifying in docs that, this case works only for ORs
* Revert "Reading entire filter tree to address combinations of ANDs and ORs"
This reverts commit 81ca8f8496.
* Covering a class cast exception and updating docs
* Counting changed
Co-authored-by: Jihoon Son <jihoonson@apache.org>
#12163 makes PARTITIONED BY a required clause in INSERT queries. While this is required, if a user accidentally omits the clause, it emits a JavaCC/Calcite error, since it's syntactically incorrect. The error message is cryptic. Since it's a custom clause, this PR aims to make the clause optional on the syntactic side, but move the validation to DruidSqlInsert where we can surface a friendlier error.
* rework sql planner expression and virtual column handling
* simplify a bit
* add back and deprecate old methods, more tests, fix multi-value string coercion bug and associated tests
* spotbugs
* fix bugs with multi-value string array expression handling
* javadocs and adjust test
* better
* fix tests
* array_concat_agg and array_agg support for array inputs
changes:
* added array_concat_agg to aggregate arrays into a single array
* added array_agg support for array inputs to make nested array
* added 'shouldAggregateNullInputs' and 'shouldCombineAggregateNullInputs' to fix a correctness issue with STRING_AGG and ARRAY_AGG when merging results, with dual purpose of being an optimization for aggregating
* fix test
* tie capabilities type to legacy mode flag about coercing arrays to strings
* oops
* better javadoc
* changes:
* remove SystemSchema duplicate ServerInventoryView in broker
* suppress duplicate segment added/removed warnings in HttpServerInventoryView when doing a full sync
* fixes
Fixes a bug because of which some SQL queries cannot be parsed using druid convention. Specifically, these queries translate to an inline datasource and have some null values. Calcite internally uses NULL as SQL type for these literals and that is not supported by the druid.
I am now allowing null column types to be returned while building RowSignature in org.apache.druid.sql.calcite.table.RowSignatures#fromRelDataType. RowSignature already allows null column type for any column. Doing so should also fix bindable queries such as select (1,2). When such queries are run with headers set to true, we get an exception in org.apache.druid.sql.http.ArrayWriter#writeHeader. This is again a similar exception to the one addressed in this PR. Because SQL type for the result column is RECORD and that doesn't have a corresponding columnType.
* init multiValue column group by
* Changing sorting to Lexicographic as default
* Adding initial tests
* 1.Fixing test cases adding
2.Optimized inmem structs
* Linking SQL layer to native layer
* Adding multiDimension support to group by column strategy
* 1. Removing array coercion in Calcite layer
2. Removing ResultRowDeserializer
* 1. Supporting all primitive array types
2. Removing dimension spec as part of columnSelector
* 1. Supporting all primitive array types
2. Removing dimension spec as part of columnSelector
* 1. Checkstyle things
2. Removing flag
* Minor naming things
* CheckStyle Things
* Fixing test case
* Fixing hashing
* 1. Adding the MV function
2. Added few test cases
* 1. Adding MV function test cases
* Adding Selector strategy function test cases
* Fixing ClientQuerySegmentWalkerTest
* Adding GroupByQueryRunnerTest test cases
* Fixing test cases
* Adding few more test cases
* Fixing Exception asset statement and intellij inspection
* Adding null compatibility tests
* Review comments
* Fixing few failing tests
* Fixing few failing tests
* Do no convert to topN Q incase of group by on array
* Fixing checkstyle
* Fixing differences between jdk's class cast exception message
* 1. Fixing ordering if the grouping key is an array
* Fixing DefaultLimitSpec
* Fixing CalciteArraysQueryTest
* Dummy commit for LGTM
* changes:
* only coerce multi-value string null values when `ExpressionPlan.Trait.NEEDS_APPLIED` is set
* correct return type inference for ARRAY_APPEND,ARRAY_PREPEND,ARRAY_SLICE,ARRAY_CONCAT
* fix bug with ExprEval.ofType when actual type of object from binding doesn't match its claimed type
* Review comments
* Fixing test cases
* Fixing spot bugs
* Fixing strict compile
Co-authored-by: Clint Wylie <cwylie@apache.org>
This PR changes the value of the property `druid.sql.planner.useGroupingSetForExactDistinct` from `false` to `true` in the runtime.properties files, so that newer installations have this property as `true`, while the default still remains as `false`.
The flag determines how queries which contain an aggregation over `DISTINCT` like `SELECT COUNT(DISTINCT foo.dim1) FILTER(WHERE foo.cnt = 1), SUM(foo.cnt) FROM druid.foo` get planned by Calcite. With the flag being set to false, it plans it via joins, whereas with it being set to true, the query is set using grouping sets.
There is a known issue with Calcite (https://github.com/apache/druid/issues/7953), where an NPE is thrown while planning the above query with joins. There is no such issue while planning the query using grouping sets.
* Pass VirtualColumnRegistry in PlannerContext for join expression planning
* Allow for including VCs from join fact table expression
* Optmize MV_FILTER functions to use a VC when in join fact table expression
* fixup! Allow for including VCs from join fact table expression
* Address review comments
Related to #11188
The above mentioned PR allowed timeseries queries to return a default result, when queries of type: select count(*) from table where dim1="_not_present_dim_" were executed. Before the PR, it returned no row, after the PR, it would return a row with value of count(*) as 0 (as expected by SQL standards of different dbs).
In Grouping#applyProject, we can sometimes perform optimization of a groupBy query to a timeseries query if possible (when the keys of the groupBy are constants, as generated by automated tools). For example, in select count(*) from table where dim1="_present_dim_" group by "dummy_key", the groupBy clause can be removed. However, in the case when the filter doesn't return anything, i.e. select count(*) from table where dim1="_not_present_dim_" group by "dummy_key", the behavior of general databases would be to return nothing, while druid (due to above change) returns an empty row. This PR aims to fix this divergence of behavior.
Example cases:
select count(*) from table where dim1="_not_present_dim_" group by "dummy_key".
CURRENT: Returns a row with count(*) = 0
EXPECTED: Return no row
select 'A', dim1 from foo where m1 = 123123 and dim1 = '_not_present_again_' group by dim1
CURRENT: Returns a row with ('A', 'wat')
EXPECTED: Return no row
To do this, a boolean droppedDimensionsWhileApplyingProject has been added to Grouping which is true whenever we make changes to the original shape with optimization. Hence if a timeseries query has a grouping with this set to true, we set skipEmptyBuckets=true in the query context (i.e. donot return any row).
DruidLogicalValuesRule while transforming to DruidRel can return incorrect values, if during the creation of the literal it was created from a float value. The BigDecimal representation stores 123.0, and it seems that using RexLiteral's method while conversion returns the inflated value (which is 1230). I am unsure if this is intentional from Calcite's perspective, and the actual change should be done somewhere else.
Extract the values of INT/LONG from the RexLiteral in the DruidLogicalValuesRule, via BigDecimal.longValue() method.
changes:
* IncrementalIndex is now a ColumnInspector
* fixes performance regression from using map of ColumnCapabilities from IncrementalIndex as a RowSignature
In this PR, we will now return 400 instead of 500 when SQL query cannot be planned. I also fixed a bug where error messages were not getting sent to the users in case the rules throw UnsupportSQLQueryException.
DruidSchema consists of a concurrent HashMap of DataSource -> Segement -> AvailableSegmentMetadata. AvailableSegmentMetadata contains RowSignature of the segment, and for each segment, a new object is getting created. RowSignature is an immutable class, and hence it can be interned, and this can lead to huge savings of memory being used in broker, since a lot of the segments of a table would potentially have same RowSignature.
This PR does two things
1. It adds the capability to surface missing features in SQL to users - The calcite planner will explore through multiple rules to convert a logical SQL query to a druid native query. Some rules change the shape of the query itself, optimize it and some rules are responsible for translating the query into a druid native query. These are DruidQueryRule, DruidOuterQueryRule, DruidJoinRule, DruidUnionDataSourceRule, DruidUnionRule etc. These rules will look at SQL and will do the necessary transformation. But if the rule can't transform the query, it returns back the control to the calcite planner without recording why was it not able to transform. E.g. there is a join query with a non-equal join condition. DruidJoinRule will look at the condition, see that it is not supported, and return back the control. The reason can be that a query can be planned in many different ways so if one rule can't parse it, the query may still be parseable by other rules. In this PR, we are intercepting these gaps and passing them back to the user if the query could not be planned at all.
2. The said capability has been used to generate actionable errors for some common unsupported SQL features. However, not all possible errors are covered and we can keep adding more in the future.
Druid currently has 2 serverViews, regular serverView and filtered serverView. The regular serverView is used to monitor all segment announcements from all data nodes (historicals, tasks, indexers). The filtered serverView is used when you want to watch segment announcements from particular tiers. Since these server views keep track of different sets of druidServers and segments in memory, they should be maintained separately. However, they currently share the same name for their executorService, which can cause confusion and make debugging harder especially in the broker since it is using both serverViews, the filtered view for normal query processing and the regular view to serve the servers table (I'm unsure whether this is intended or whether this is a good behavior). This PR changes it to a more obvious name.
This PR also removes SingleServerInventoryView. This view was deprecated a long time ago and has not been documented at least since 0.13 (#6127). I also don't think this can be better in any case than BatchServerInventoryView. Finally, I merged AbstractCuratorServerInventoryView and BatchServerInventoryView as we no longer need AbstractCuratorServerInventoryView after SingleServerInventoryView is removed.
* Enhancements to IndexTaskClient.
1) Ability to use handlers other than StringFullResponseHandler. This
functionality is not used in production code yet, but is useful
because it will allow tasks to communicate with each other in
non-string-based formats and in streaming fashion. In the future,
we'll be able to use this to make task-to-task communication
more efficient.
2) Truncate server errors at 1KB, so long errors do not pollute logs.
3) Change error log level for retryable errors from WARN to INFO. (The
final error is still WARN.)
4) Harmonize log and exception messages to have a more consistent format.
* Additional tests and improvements.
* allow `DruidSchema` to fallback to segment metadata type if typeSignature is null, to avoid producing incorrect SQL schema if broker is upgraded to 0.23 before historicals
* mmm, forbidden tests
changes:
* adds new config, druid.expressions.useStrictBooleans which make longs the official boolean type of all expressions
* vectorize logical operators and boolean functions, some only if useStrictBooleans is true
* Code cleanup from query profile project
* Fix spelling errors
* Fix Javadoc formatting
* Abstract out repeated test code
* Reuse constants in place of some string literals
* Fix up some parameterized types
* Reduce warnings reported by Eclipse
* Reverted change due to lack of tests
Currently, when we try to do EXPLAIN PLAN FOR, it returns the structure of the SQL parsed (via Calcite's internal planner util), which is verbose (since it tries to explain about the nodes in the SQL, instead of the Druid Query), and not representative of the native Druid query which will get executed on the broker side.
This PR aims to change the format when user tries to EXPLAIN PLAN FOR for queries which are executed by converting them into Druid's native queries (i.e. not sys schemas).
Add the ability to pass time column in first/last aggregator (and latest/earliest SQL functions). It is to support cases where the time to query upon is stored as a part of a column different than __time. Also, some other logical time column can be specified.
* SQL INSERT planner support.
The main changes are:
1) DruidPlanner is able to validate and authorize INSERT queries. They
require WRITE permission on the target datasource.
2) QueryMaker is now an interface, and there is a QueryMakerFactory that
creates instances of it. There is only one production implementation
of each (NativeQueryMaker and NativeQueryMakerFactory), which
together behave the same way as the former QueryMaker class. But this
opens the door to executing queries in ways other than the Druid
query stack, and is used by unit tests (CalciteInsertDmlTest) to
test the INSERT planning functionality.
3) Adds an EXTERN table macro that allows references external data using
InputSource and InputFormat from Druid's batch ingestion API. This is
not exposed in production yet, but is used by unit tests.
4) Adds a QueryFeature concept that enables the planner to change its
behavior slightly depending on the capabilities of the execution
system.
5) Adds an "AuthorizableOperator" concept that enables SqlOperators
to require additional permissions. This is used by the EXTERN table
macro.
Related odds and ends:
- Add equals, hashCode, toString methods to InlineInputSource. Aids in
the "from external" tests in CalciteInsertDmlTest.
- Add JSON-serializability to RowSignature.
- Move the SQL string inside PlannerContext so it is "baked into" the
planner when the planner is created. Cleans up the code a bit, since
in practice, the same query is passed in every time to the
same planner anyway.
* Fix up calls to CalciteTests.createMockQueryLifecycleFactory.
* Fix checkstyle issues.
* Adjustments for CI.
* Adjust DruidAvaticaHandlerTest for stricter test authorizations.
* add impl
* fix checkstyle
* add test
* add test
* add unit tests
* fix unit tests
* fix unit tests
* fix unit tests
* add IT
* add IT
* add comments
* fix spelling
DruidRexExecutor while reducing Arrays, specially numeric arrays, doesn't convert the value from ExprResult's type to BigDecimal, which causes makeLiteral to cast the values. Also, if NaN or Infinite values are present in the array, the error is a generic NumberFormatException. For example:
SELECT ARRAY[1.11, 2.22] returns [1, 2]
SELECT SQRT(-1) throws a generic NumberFormatException instead of IAE
This PR introduces change to cast the numeric values to BigDecimal since Calcite's library understands that easily, and doesn't perform casts.
* Scan: Add "orderBy" parameter.
This patch adds an API for requesting non-time orderings, although it
does not actually add the ability to execute such queries.
The changes are done in such a way that no matter how Scan query objects
are constructed, they will have a correct "getOrderBy". This will enable
us to switch the execution to exclusively use "getOrderBy" later on when
it's implemented.
Scan queries are serialized such that they only include "order" (time
order) if the ordering is time-based, and they only include "orderBy" if
the ordering is non-time-based. This maximizes compatibility with
the existing API while also providing a clean look for formatted queries.
Because this patch does not include execution logic, if someone actually
tries to run a query with non-time ordering, then they will get an error
like "Cannot execute query with orderBy [quality ASC]".
* SQL module fixes.
* Add spotbugs-exclude.
* Remove unused method.
* IMPLY-4344: Adding safe divide function along with testcases and documentation updates
* Changing based on review comments
* Addressing review comments, fixing coding style, docs and spelling
* Checkstyle passes for all code
* Fixing expected results for infinity
* Revert "Fixing expected results for infinity"
This reverts commit 5fd5cd480d.
* Updating test result and a space in docs
Unlike a real one, TestServerInventoryView would call segmentRemoved
any time _any_ segment was removed. It should only be called when _all_
segments have been removed.
* Use a simple class to sanitize sanitizable errors and log them
The purpose of this is to sanitize JDBC errors, but can sanitize other errors
if they implement SanitizableError Interface
add a class to log errors and sanitize them
added a simple test that tests out that the error gets sanitized
add @NonNull annotation to serverconfig's ErrorResponseTransfromStrategy
* return less information as part of too many connections, and instead only log specific details
This is so an end user gets relevant information but not too much info since they might now how
many brokers they have
* return only runtime exceptions
added new error types that need to be sanitized
also sanitize deprecated and unsupported exceptions.
* dont reqrewite exceptions unless necessary for checked exceptions
add docs
avoid blanket turning all exceptions into runtime exceptions
* address comments, to fix up docs.
add more javadocs
add support UOE sanitization
* use try catch instead and sanitize at public methods
* checkstyle fixes
* throw noSuchStatement and NoSuchConnection as Avatica is affected by those
* address comments. move log error back to druid meta
clean up bad formatting and commented code. add missed catch for NoSuchStatementException
clean up comments for error handler and add comment explainging not wanting to santize avatica exceptions
* alter test to reflect new error message
* revert ColumnAnalysis type, add typeSignature and use it for DruidSchema
* review stuffs
* maybe null
* better maybe null
* Update docs/querying/segmentmetadataquery.md
* Update docs/querying/segmentmetadataquery.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* fix null right
* sad
* oops
* Update batch_hadoop_queries.json
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* add missing json type for ListFilteredVirtualColumn, and tests to try to avoid this happening again
* fixes
* ugly, but maybe this
* oops
* too many mappers
* complex typed expressions
* add built-in hll collector expressions to get coverage on druid-processing, more types, more better
* rampage!!!
* more javadoc
* adjustments
* oops
* lol
* remove unused dependency
* contradiction?
* more test
* Add druid.sql.approxCountDistinct.function property.
The new property allows admins to configure the implementation for
APPROX_COUNT_DISTINCT and COUNT(DISTINCT expr) in approximate mode.
The motivation for adding this setting is to enable site admins to
switch the default HLL implementation to DataSketches.
For example, an admin can set:
druid.sql.approxCountDistinct.function = APPROX_COUNT_DISTINCT_DS_HLL
* Fixes
* Fix tests.
* Remove erroneous cannotVectorize.
* Remove unused import.
* Remove unused test imports.
* Revert "Require Datasource WRITE authorization for Supervisor and Task access (#11718)"
This reverts commit f2d6100124.
* Revert "Require DATASOURCE WRITE access in SupervisorResourceFilter and TaskResourceFilter (#11680)"
This reverts commit 6779c4652d.
* Fix docs for the reverted commits
* Fix and restore deleted tests
* Fix and restore SystemSchemaTest
* SQL: Allow Scans to be used as outer queries.
This has been possible in the native query system for a while, but the capability
hasn't yet propagated into the SQL layer. One example of where this is useful is
a query like:
SELECT * FROM (... LIMIT X) WHERE <filter>
Because this expands the kinds of subquery structures the SQL layer will consider,
it was also necessary to improve the cost calculations. These changes appear in
PartialDruidQuery and DruidOuterQueryRel. The ideas are:
- Attach per-column penalties to the output signature of each query, instead of to
the initial projection that starts a query. This encourages moving projections
into subqueries instead of leaving them on outer queries.
- Only attach penalties to projections if there are actually expressions happening.
So, now, projections that simply reorder or remove fields are free.
- Attach a constant penalty to every outer query. This discourages creating them
when they are not needed.
The changes are generally beneficial to the test cases we have in CalciteQueryTest.
Most plans are unchanged, or are changed in purely cosmetic ways. Two have changed
for the better:
- testUsingSubqueryWithLimit now returns a constant from the subquery, instead of
returning every column.
- testJoinOuterGroupByAndSubqueryHasLimit returns a minimal set of columns from
the innermost subquery; two unnecessary columns are no longer there.
* Fix various DS operator conversions.
These were all implemented as direct conversions, which isn't appropriate
because they do not actually map onto native functions. These are only
usable as post-aggregations.
* Test case adjustment.
* better type system
* needle in a haystack
* ColumnCapabilities is a TypeSignature instead of having one, INFORMATION_SCHEMA support
* fixup merge
* more test
* fixup
* intern
* fix
* oops
* oops again
* ...
* more test coverage
* fix error message
* adjust interning, more javadocs
* oops
* more docs more better
Follow up PR for #11680
Description
Supervisor and Task APIs are related to ingestion and must always require Datasource WRITE
authorization even if they are purely informative.
Changes
Check Datasource WRITE in SystemSchema for tables "supervisors" and "tasks"
Check Datasource WRITE for APIs /supervisor/history and /supervisor/{id}/history
Check Datasource for all Indexing Task APIs
* Add the ability to add a context to internally generated druid broker queries
* fix docs
* changes after first CI failure
* cleanup after merge with master
* change default to empty map and improve unit tests
* add doc info and fix checkstyle
* refactor DruidSchema#runSegmentMetadataQuery and add a unit test
* refactor sql authorization to get resource type from schema, refactor resource type from enum to string
* information schema auth filtering adjustments
* refactor
* minor stuff
* Update SqlResourceCollectorShuttle.java
* add MV_FILTER_ONLY SQL function, and list filter virtual column
* MV_FILTER_NONE and more tests
* formatting
* o yeah, forgot can do easy thing
* style
* hmm why was that there
* test filtering on virtual column
* style
* meh
* do it right
* good bot
The SQL "array" and "object" formats are intended to return invalid JSON
(lacking a ] terminator) if an error occurs midstream. This enables callers
to detect truncated responses. But JsonGenerators, by default, close JSON
arrays even when not explicitly told to.
This patch disables automatic array closing, which fixes the problem with
truncated response detection. It also adds tests for truncated responses
for all result formats.
* initial work
* reduce lock in sqlLifecycle
* Integration test for sql canceling
* javadoc, cleanup, more tests
* log level to debug
* fix test
* checkstyle
* fix flaky test; address comments
* rowTransformer
* cancelled state
* use lock
* explode instead of noop
* oops
* unused import
* less aggressive with state
* fix calcite charset
* don't emit metrics when you are not authorized
* Configurable maxStreamLength for doubles sketches
* fix equals/hashcode and it test failure
* fix test
* fix it test
* benchmark
* doc
* grouping key
* fix comment
* dependency check
* Update docs/development/extensions-core/datasketches-quantiles.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* add binary_byte_format/decimal_byte_format/decimal_format
* clean code
* fix doc
* fix review comments
* add spelling check rules
* remove extra param
* improve type handling and null handling
* remove extra zeros
* fix tests and add space between unit suffix and number as most size-format functions do
* fix tests
* add examples
* change function names according to review comments
* fix merge
Signed-off-by: frank chen <frank.chen021@outlook.com>
* no need to configure NullHandling explicitly for tests
Signed-off-by: frank chen <frank.chen021@outlook.com>
* fix tests in SQL-Compatible mode
Signed-off-by: frank chen <frank.chen021@outlook.com>
* Resolve review comments
* Update SQL test case to check null handling
* Fix intellij inspections
* Add more examples
* Fix example
* Improve concurrency between DruidSchema and BrokerServerView
* unused imports and workaround for error prone faiure
* count only known segments
* add comments
This PR fixes the incorrect results for query :
SELECT dim1, l1.k FROM foo LEFT JOIN (select k || '' as k from lookup.lookyloo group by 1) l1 ON foo.dim1 = l1.k WHERE l1.k IS NOT NULL (in CalciteQueryTests)
In the current code, the WHERE clause gets removed from the top of the left join and is pushed to the table foo
leading to incorrect results.
The fix for such a situation is done by :
Converting such left joins into inner joins (since logically the mentioned left join query is equivalent to an inner join) using Calcite while maintaining that the druid execution layer can execute such inner joins.
Preferring converted inner joins over original left joins in our cost model
This PR splits current SegmentLoader into SegmentLoader and SegmentCacheManager.
SegmentLoader - this class is responsible for building the segment object but does not expose any methods for downloading, cache space management, etc. Default implementation delegates the download operations to SegmentCacheManager and only contains the logic for building segments once downloaded. . This class will be used in SegmentManager to construct Segment objects.
SegmentCacheManager - this class manages the segment cache on the local disk. It fetches the segment files to the local disk, can clean up the cache, and in the future, support reserve and release on cache space. [See https://github.com/Make SegmentLoader extensible and customizable #11398]. This class will be used in ingestion tasks such as compaction, re-indexing where segment files need to be downloaded locally.
* improve groupBy query granularity translation when issued from sql layer
* fix style
* use virtual column to determine timestampResult granularity
* dont' apply postaggregators on compute nodes
* relocate constants
* fix order by correctness issue
* fix ut
* use more easier understanding code in DefaultLimitSpec
* address comment
* rollback use virtual column to determine timestampResult granularity
* fix style
* fix style
* address the comment
* add more detail document to explain the tradeoff
* address the comment
* address the comment
Users sometimes make typos when picking timezones - like `America/Los Angeles`
instead of `America/Los_Angeles` instead of defaulting to UTC, this change
makes it so that an error is thrown instead notifying the user of their mistake.
A constant expression may evaluate to Double.NEGATIVE_INFINITY/Double.POSITIVE_INFINITY/Double.NAN e.g. log10(0). When using such an expression in native queries, the user will get the corresponding value without any error. In SQL, however, the user will run into NumberFormatException because we convert the double to big-decimal while constructing a literal numeric expression. This probably should be fixed in calcite - see https://issues.apache.org/jira/browse/CALCITE-2067. This PR adds a verbose error message so that users can take corrective action without scratching their heads.
* add single input string expression dimension vector selector and better expression planning
* better
* fixes
* oops
* rework how vector processor factories choose string processors, fix to be less aggressive about vectorizing
* oops
* javadocs, renaming
* more javadocs
* benchmarks
* use string expression vector processor with vector size 1 instead of expr.eval
* better logging
* javadocs, surprising number of the the
* more
* simplify
* Fix is null selector returning incorrect value for Long data type
* Fix style errors
* Refactor getObject method to also cache null column values
* Make lastInput variable nullable
* Refactor unit test
* Use new boolean lastInputIsNull instead of Long for lastInput to avoid boxing
* Refactor to remove Long for input variable
* Make a separate null caching variable
* Cleaner null caching implementation
* fix count and average SQL aggregators on constant virtual columns
* style
* even better, why are we tracking virtual columns in aggregations at all if we have a virtual column registry
* oops missed a few
* remove unused
* this will fix it
* SQL timeseries no longer skip empty buckets with all granularity
* add comment, fix tests
* the ol switcheroo
* revert unintended change
* docs and more tests
* style
* make checkstyle happy
* docs fixes and more tests
* add docs, tests for array_agg
* fixes
* oops
* doc stuffs
* fix compile, match doc style
* Fix vectorized cardinality bug on certain string columns.
Fixes a bug introduced in #11182, related to the fact that in some cases,
ColumnProcessors.makeVectorProcessor will call "makeObjectProcessor"
instead of "makeSingleValueDimensionProcessor" or
"makeMultiValueDimensionProcessor". CardinalityVectorProcessorFactory
improperly ignored calls to "makeObjectProcessor".
In addition to fixing the bug, I added this detail to the javadocs for
VectorColumnProcessorFactory, to prevent others from running into the
same thing in the future. They do not currently call out this case.
* Improve test coverage.
* Additional fixes.
* ARRAY_AGG sql aggregator function
* add javadoc
* spelling
* review stuff, return null instead of empty when nil input
* review stuff
* Update sql.md
* use type inference for finalize, refactor some things
* Vectorize the cardinality aggregator.
Does not include a byRow implementation, so if byRow is true then
the aggregator still goes through the non-vectorized path.
Testing strategy:
- New tests that exercise both styles of "aggregate" for supported types.
- Some existing tests have also become active (note the deleted
"cannotVectorize" lines).
* Adjust whitespace.
* Enable rewriting certain inner joins as filters.
The main logic for doing the rewrite is in JoinableFactoryWrapper's
segmentMapFn method. The requirements are:
- It must be an inner equi-join.
- The right-hand columns referenced by the condition must not contain any
duplicate values. (If they did, the inner join would not be guaranteed
to return at most one row for each left-hand-side row.)
- No columns from the right-hand side can be used by anything other than
the join condition itself.
HashJoinSegmentStorageAdapter is also modified to pass through to
the base adapter (even allowing vectorization!) in the case where 100%
of join clauses could be rewritten as filters.
In support of this goal:
- Add Query getRequiredColumns() method to help us figure out whether
the right-hand side of a join datasource is being used or not.
- Add JoinConditionAnalysis getRequiredColumns() method to help us
figure out if the right-hand side of a join is being used by later
join clauses acting on the same base.
- Add Joinable getNonNullColumnValuesIfAllUnique method to enable
retrieving the set of values that will form the "in" filter.
- Add LookupExtractor canGetKeySet() and keySet() methods to support
LookupJoinable in its efforts to implement the new Joinable method.
- Add "enableRewriteJoinToFilter" feature flag to
JoinFilterRewriteConfig. The default is disabled.
* Test improvements.
* Test fixes.
* Avoid slow size() call.
* Remove invalid test.
* Fix style.
* Fix mistaken default.
* Small fixes.
* Fix logic error.