Commit Graph

957 Commits

Author SHA1 Message Date
Gian Merlino 9f358f5f4a
SQL tests: avoid mixing skip and cannot vectorize. (#16251)
* SQL tests: avoid mixing skip and cannot vectorize.

skipVectorize switches off vectorization tests completely, and
cannotVectorize turns vectorization tests into negative tests. It doesn't
make sense to use them together, so this patch makes it an error to do so,
and cleans up cases where both are mentioned.

This patch also has the effect of changing various tests from skipVectorize
to cannotVectorize, because in the past when both were mentioned,
skipVectorize would take priority.

* Fix bug with StringAnyAggregatorFactory attempting to vectorize when it cannt.

* Fix tests.
2024-04-11 15:06:11 -07:00
Soumyava 7759f25095
Moving bitwise_or to use native calcite operator (#16237) 2024-04-04 12:49:29 -07:00
Soumyava 972937659d
Fixing return type for IPV4 (#15916)
* Fixing return type for IPV4

* Update ipv4match
2024-04-04 08:49:50 -07:00
Soumyava 4bea865697
Restore context flag for window functions (#16229) 2024-04-03 13:57:13 +05:30
zachjsh 9b52c909e0
fix complex types returning UNKNOWN as their SQL type inference (#16216)
* * fix

* * fix

* * address review comments
2024-04-02 14:36:01 -04:00
Aleksey Plekhanov a818b8acb6
Fix CalciteQueryTest#testCountStarWithTimeFilterUsingStringLiterals (#16221)
* Add cases to check handling equals between timestamp and string literal
2024-04-01 13:46:24 -07:00
Soumyava 524842a3bb
Window function on msq (#15470)
This PR aims to introduce Window functions on MSQ by doing the following:

    Introduce a Window querykit for handling window queries along with its factory and a processor for window queries
    If a window operator is present with a partition by clause, pushes the partition as a shuffle spec of the previous stage
    In presence of empty OVER() clause lets all operators loose on a single rac
    In presence of no empty OVER() clause, breaks down each window into individual stages
    Associated machinery to handle window functions in MSQ
    Introduced a separate hidden engine feature WINDOW_LEAF_OPERATOR which is set only for MSQ engine. In presence of this feature, the planner plans without the leaf operators by creating a window query over an inner scan query. In case of native this is set to false and the planner generates the leafOperators
    Guardrails around materialization
    Comprehensive UTs
2024-03-28 14:58:34 +05:30
Sree Charan Manamala f29c8ac368
Allow non literal rhs in MV_FILTER_ONLY and MV_FILTER_NONE (#16113)
This commit allows to use the MV_FILTER_ONLY & MV_FILTER_NONE functions
with a non literal argument. 
Currently `select mv_filter_only('mvd_dim', 'array_dim') from 'table'`
returns a `Unhandled Query Planning Failure`

This is being tackled and also considered for the cases where the `array_dim`
having null & empty values.

Changed classes:
 * `MultiValueStringOperatorConversions`
 * `ApplyFunction`
 * `CalciteMultiValueStringQueryTest`
2024-03-26 12:31:09 +05:30
Zoltan Haindrich a16092b16a
Rewrite exotic LAST_VALUE/FIRST_VALUE to self-reference. (#16063)
* Rewrite exotic LAST_VALUE/FIRST_VALUE to self-reference.

* rewrite `LAST_VALUE(x) OVER (ORDER BY y)` to `LAG(x,0) OVER (ORDER BY y)`
* not directly to `x` because some queries get unplannable that way
* restrict `NTILE` from framing - as its not supported
* add test to ensure that all of the `KNOWN_WINDOW_FNS`'s framing is accounted for

* checkstyle/etc

* add test

* apidoc

* add assume to avoid MSQ fail
2024-03-25 11:03:47 -07:00
zachjsh 8370db106c
INSERT/REPLACE dimension target column types are validated against source input expressions (#15962)
* * address remaining comments from https://github.com/apache/druid/pull/15836

* *  address remaining comments from https://github.com/apache/druid/pull/15908

* * add test that exposes relational algebra issue

* * simplify test exposing issue

* * fix

* * add tests for sealed / non-sealed

* * update test descriptions

* * fix test failure when -Ddruid.generic.useDefaultValueForNull=true

* * check type assignment based on natice Druid types

* * add tests that cover missing jacoco coverage

* * add replace tests

* * add more tests and comments about column ordering

* * simplify tests

* * review comments

* * remove commented line

* * STRING family types should be validated as non-null
2024-03-25 12:34:07 -04:00
Clint Wylie b0a9c318d6
add new typed in filter (#16039)
changes:
* adds TypedInFilter which preserves matching sets in the native match value type
* SQL planner uses new TypedInFilter when druid.generic.useDefaultValueForNull=false (the default)
2024-03-22 12:45:08 -07:00
Clint Wylie 48b8d42698
fix regexp_like, contains_string, icontains_string to return null instead of false for null inputs in sql compatible mode (#15963) 2024-03-19 22:12:47 -07:00
Gian Merlino c96b215dd6
SortMerge join support for IS NOT DISTINCT FROM. (#16003)
* SortMerge join support for IS NOT DISTINCT FROM.

The patch adds a "requiredNonNullKeyParts" field to the sortMerge
processor, which has the list of key parts that must be nonnull for
an equijoin condition to match. Conditions with SQL "=" are present in
the list; conditions with SQL "IS NOT DISTINCT FROM" are absent from
the list.

* Fix test.

* Update javadoc.
2024-03-19 12:02:13 -07:00
Zoltan Haindrich 0a42342cef
Update Calcite*Test to use junit5 (#16106)
* Update Calcite*Test to use junit5

* change the way temp dirs are handled
* add openrewrite workflow to safeguard upgrade
* replace junitparamrunner with standard junit5 parametered tests
* update a few rules to junit5 api
* lots of boring changes

* cleanup QueryLogHook

* cleanup

* fix compile error: ARRAYS_DATASOURCE

* fix test

* remove enclosed

* empty

+TEST:TDigestSketchSqlAggregatorTest,HllSketchSqlAggregatorTest,DoublesSketchSqlAggregatorTest,ThetaSketchSqlAggregatorTest,ArrayOfDoublesSketchSqlAggregatorTest,BloomFilterSqlAggregatorTest,BloomDimFilterSqlTest,CatalogIngestionTest,CatalogQueryTest,FixedBucketsHistogramQuantileSqlAggregatorTest,QuantileSqlAggregatorTest,MSQArraysTest,MSQDataSketchesTest,MSQExportTest,MSQFaultsTest,MSQInsertTest,MSQLoadedSegmentTests,MSQParseExceptionsTest,MSQReplaceTest,MSQSelectTest,InsertLockPreemptedFaultTest,MSQWarningsTest,SqlMSQStatementResourcePostTest,SqlStatementResourceTest,CalciteSelectJoinQueryMSQTest,CalciteSelectQueryMSQTest,CalciteUnionQueryMSQTest,MSQTestBase,VarianceSqlAggregatorTest,SleepSqlTest,SqlRowTransformerTest,DruidAvaticaHandlerTest,DruidStatementTest,BaseCalciteQueryTest,CalciteArraysQueryTest,CalciteCorrelatedQueryTest,CalciteExplainQueryTest,CalciteExportTest,CalciteIngestionDmlTest,CalciteInsertDmlTest,CalciteJoinQueryTest,CalciteLookupFunctionQueryTest,CalciteMultiValueStringQueryTest,CalciteNestedDataQueryTest,CalciteParameterQueryTest,CalciteQueryTest,CalciteReplaceDmlTest,CalciteScanSignatureTest,CalciteSelectQueryTest,CalciteSimpleQueryTest,CalciteSubqueryTest,CalciteSysQueryTest,CalciteTableAppendTest,CalciteTimeBoundaryQueryTest,CalciteUnionQueryTest,CalciteWindowQueryTest,DecoupledPlanningCalciteJoinQueryTest,DecoupledPlanningCalciteQueryTest,DecoupledPlanningCalciteUnionQueryTest,DrillWindowQueryTest,DruidPlannerResourceAnalyzeTest,IngestTableFunctionTest,QueryTestRunner,SqlTestFrameworkConfig,SqlAggregationModuleTest,ExpressionsTest,GreatestExpressionTest,IPv4AddressMatchExpressionTest,IPv4AddressParseExpressionTest,IPv4AddressStringifyExpressionTest,LeastExpressionTest,TimeFormatOperatorConversionTest,CombineAndSimplifyBoundsTest,FiltrationTest,SqlQueryTest,CalcitePlannerModuleTest,CalcitesTest,DruidCalciteSchemaModuleTest,DruidSchemaNoDataInitTest,InformationSchemaTest,NamedDruidSchemaTest,NamedLookupSchemaTest,NamedSystemSchemaTest,RootSchemaProviderTest,SystemSchemaTest,CalciteTestBase,SqlResourceTest

* use @Nested

* add rule to remove enclosed; upgrade surefire

* remove enclosed

* cleanup

* add comment about surefire exclude
2024-03-19 04:05:12 -07:00
Clint Wylie 5afd5c41a5
fix ColumnType to RelDataType conversion for nested arrays (#16138)
* fix ColumnType to RelDataType conversion for nested arrays

* fix test
2024-03-18 23:34:08 -07:00
Zoltan Haindrich d3e22c6e92
fix compile error: ARRAYS_DATASOURCE (#16120) 2024-03-14 18:15:43 +05:30
Clint Wylie dd9bc3749a
fix issues with array_contains and array_overlap with null left side arguments (#15974)
changes:
* fix issues with array_contains and array_overlap with null left side arguments
* modify singleThreaded stuff to allow optimizing Function similar to how we do for ExprMacro - removed SingleThreadSpecializable in favor of default impl of asSingleThreaded on Expr with clear javadocs that most callers shouldn't be calling it directly and should be using Expr.singleThreaded static method which uses a shuttle and delegates to asSingleThreaded instead
* add optimized 'singleThreaded' versions of array_contains and array_overlap
* add mv_harmonize_nulls native expression to use with MV_CONTAINS and MV_OVERLAP to allow them to behave consistently with filter rewrites, coercing null and [] into [null]
* fix bug with casting rhs argument for native array_contains and array_overlap expressions
2024-03-13 18:16:10 -07:00
Sree Charan Manamala e9d2caccb6
Handling null operand in JSON_QUERY_ARRAY (#16118)
* fix return type inference for JSON_QUERY_ARRAY to be nullable
2024-03-13 18:06:27 -07:00
Gian Merlino 256160aba6
MSQ: Validate that strings and string arrays are not mixed. (#15920)
* MSQ: Validate that strings and string arrays are not mixed.

When multi-value strings and string arrays coexist in the same column,
it causes problems with "classic MVD" style queries such as:

  select * from wikipedia -- fails at runtime
  select count(*) from wikipedia where flags = 'B' -- fails at planning time
  select flags, count(*) from wikipedia group by 1 -- fails at runtime

To avoid these problems, this patch adds type verification for INSERT
and REPLACE. It is targeted: the only type changes that are blocked are
string-to-array and array-to-string. There is also a way to exclude
certain columns from the type checks, if the user really knows what
they're doing.

* Fixes.

* Tests and docs and error messages.

* More docs.

* Adjustments.

* Adjust message.

* Fix tests.

* Fix test in DV mode.
2024-03-13 15:37:27 -07:00
Gian Merlino 910124d4de
MSQ: Plan without implicit sorting. (#16073)
* MSQ: Plan without implicit sorting.

This patch adds an EngineFeature "GROUPBY_IMPLICITLY_SORTS" and sets
it true for native, false for MSQ. It's useful for two reasons:

1) In the future we'll likely want MSQ to hash-partition for GROUP BY
   instead of using a global sort, which would mean MSQ would not
   implicitly ORDER BY when there is a GROUP BY.

2) When doing REPLACE with MSQ, CLUSTERED BY is transformed to ORDER BY.
   We should retain that ORDER BY, as it may be a subset of the GROUP BY,
   and it is important to remember which fields the user wanted to include in
   range shard specs.

* Fix tests.

* Fix tests for real.

* Fix test.
2024-03-13 08:27:39 -07:00
Clint Wylie 795e342ba8
fix sql results mixed array and scalar values (#16105)
* fix sql results mixed array and scalar values

* simplify
2024-03-12 23:47:35 -07:00
Zoltan Haindrich 8252d72e2a
Pull up literals in InputAccessor (#16033)
* Pull up literals in InputAccessor

* pull up literals in `InputAccessor`
* remove the need to pass `constants` of `Window`  operator

Fixes #15353

* update test

* enable relax_nulls
2024-03-12 09:14:31 -07:00
Sree Charan Manamala ef9637eef1
Handling array with boolean literals (#16093)
Handling array with boolean literals like ARRAY[true, false]

Druid appears to be able to convert an array with boolean expressions like this array[added=deleted, added=delta] into a numeric array of 0 and 1: select array[added=deleted, added=delta] from wikipedia

However, select array[true, false] from wikipedia doesn't work.
This PR fixes this.
2024-03-12 12:28:16 +05:30
Soumyava 85ee775390
Handling latest_by and earliest_by on numeric columns correctly (#15939)
* Handling latest_by and earliest_by on numeric columns correctly

* Adding test
2024-03-11 13:49:21 -07:00
Zoltan Haindrich 2eb7d7a89b
Calcite tests remove expected exception (#16046)
* Calcite tests remove expected exception

* update testcases using `expectedException` to utilize `assertThrows` instead
* remove `BaseCalciteQueryTest#expectedException`
* fixes `cannotVectorize` so it doesn't anymore stops further processing
* `msqIncompatible` is not anymore toggles a boolean - its an `Assume` instead

Fixes #15423

* cleanup

* move msqIncompat

* update test

* cleanup

* remove comment

* empty-commit

* empty-commit
2024-03-11 13:23:57 +05:30
Zoltan Haindrich aaa64832fd
Disable DecoupledPlanningCalciteJoinQueryTest until it gets fixed (#16070)
Recently this test started other tests from executing by triggering a bug somewhere in surefire.
This patch disables the testcases in case of non-sql compat mode.
2024-03-07 12:55:48 -08:00
Laksh Singla 5f588fa45c
Fix bug while materializing scan's result to frames (#15987)
While converting Sequence<ScanResultValue> to Sequence<Frames>, when maxSubqueryBytes is enabled, we batch the results to prevent creating a single frame per ScanResultValue. Batching requires peeking into the actual value, and checking if the row signature of the scan result’s value matches that of the previous value.

Since we can do this indefinitely (in the worst case all of them have the same signature), we keep fetching them and accumulating them in a list (on the heap). We don’t really know how much to batch before we actually write the value as frames.

The PR modifies the batching logic to not accumulate the results in an intermediary list
2024-03-07 17:11:44 +05:30
Vishesh Garg cf9bc507f6
Fix compilation failure due to missing constant MISSING_JOIN_CONVERSION (#16050)
* Reintroduce variable MISSING_JOIN_CONVERSION

* Remove redundant constant MISSING_JOIN_CONVERSION2

* Correct fix to address failing tests
2024-03-06 15:34:39 +08:00
Zoltan Haindrich 65c3b4d31a
Support join in decoupled mode (#15957)
* plan join(s) in decoupled mode
* configure DecoupledPlanningCalciteJoinQueryTest
        the test has 593 cases; however there are quite a few parameterized
        from the 107 methods annotated with @Test - 42 is not yet working
 * replace the isRoot hack in DruidQueryGenerator with a logic that instead looks ahead for the next node; and doesn't let the previous node do the Project - this makes it plan more likely than the existing planner
2024-03-05 19:10:13 -06:00
Zoltan Haindrich bb882727c0
Fix Windowing/scanAndSort query issues on top of Joins. (#15996)
allow a hashjoin result to be converted to RowsAndColumns
added StorageAdapterRowsAndColumns
fix incorrect isConcrete() return values during early phase of planning
2024-03-05 15:05:31 +05:30
Zoltan Haindrich e469b7ed34
Make setting QUERY_CONTEXT_DEFAULT explicit in tests (#16010) 2024-03-05 10:54:16 +05:30
Adarsh Sanjeev 93eeb05eaf
Revert explain attributes change to old behaviour. (#16004)
* Revert explain attributes change

* Fix tests

* Fix tests

* Rename function
2024-03-04 15:56:02 +05:30
Zoltan Haindrich bf0995f846
Introduce dynamic table append (#15897) 2024-03-01 04:31:57 -05:00
Laksh Singla 17e4f3ac60
Refactor GroupBy and TopN code to relax the constraint of dimensions being comparable (#15559)
The code in the groupBy engine and the topN engine assume that the dimensions are comparable and can call dimA.compareTo(dimB) to sort the dimensions and group them together.
This works well for the primitive dimensions, because they are Comparable, however falls apart when the dimensions can be arrays (or in future scenarios complex columns). In cases when the dimensions are not comparable, Druid resorts to having a wrapper type ComparableStringArray and ComparableList, which is a Comparable, based on the list comparator.
2024-02-27 11:39:29 +05:30
Soumyava 51cc729fd1
Enforcing type checking for flatten concat (#15903) 2024-02-26 21:53:49 -08:00
Abhishek Radhakrishnan 67a6224d91
Fix up incorrect `PARTITIONED BY` error messages (#15961)
* Fix up typos, inaccuracies and clean up code related to PARTITIONED BY.

* Remove wrapper function and update tests to use DruidExceptionMatcher.

* Checkstyle and Intellij inspection fixes.
2024-02-26 14:17:53 -05:00
Zoltan Haindrich 06deda9415
ScanAndSort query fails with NPE for simple queries (#15914)
* some stuff

* add dummy fields

* draft-fix

* rename test

* cleanup

* add null

* cleanup

* cleanup

* add test

* updates

* move check tp constructore

* cleanup

* updates/etc

* fix some more

* add rowSignatureMode

* checkstyle/etc

* override

* missing msqIncompat

* fix test

* fixes

* undo

* updates

* remove param
2024-02-24 15:33:50 -08:00
zachjsh 8ebf237576
Move INSERT & REPLACE validation to the Calcite validator (#15908)
This PR contains a portion of the changes from the inactive draft PR for integrating the catalog with the Calcite planner https://github.com/apache/druid/pull/13686 from @paul-rogers, Refactoring the IngestHandler and subclasses to produce a validated SqlInsert instance node instead of the previous Insert source node. The SqlInsert node is then validated in the calcite validator. The validation that is implemented as part of this pr, is only that for the source node, and some of the validation that was previously done in the ingest handlers. As part of this change, the partitionedBy clause can be supplied by the table catalog metadata if it exists, and can be omitted from the ingest time query in this case.
2024-02-22 14:01:59 -05:00
Zoltan Haindrich bcce0806d7
Support Union in decoupled mode (#15870) 2024-02-21 10:54:50 -05:00
Gian Merlino 9c41827dba
Globally disable AUTO_CLOSE_JSON_CONTENT. (#15880)
* Globally disable AUTO_CLOSE_JSON_CONTENT.

This JsonGenerator feature is on by default. It causes problems with code
like this:

  try (JsonGenerator jg = ...) {
    jg.writeStartArray();
    for (x : xs) {
      jg.writeObject(x);
    }
    jg.writeEndArray();
  }

If a jg.writeObject call fails due to some problem with the data it's
reading, the JsonGenerator will write the end array marker automatically
when closed as part of the try-with-resources. If the generator is writing
to a stream where the reader does not have some other mechanism to realize
that an exception was thrown, this leads the reader to believe that the
array is complete when it actually isn't.

Prior to this patch, we disabled AUTO_CLOSE_JSON_CONTENT for JSON-wrapped
SQL result formats in #11685, which fixed an issue where such results
could be erroneously interpreted as complete. This patch fixes a similar
issue with task reports, and all similar issues that may exist elsewhere,
by disabling the feature globally.

* Update test.
2024-02-16 08:52:48 -08:00
Clint Wylie fe2ba8cc28
fix return type inference of parse_long, which can also be null if string is not parseable into a long (#15909)
* fix return type inference of parse_long, which can also be null if string is not parseable into a long

* fix msq test
2024-02-15 08:45:34 -08:00
zachjsh f9ee2c353b
Extend the PARTITION BY clause to accept string literals for the time partitioning (#15836)
This PR contains a portion of the changes from the inactive draft PR for integrating the catalog with the Calcite planner https://github.com/apache/druid/pull/13686 from @paul-rogers, extending the PARTITION BY clause to accept string literals for the time partitioning
2024-02-09 11:45:38 -05:00
Sree Charan Manamala 57e12df352
Sql Single Value Aggregator for scalar queries (#15700)
Executing single value correlated queries will throw an exception today since single_value function is not available in druid.
With these added classes, this provides druid, the capability to plan and run such queries.
2024-02-08 19:20:30 +05:30
Soumyava f3996b96ff
Fixes for safe_divide with vectorize and datatypes (#15839)
* Fix for save_divide with vectorize

* More fixes

* Update to use expr.eval(null) for both cases when denominator is 0
2024-02-08 14:40:42 +05:30
Adarsh Sanjeev 514b3b4d01
Add export capabilities to MSQ with SQL syntax (#15689)
* Add test

* Parser changes to support export statements

* Fix builds

* Address comments

* Add frame processor

* Address review comments

* Fix builds

* Update syntax

* Webconsole workaround

* Refactor

* Refactor

* Change export file path

* Update docs

* Remove webconsole changes

* Fix spelling mistake

* Parser changes, add tests

* Parser changes, resolve build warnings

* Fix failing test

* Fix failing test

* Fix IT tests

* Add tests

* Cleanup

* Fix unparse

* Fix forbidden API

* Update docs

* Update docs

* Address review comments

* Address review comments

* Fix tests

* Address review comments

* Fix insert unparse

* Add external write resource action

* Fix tests

* Add resource check to overlord resource

* Fix tests

* Add IT

* Update syntax

* Update tests

* Update permission

* Address review comments

* Address review comments

* Address review comments

* Add tests

* Add check for runtime parameter for bucket and path

* Add check for runtime parameter for bucket and path

* Add tests

* Update docs

* Fix NPE

* Update docs, remove deadcode

* Fix formatting
2024-02-07 22:08:50 +05:30
Clint Wylie 23d4fade90
use NullFilter for SQL rewrite of MV_CONTAINS and MV_OVERLAP for null array elements (#15855)
Fixes an oversight after #14542 that happens in the SQL planner rewrite of MV_CONTAINS and MV_OVERLAP when faced with array elements that are NULL, where we were incorrectly using EqualityFilter instead of NullFilter for null elements (EqualityFilter does not accept null elements).
2024-02-07 19:40:41 +05:30
Zoltan Haindrich fdc7cec271
Support Window operators in decoupled planning (#15815) 2024-02-07 04:09:48 -05:00
Gian Merlino 54b30646f3
Add sqlReverseLookupThreshold for ReverseLookupRule. (#15832)
If lots of keys map to the same value, reversing a LOOKUP call can slow
things down unacceptably. To protect against this, this patch introduces
a parameter sqlReverseLookupThreshold representing the maximum size of an
IN filter that will be created as part of lookup reversal.

If inSubQueryThreshold is set to a smaller value than
sqlReverseLookupThreshold, then inSubQueryThreshold will be used instead.
This allows users to use that single parameter to control IN sizes if they
wish.
2024-02-06 16:32:05 +05:30
Soumyava b86f31f2c0
Addressing shapeshifting issues with window functions (#15807)
Addressing shapeshifting issues with window functions
2024-02-06 11:12:20 +05:30
Zoltan Haindrich 392d585ff8
Identify not range filters without negating subexpressions (#15766)
* Identify not range filters without negating subexpressions

Earlier betweenish (range/bounds) filters were identified thru
a process of negating the subexpressions which may have not performed that well.
(it could have dominated the runtime in some cases)
This patch makes that unnecessary as its able to create the negate expression directly.

* add test;fix for multiple intervals
2024-02-05 19:12:58 -08:00