Better fallback strategy when the broker is unable to materialize the subquery's results as frames for estimating the bytes:
a. We don't touch the subquery sequence till we know that we can materialize the result as frames
In case of few aggregators for example BloomSqlAggregator, BaseVarianceSqlAggregator etc, the aggName is being updated from a0 to a0:agg, breaching the contract as we would expect the aggName as the name which is passed. This is causing a mismatch while creating a column accessor.
This commit aims to correct those violating sql aggregators.
* Fallback vectorization for FunctionExpr and BaseMacroFunctionExpr.
This patch adds FallbackVectorProcessor, a processor that adapts non-vectorizable
operations into vectorizable ones. It is used in FunctionExpr and BaseMacroFunctionExpr.
In addition:
- Identifiers are updated to offer getObjectVector for ARRAY and COMPLEX in addition
to STRING. ExprEvalObjectVector is updated to offer ARRAY and COMPLEX as well.
- In SQL tests, cannotVectorize now fails tests if an exception is not thrown. This makes
it easier to identify tests that can now vectorize.
- Fix a null-matcher bug in StringObjectVectorValueMatcher.
* Fix tests.
* Fixes.
* Fix tests.
* Fix test.
* Fix test.
* Fix serde for ArrayOfDoublesSketchConstantPostAggregator.
The version originally added in #13819 was missing an annotation for
the "value" property. Fixes#16539.
Line endings for ArrayOfDoublesSketchConstantPostAggregator.java are changed
from \r\n to \n.
Adds a serde test, and improves various other datasketches post-aggregator
serde tests to deserialize into PostAggregator. This verifies that the type
information is set up correctly.
* Fix excessive imports.
* Fix equals, hashCode.
* enable quidem uri support for `druidtest:///?ComponentSupplier=Nested` and similar
* changes the way `SqlTestFrameworkConfig` is being applied; all options will have their own annotation (its kinda impossible to detect that an annotation has a set value or its the default)
* enables hierarchical processing of config annotation (was needed to enable class level supplier annotation)
* moves uri processing related string2config stuff into `SqlTestFrameworkConfig`
Fixes a bug introduced in #16296, where the sketch might not be
initialized if get() is called without calling aggregate(). Also adds
a test for this case.
Buffer aggregators can contain some cached objects within them, such as
Memory references or HLL Unions. Prior to this patch, various Grouper
implementations were not releasing this state when resetting their own
internal state, which could lead to excessive memory use.
This patch renames AggregatorAdapater#close to "reset", and updates
Grouper implementations to call this reset method whenever they reset
their internal state.
The base method on BufferAggregator and VectorAggregator remains named
"close", for compatibility with existing extensions, but the contract
is adjusted to say that the aggregator may be reused after the method
is called. All existing implementations in core already adhere to this
new contract, except for the ArrayOfDoubles build flavors, which are
updated in this patch to adhere.
Additionally, this patch harmonizes buffer sketch helpers to call their
clear method "clear" rather than a mix of "clear" and "close". (Others
were already using "clear".)
* 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.
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
* 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
* 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
* Rework ExprMacro base classes to simplify implementations.
This patch removes BaseScalarUnivariateMacroFunctionExpr, adds
BaseMacroFunctionExpr at the top of the hierarchy (a suitable base class
for ExprMacros that take either arrays or scalars), and adds an
implementation for "visit" to BaseMacroFunctionExpr.
The effect on implementations is generally cleaner code:
- Exprs no longer need to implement "visit".
- Exprs no longer need to implement "stringify", even if they don't
use all of their args at runtime, because BaseMacroFunctionExpr has
access to even unused args.
- Exprs that accept arrays can extend BaseMacroFunctionExpr and
inherit a bunch of useful methods. The only one they need to
implement themselves that scalar exprs don't is "supplyAnalyzeInputs".
* Make StringDecodeBase64UTFExpression a static class.
* Remove unused import.
* Formatting, annotation changes.
* Fix HllSketchHolderObjectStrategy#isSafeToConvertToNullSketch.
The prior code from #15162 was reading only the low-order byte of an int
representing the size of a coupon set. As a result, it would erroneously
believe that a coupon set with a multiple of 256 elements was empty.
* support groups windowing mode; which is a close relative of ranges (but not in the standard)
* all windows with range expressions will be executed wit it groups
* it will be 100% correct in case for both bounds its true that: isCurrentRow() || isUnBounded()
* this covers OVER ( ORDER BY COL )
* for other cases it will have some chances of getting correct results...
Fixes a potential NPE which could occur while folding the HllSketchAggregator. If the sketch is null, druid could return a null HllSketchHolder object. Adding a null check here could help here
Resolves a null pointer exception in HllSketchAggregatorFactory
There is a problem with Quantiles sketches and KLL Quantiles sketches.
Queries using the histogram post-aggregator fail if:
- the sketch contains at least one value, and
- the values in the sketch are all equal, and
- the splitPoints argument is not passed to the post-aggregator, and
- the numBins argument is greater than 2 (or not specified, which
leads to the default of 10 being used)
In that case, the query fails and returns this error:
{
"error": "Unknown exception",
"errorClass": "org.apache.datasketches.common.SketchesArgumentException",
"host": null,
"errorCode": "legacyQueryException",
"persona": "OPERATOR",
"category": "RUNTIME_FAILURE",
"errorMessage": "Values must be unique, monotonically increasing and not NaN.",
"context": {
"host": null,
"errorClass": "org.apache.datasketches.common.SketchesArgumentException",
"legacyErrorCode": "Unknown exception"
}
}
This behaviour is undesirable, since the caller doesn't necessarily
know in advance whether the sketch has values that are diverse
enough. With this change, the post-aggregators return [N, 0, 0...]
instead of crashing, where N is the number of values in the sketch,
and the length of the list is equal to numBins. That is what they
already returned for numBins = 2.
Here is an example of a query that would fail:
{"queryType":"timeseries",
"dataSource": {
"type": "inline",
"columnNames": ["foo", "bar"],
"rows": [
["abc", 42.0],
["def", 42.0]
]
},
"intervals":["0000/3000"],
"granularity":"all",
"aggregations":[
{"name":"the_sketch", "fieldName":"bar", "type":"quantilesDoublesSketch"}],
"postAggregations":[
{"name":"the_histogram",
"type":"quantilesDoublesSketchToHistogram",
"field":{"type":"fieldAccess","fieldName":"the_sketch"},
"numBins": 3}]}
I believe this also fixes issue #10585.
In the current design, brokers query both data nodes and tasks to fetch the schema of the segments they serve. The table schema is then constructed by combining the schemas of all segments within a datasource. However, this approach leads to a high number of segment metadata queries during broker startup, resulting in slow startup times and various issues outlined in the design proposal.
To address these challenges, we propose centralizing the table schema management process within the coordinator. This change is the first step in that direction. In the new arrangement, the coordinator will take on the responsibility of querying both data nodes and tasks to fetch segment schema and subsequently building the table schema. Brokers will now simply query the Coordinator to fetch table schema. Importantly, brokers will still retain the capability to build table schemas if the need arises, ensuring both flexibility and resilience.
* better documentation for the differences between arrays and mvds
* add outputType to ExpressionPostAggregator to make docs true
* add output coercion if outputType is defined on ExpressionPostAgg
* updated post-aggregations.md to be consistent with aggregations.md and filters.md and use tables
Functions that accept literals also allow casted literals. This shouldn't have an impact on the queries that the user writes. It enables the SQL functions to accept explicit cast, which is required with JDBC.
* sql compatible tri-state native logical filters when druid.expressions.useStrictBooleans=true and druid.generic.useDefaultValueForNull=false, and new druid.generic.useThreeValueLogicForNativeFilters=true
* log.warn if non-default configurations are used to guide operators towards SQL complaint behavior
Instead of passing the constants around in a new parameter; InputAccessor was introduced to take care of transparently handling the constants - this new class started picking up some copy-paste debris around field accesses; and made them a little bit more readble.
The sql standard is not very restrictive regarding this:
If AVG is specified and DT is exact numeric, then the declared type of the result is an implemen-
tation-defined exact numeric type with precision not less than the precision of DT and scale not
less than the scale of DT.
so; using the same type is also ok (without patch);
however the avg of 0 and 1 is 0 right now because of the retention of the integer typ
Postgres,MySql and Oracle and Drill seem to increase precision ; mssql returns 0
http://sqlfiddle.com/#!9/6f7248/1
I think we should also increase precision as its already calculated more precisely
The aggregators had incorrect types for getResultType when shouldFinalze
is false. They had the finalized type, but they should have had the
intermediate type.
Also includes a refactor of how ExprMacroTable is handled in tests, to make
it easier to add tests for this to the MSQ module. The bug was originally
noticed because the incorrect result types caused MSQ queries with DS_HLL
to behave erratically.
* Update to Calcite 1.35.0
* Update from.ftl for Calcite 1.35.0.
* Fixed tests in Calcite upgrade by doing the following:
1. Added a new rule, CoreRules.PROJECT_FILTER_TRANSPOSE_WHOLE_PROJECT_EXPRESSIONS, to Base rules
2. Refactored the CorrelateUnnestRule
3. Updated CorrelateUnnestRel accordingly
4. Fixed a case with selector filters on the left where Calcite was eliding the virtual column
5. Additional test cases for fixes in 2,3,4
6. Update to StringListAggregator to fail a query if separators are not propagated appropriately
* Refactored for testcases to pass after the upgrade, introduced 2 new data sources for handling filters and select projects
* Added a literalSqlAggregator as the upgraded Calcite involved changes to subquery remove rule. This corrected plans for 2 queries with joins and subqueries by replacing an useless literal dimension with a post agg. Additionally a test with COUNT DISTINCT and FILTER which was failing with Calcite 1.21 is added here which passes with 1.35
* Updated to latest avatica and updated code as SqlUnknownTimeStamp is now used in Calcite which needs to be resolved to a timestamp literal
* Added a wrapper segment ref to use for unnest and filter segment reference
changes:
* new filters that preserve match value typing to better handle filtering different column types
* sql planner uses new filters by default in sql compatible null handling mode
* remove isFilterable from column capabilities
* proper handling of array filtering, add array processor to column processors
* javadoc for sql test filter functions
* range filter support for arrays, tons more tests, fixes
* add dimension selector tests for mixed type roots
* support json equality
* rename semantic index maker thingys to mostly have plural names since they typically make many indexes, e.g. StringValueSetIndex -> StringValueSetIndexes
* add cooler equality index maker, ValueIndexes
* fix missing string utf8 index supplier
* expression array comparator stuff
* Refactor HllSketchBuildAggregatorFactory
The usage of ColumnProcessors and HllSketchBuildColumnProcessorFactory
made it very difficult to figure out what was going on from just looking
at the AggregatorFactory or Aggregator code. It also didn't properly
double check that you could use UTF8 ahead of time, even though it's
entirely possible to validate it before trying to use it. This refactor
makes keeps the general indirection that had been implemented by
the Consumer<Supplier<HllSketch>> but centralizes the decision logic and
makes it easier to understand the code.
* Test fixes
* Add test that validates the types are maintained
* Add back indirection to avoid buffer calls
* Cover floats and doubles are the same thing
* Static checks
* Add "stringEncoding" parameter to DataSketches HLL.
Builds on the concept from #11172 and adds a way to feed HLL sketches
with UTF-8 bytes.
This must be an option rather than always-on, because prior to this
patch, HLL sketches used UTF-16LE encoding when hashing strings. To
remain compatible with sketch images created prior to this patch -- which
matters during rolling updates and when reading sketches that have been
written to segments -- we must keep UTF-16LE as the default.
Not currently documented, because I'm not yet sure how best to expose
this functionality to users. I think the first place would be in the SQL
layer: we could have it automatically select UTF-8 or UTF-16LE when
building sketches at query time. We need to be careful about this, though,
because UTF-8 isn't always faster. Sometimes, like for the results of
expressions, UTF-16LE is faster. I expect we will sort this out in
future patches.
* Fix benchmark.
* Fix style issues, improve test coverage.
* Put round back, to make IT updates easier.
* Fix test.
* Fix issue with filtered aggregators and add test.
* Use DS native update(ByteBuffer) method. Improve test coverage.
* Add another suppression.
* Fix ITAutoCompactionTest.
* Update benchmarks.
* Updates.
* Fix conflict.
* Adjustments.
* SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.
Four main changes:
1) Provide aggregatorBuilder, a more consistent way of defining the
SqlAggFunction we need for all of our SQL aggregators. The mechanism
is analogous to the one we already use for SQL functions
(OperatorConversions.operatorBuilder).
2) Allow CASTs of constants to be considered as "literalOperands". This
fixes an issue where various of our operators are defined with
OperandTypes.LITERAL as part of their checkers, which doesn't allow
casts. However, in these cases we generally _do_ want to allow casts.
The important piece is that the value must be reducible to a constant,
not that the SQL text is literally a literal.
3) Update DataSketches SQL aggregators to use the new aggregatorBuilder
functionality. The main user-visible effect here is [2]: the aggregators
would now accept, for example, "CAST(0.99 AS DOUBLE)" as a literal
argument. Other aggregators could be updated in a future patch.
4) Rename "requiredOperands" to "requiredOperandCount", because the
old name was confusing. (It rhymes with "literalOperands" but the
arguments mean different things.)
* Adjust method calls.
Introduce DruidException, an exception whose goal in life is to be delivered to a user.
DruidException itself has javadoc on it to describe how it should be used. This commit both introduces the Exception and adjusts some of the places that are generating exceptions to generate DruidException objects instead, as a way to show how the Exception should be used.
This work was a 3rd iteration on top of work that was started by Paul Rogers. I don't know if his name will survive the squash-and-merge, so I'm calling it out here and thanking him for starting on this.
* Fix EarliestLatestBySqlAggregator signature; Include function name for all signatures.
* Single quote function signatures, space between args and remove \n.
* fixup UT assertion
This PR is a follow-up to #13819 so that the Tuple sketch functionality can be used in SQL for both ingestion using Multi-Stage Queries (MSQ) and also for analytic queries against Tuple sketch columns.
You can now do the following operations with TupleSketches in Post Aggregation Step
Get the Sketch Output as Base64 String
Provide a constant Tuple Sketch in post-aggregation step that can be used in Set Operations
Get the Estimated Value(Sum) of Summary/Metrics Objects associated with Tuple Sketch