Commit Graph

976 Commits

Author SHA1 Message Date
Laksh Singla e695e52d3f
Improve code flow in the First/Last vector aggregators and unify the numeric aggregators with the String implementations (#16230)
This PR fixes the first and last vector aggregators and improves their readability. Following changes are introduced

    The folding is broken in the vectorized versions. We consider time before checking the folded object.

    If the numerical aggregator gets passed any other object type for some other reason (like String), then the aggregator considers it to be folded, even though it shouldn’t be. We should convert these objects to the desired type, and aggregate them properly.

    The aggregators must properly use generics. This would minimize the ClassCastException issues that can happen with mixed segment types. We are unifying the string first/last aggregators with numeric versions as well.

    The aggregators must aggregate null values (https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java#L55-L56 ). The aggregator should only ignore pairs with time == null, and not value == null

    Time nullity is ignored when trying to vectorize the data.

    String versions initialized with DateTimes.MIN that is equal to Long.MIN / 2. This can cause incorrect results in case the user enters a custom time column. NOTE: This is still present because it would require a larger refactor in all of the versions.

    There is a difference in what users might expect from the results because the code flow is changed (for example, the direction of the for loops, etc), however, this will only change the results, and not the contract set by first/last aggregators, which is that if multiple values have the same timestamp, then any of them can get picked.

    If the column is non-existent, the users might expect a change in the timestamp from DateTime.MAX to Long.MAX, because the code incorrectly used DateTime.MAX to initialize the aggregator, however, in case of a custom timestamp column, this might not be the case. The SQL query might be prohibited from using any Long since it requires a cast to the timestamp function that can fail, but AFAICT native queries don't have such limitations.
2024-04-30 15:13:14 +05:30
Laksh Singla 26d63e7b65
Prevent joining on nested arrays and complex types (#16349)
#16068 modified DimensionHandlerUtils to accept complex types to be dimensions. This had an unintended side effect of allowing complex types to be joined upon (which wasn't guarded explicitly, it doesn't work).
This PR modifies the IndexedTable to reject building the index on the complex types to prevent joining on complex types. The PR adds back the check in the same place, explicitly.
2024-04-30 11:36:53 +05:30
Akshat Jain 9d2cae40c3
Add support for selective loading of lookups in the task layer (#16328)
Changes:
- Add `LookupLoadingSpec` to support 3 modes of lookup loading: ALL, NONE, ONLY_REQUIRED
- Add method `Task.getLookupLoadingSpec()`
- Do not load any lookups for `KillUnusedSegmentsTask`
2024-04-29 07:19:59 +05:30
zachjsh 365cd7e8e7
INSERT/REPLACE can omit clustering when catalog has default (#16260)
* * fix

* * fix

* * address review comments

* * fix

* * simplify tests

* * fix complex type nullability issue

* * implement and add tests

* * address review comments

* * address test review comments

* * fix checkstyle

* * fix dependencies

* * all tests passing

* * cleanup

* * remove unneeded code

* * remove unused dependency

* * fix checkstyle
2024-04-26 10:19:45 -04:00
Adarsh Sanjeev 9a2d7c28bc
Prepare master branch for 31.0.0 release (#16333) 2024-04-26 09:22:43 +05:30
Gian Merlino 68d6e682e8
Fix TimeBoundary planning when filters require virtual columns. (#16337)
The timeBoundary query does not support virtual columns, so we should
avoid it if the query requires virtual columns.
2024-04-25 16:49:40 -07:00
Zoltan Haindrich 9c0bd56f5b
Make QueryComponentSupliers independent from test classes (#16275) 2024-04-25 02:12:07 -04:00
Laksh Singla 6bca406d31
Grouping on complex columns aka unifying GroupBy strategies (#16068)
Users can pass complex types as dimensions to the group by queries. For example:

SELECT nested_col1, count(*) FROM foo GROUP BY nested_col1
2024-04-24 23:00:14 +05:30
Rishabh Singh e30790e013
Introduce Segment Schema Publishing and Polling for Efficient Datasource Schema Building (#15817)
Issue: #14989

The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Thereafter, we addressed the problem of publishing schema for realtime segments (#15475). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information.

This is the final change which involves publishing segment schema for finalized segments from task and periodically polling them in the Coordinator.
2024-04-24 22:22:53 +05:30
Sree Charan Manamala 080476f9ea
WINDOWING - Fix 2 nodes with same digest causing mapping issue (#16301)
Fixes the mapping issue in window fucntions where 2 nodes get the same reference.
2024-04-24 16:45:02 +05:30
Laksh Singla b9bbde5c0a
Fix deadlock that can occur while merging group by results (#15420)
This PR prevents such a deadlock from happening by acquiring the merge buffers in a single place and passing it down to the runner that might need it.
2024-04-22 14:10:44 +05:30
Sree Charan Manamala ad5701e891
new SCALAR_IN_ARRAY function analogous to DRUID_IN (#16306)
* scalar_in function

* api doc

* refactor
2024-04-18 21:15:15 -07:00
Sree Charan Manamala 960a674442
Corrected Strict NON NULL return type checks (#16279) 2024-04-18 12:17:13 +02:00
Gian Merlino ccc1ffb032
Additional short circuiting knowledge in filter bundles. (#16292)
* Additional short circuiting knowledge in filter bundles.

Three updates:

1) The parameter "selectionRowCount" on "makeFilterBundle" is renamed
   "applyRowCount", and redefined as an upper bound on rows remaining
   after short-circuiting (rather than number of rows selected so far).
   This definition works better for OR filters, which pass through the
   FALSE set rather than the TRUE set to the next subfilter.

2) AndFilter uses min(applyRowCount, indexIntersectionSize) rather
   than using selectionRowCount for the first subfilter and indexIntersectionSize
   for each filter thereafter. This improves accuracy when the incoming
   applyRowCount is smaller than the row count from the first few indexes.

3) OrFilter uses min(applyRowCount, totalRowCount - indexUnionSize) rather
   than applyRowCount for subfilters. This allows an OR filter to pass
   information about short-circuiting to its subfilters.

To help write tests for this, the patch also moves the sampled
wikiticker data file from sql to processing.

* Forbidden APIs.

* Forbidden APIs.

* Better comments.

* Fix inspection.

* Adjustments to tests.
2024-04-16 22:42:28 -07:00
zachjsh a5428e75ff
INSERT/REPLACE complex target column types are validated against source input expressions (#16223)
* * fix

* * fix

* * address review comments

* * fix

* * simplify tests

* * fix complex type nullability issue

* * address review comments

* * address test review comments

* * fix checkstyle
2024-04-16 17:20:35 -04:00
Sree Charan Manamala 5247059d2f
Allow Double & null values in sql type array through dynamic params (#16274) 2024-04-15 10:44:42 +02:00
Adarsh Sanjeev 3df00aef9d
Add manifest file for MSQ export (#15953)
Currently, export creates the files at the provided destination. The addition of the manifest file will provide a list of files created as part of the manifest. This will allow easier consumption of the data exported from Druid, especially for automated data pipelines
2024-04-15 11:37:31 +05:30
Sree Charan Manamala 3340b200db
Fix window function drill tests failures falling under RESULT_MISMATCH & RESULT_COUNT_MISMATCH (#16264)
* Updated the drill test expected results which are failing due to druid's default sorting algorithm taking nulls first approach.
* Corrected the queries where date time values are directly provided
* marked 2 cases failing with resultset casting issues
2024-04-12 13:54:48 +02:00
Sree Charan Manamala f65c166327
Windowed aggregates should update the aggregation value based on final compute (#16244) 2024-04-12 08:28:33 +02:00
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