* Add IndexMergerRollupTest
This changelist adds a test to merge indexes with StringFirst/StringLast aggregator.
* Fix StringFirstAggregateCombiner/StringLastAggregateCombiner
The segment-level type for stringFirst/stringLast is SerializablePairLongString,
not String. This changelist fixes it.
* Fix EarliestLatestAnySqlAggregator to handle COMPLEX type
This changelist allows EarliestLatestAnySqlAggregator to accept COMPLEX
type as an operand. For its return type, we set it to VARCHAR, since
COMPLEX column is only generated by stringFirst/stringLast during ingestion
rollup.
* Return value with smaller timestamp in StringFirstAggregatorFactory.combine function
* Add integration tests for stringFirst/stringLast during ingestion
* Use one EarliestLatestReturnTypeInference instance
Co-authored-by: Joy Kent <joy@automonic.ai>
* SQL support for union datasources.
Exposed via the "UNION ALL" operator. This means that there are now two
different implementations of UNION ALL: one at the top level of a query
that works by concatenating subquery results, and one at the table level
that works by creating a UnionDataSource.
The SQL documentation is updated to discuss these two use cases and how
they behave.
Future work could unify these by building support for a native datasource
that represents the union of multiple subqueries. (Today, UnionDataSource
can only represent the union of tables, not subqueries.)
* Fixes.
* Error message for sanity check.
* Additional test fixes.
* Add some error messages.
* better type tracking: add typed postaggs, finalized types for agg factories
* more javadoc
* adjustments
* transition to getTypeName to be used exclusively for complex types
* remove unused fn
* adjust
* more better
* rename getTypeName to getComplexTypeName
* setup expression post agg for type inference existing
* more javadocs
* fixup
* oops
* more test
* more test
* more comments/javadoc
* nulls
* explicitly handle only numeric and complex aggregators for incremental index
* checkstyle
* more tests
* adjust
* more tests to showcase difference in behavior
* timeseries longsum array
* Add SQL "OFFSET" clause.
Under the hood, this uses the new offset features from #10233 (Scan)
and #10235 (GroupBy). Since Timeseries and TopN queries do not currently
have an offset feature, SQL planning will switch from one of those to
Scan or GroupBy if users add an OFFSET.
Includes a refactoring to harmonize offset and limit planning using an
OffsetLimit wrapper class. This is useful because it ensures that the
various places that need to deal with offset and limit collapsing all
behave the same way, using its "andThen" method.
* Fix test and add another test.
* Segment backed broadcast join IndexedTable
* fix comments
* fix tests
* sharing is caring
* fix test
* i hope this doesnt fix it
* filter by schema to maybe fix test
* changes
* close join stuffs so it does not leak, allow table to directly make selector factory
* oops
* update comment
* review stuffs
* better check
* remove DruidLeaderClient.goAsync(..) that does not follow redirect.
Replace its usage by DruidLeaadereClient.go(..) with
InputStreamFullResponseHandler
* remove ByteArrayResponseHolder dependency from JsonParserIterator
* add UT to cover lines in InputStreamFullResponseHandler
* refactor SystemSchema to reduce branches
* further reduce branches
* Revert "add UT to cover lines in InputStreamFullResponseHandler"
This reverts commit 330aba3dd9.
* UTs for InputStreamFullResponseHandler
* remove unused imports
* Add "offset" parameter to the Scan query.
It works by doing the query as normal and then throwing away the first
"offset" number of rows on the broker.
* Fix constructor call.
* Fix up JSONs.
* Fix call to ScanQuery.
* Doc update.
* Fix javadocs.
* Spotbugs, LGTM suppressions.
* Javadocs.
* Fix suppression.
* Stabilize Scan query result order, add tests.
* Update LGTM comment.
* Fixup.
* Test different batch sizes too.
* Nicer tests.
* Fix comment.
* LongMaxVectorAggregator support and test case.
* DoubleMinVectorAggregator and test cases.
* DoubleMaxVectorAggregator and unit test.
* FloatMinVectorAggregator and FloatMaxVectorAggregator.
* Documentation update to include the other vector aggregators.
* Bug fix.
* checkstyle formatting fixes.
* CalciteQueryTest cases update.
* Separate test classes for FloatMaxAggregation and FloatMniAggregation.
* remove the cannotVectorize for float max/min aggregator in test.
* Tests in GroupByQueryRunner, GroupByTimeseriesQueryRunner and TimeseriesQueryRunner.
* Add "offset" parameter to GroupBy query.
It works by doing the query as normal and then throwing away the first
"offset" number of rows on the broker.
* Stabilize GroupBy sorts.
* Fix inspections.
* Fix suppression.
* Fixups.
* Move TopNSequence to druid-core.
* Addl comments.
* NumberedElement equals verification.
* Changes from review.
* Fix minor formatting in docs.
* Add Nullhandling initialization for test to run from IDE.
* Vectorize longMin aggregator.
- A new vectorized class for the vectorized long min aggregator.
- Changes to AggregatorFactory to support vectorize functionality.
- Few changes to schema evolution test to add LongMinAggregatorFactory.
* Add longSum to the supported vectorized aggregator implementations.
* Add MIN() long min to calcite query test that can vectorize.
* Add simple long aggregations test.
* Fixup formatting per checkstyle guide.
* fixup and add more tests for long min aggregator.
* Override test for groupBy since timestamps are handled differently.
* Null compatibility check in test.
* Review comment: Add a test case to LongMinAggregationTest.
* Fix timeseries query constructor when postAggregator has an expression reading timestamp result column
* fix npe
* Fix postAgg referencing timestampResultField and add a test for it
* fix test
* doc
* revert doc
* new average aggregator
* method to create count aggregator factory
* test everything
* update other usages
* fix style
* fix more tests
* fix datasketches tests
* Ensure that join filter pre-analysis operates on optimized filters, add DimFilter.toOptimizedFilter
* Remove aggressive equality check that was used for testing
* Use Suppliers.memoize
* Checkstyle
* ROUND and having comparators correctly handle doubles
Double.NaN, Double.POSITIVE_INFINITY and Double.NEGATIVE_INFINITY are not real
numbers. Because of this, they can not be converted to BigDecimal and instead
throw a NumberFormatException.
This change adds support for calculations that produce these numbers either
for use in the `ROUND` function or the HavingSpecMetricComparator by not
attempting to convert the number to a BigDecimal.
The bug in ROUND was first introduced in #7224 where we added the ability to
round to any decimal place. This PR changes the behavior back to using
`Math.round` if we recognize a number that can not be converted to a
BigDecimal.
* Add tests and fix spellcheck
* update error message in ExpressionsTest
* Address comments
* fix up round for infinity
* round non numeric doubles returns a double
* fix spotbugs
* Update docs/misc/math-expr.md
* Update docs/querying/sql.md
* make joinables closeable
* tests and adjustments
* refactor to make join stuffs impelement ReferenceCountedObject instead of Closable, more tests
* fixes
* javadocs and stuff
* fix bugs
* more test
* fix lgtm alert
* simplify
* fixup javadoc
* review stuffs
* safeguard against exceptions
* i hate this checkstyle rule
* make IndexedTable extend Closeable
* fix groupBy with literal in subquery grouping
* fix groupBy with literal in subquery grouping
* fix groupBy with literal in subquery grouping
* address comments
* update javadocs
* Fix join
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* Fix Subquery could not be converted to groupBy query
* add tests
* address comments
* fix failing tests
* Add REGEXP_LIKE, fix empty-pattern bug in REGEXP_EXTRACT.
- Add REGEXP_LIKE function that returns a boolean, and is useful in
WHERE clauses.
- Fix REGEXP_EXTRACT return type (should be nullable; causes incorrect
filter elision).
- Fix REGEXP_EXTRACT behavior for empty patterns: should always match
(previously, they threw errors).
- Improve error behavior when REGEXP_EXTRACT and REGEXP_LIKE are passed
non-literal patterns.
- Improve documentation of REGEXP_EXTRACT.
* Changes based on PR review.
* Fix arg check.
* Important fixes!
* Add speller.
* wip
* Additional tests.
* Fix up tests.
* Add validation error tests.
* Additional tests.
* Remove useless call.
* Refactor JoinFilterAnalyzer
This patch attempts to make it easier to follow the join filter analysis code
with the hope of making it easier to add rewrite optimizations in the future.
To keep the patch small and easy to review, this is the first of at least 2
patches that are planned.
This patch adds a builder to the Pre-Analysis, so that it is easier to
instantiate the preAnalysis. It also moves some of the filter normalization
code out to Fitlers with associated tests.
* fix tests
* Refactor JoinFilterAnalyzer - part 2
This change introduces the following components:
* RhsRewriteCandidates - a wrapper for a list of candidates and associated
functions to operate on the set of candidates.
* JoinableClauses - a wrapper for the list of JoinableClause that represent
a join condition and the associated functions to operate on the clauses.
* Equiconditions - a wrapper representing the equiconditions that are used
in the join condition.
And associated test changes.
This refactoring surfaced 2 bugs:
- Missing equals and hashcode implementation for RhsRewriteCandidate, thus
allowing potential duplicates in the rhs rewrite candidates
- Missing Filter#supportsRequiredColumnRewrite check in
analyzeJoinFilterClause, which could result in UnsupportedOperationException
being thrown by the filter
* fix compile error
* remove unused class
* Refactor JoinFilterAnalyzer - Correlations
Move the correlation related code out into it's own class so it's easier
to maintain.
Another patch should follow this one so that the query path uses the
correlation object instead of it's underlying maps.
* Optimize join queries where filter matches nothing
Fixes#9787
This PR changes the Joinable interface to return an Optional set of correlated
values for a column.
This allows the JoinFilterAnalyzer to differentiate between the case where the
column has no matching values and when the column could not find matching
values.
This PR chose not to distinguish between cases where correlated values could
not be computed because of a config that has this behavior disabled or because
of user error - like a column that could not be found. The reasoning was that
the latter is likely an error and the non filter pushdown path will surface the
error if it is.
* Add ingestion specs for CalciteQueryTests
This PR introduces ingestion specs that can be used for local testing
so that CalciteQueryTests can be built on a druid cluster.
* Add README
* Update sql/src/test/resources/calcite/tests/README.md
* Bad plan for table-lookup-lookup join with filter on first lookup and outer limit
* Bad plan for table-lookup-lookup join with filter on first lookup and outer limit
* Bad plan for table-lookup-lookup join with filter on first lookup and outer limit
* Bad plan for table-lookup-lookup join with filter on first lookup and outer limit
* Bad plan for table-lookup-lookup join with filter on first lookup and outer limit
* Bad plan for table-lookup-lookup join with filter on first lookup and outer limit
* address comments
* address comments
* fix checkstyle
* address comments
* address comments
* Fix potential NPEs in joins
intelliJ reported issues with potential NPEs. This was first hit in testing
with a filter being pushed down to the left hand table when joining against
an indexed table.
* More null check cleanup
* Optimize filter value rewrite for IndexedTable
* Add unit tests for LookupJoinable
* Add tests for IndexedTableJoinable
* Add non null assert for dimension selector
* Supress null warning in LookupJoinMatcher
* remove some null checks on hot path
* fix issue with group by limit pushdown for extractionFn, expressions, joins, etc
* remove unused
* fix test
* revert unintended change
* more tests
* consider capabilities for StringGroupByColumnSelectorStrategy
* fix test
* fix and more test
* revert because im scared
* SQL: More straightforward handling of join planning.
Two changes that simplify how joins are planned:
1) Stop using JoinProjectTransposeRule as a way of guiding subquery
removal. Instead, add logic to DruidJoinRule that identifies removable
subqueries and removes them at the point of creating a DruidJoinQueryRel.
This approach reduces the size of the planning space and allows the
planner to complete quickly.
2) Remove rules that reorder joins. Not because of an impact on the
planning time (it seems minimal), but because the decisions that the
planner was making in the new tests were sometimes worse than the
user-provided order. I think we'll need to go with the user-provided
order for now, and revisit reordering when we can add more smarts to
the cost estimator.
A third change updates numeric ExprEval classes to store their
value as a boxed type that corresponds to what it is supposed to be.
This is useful because it affects the behavior of "asString", and
is included in this patch because it is needed for the new test
"testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse". This
test relies on CAST('6', 'DOUBLE') stringifying to "6.0" like an
actual double would.
Fixes#9646.
* Fix comments.
* Fix tests.
* SQL support for joins on subqueries.
Changes to SQL module:
- DruidJoinRule: Allow joins on subqueries (left/right are no longer
required to be scans or mappings).
- DruidJoinRel: Add cost estimation code for joins on subqueries.
- DruidSemiJoinRule, DruidSemiJoinRel: Removed, since DruidJoinRule can
handle this case now.
- DruidRel: Remove Nullable annotation from toDruidQuery, because
it is no longer needed (it was used by DruidSemiJoinRel).
- Update Rules constants to reflect new rules available in our current
version of Calcite. Some of these are useful for optimizing joins on
subqueries.
- Rework cost estimation to be in terms of cost per row, and place all
relevant constants in CostEstimates.
Other changes:
- RowBasedColumnSelectorFactory: Don't set hasMultipleValues. The lack
of isComplete is enough to let callers know that columns might have
multiple values, and explicitly setting it to true causes
ExpressionSelectors to think it definitely has multiple values, and
treat the inputs as arrays. This behavior interfered with some of the
new tests that involved queries on lookups.
- QueryContexts: Add maxSubqueryRows parameter, and use it in druid-sql
tests.
* Fixes for tests.
* Adjustments.
* Broker: Add ability to inline subqueries.
The main changes:
- ClientQuerySegmentWalker: Add ability to inline queries.
- Query: Add "getSubQueryId" and "withSubQueryId" methods.
- QueryMetrics: Add "subQueryId" dimension.
- ServerConfig: Add new "maxSubqueryRows" parameter, which is used by
ClientQuerySegmentWalker to limit how many rows can be inlined per
query.
- IndexedTableJoinMatcher: Allow creating keys on top of unknown types,
by assuming they are strings. This is useful because not all types are
known for fields in query results.
- InlineDataSource: Store RowSignature rather than component parts. Add
more zealous "equals" and "hashCode" methods to ease testing.
- Moved QuerySegmentWalker test code from CalciteTests and
SpecificSegmentsQueryWalker in druid-sql to QueryStackTests in
druid-server. Use this to spin up a new ClientQuerySegmentWalkerTest.
* Adjustments from CI.
* Fix integration test.
* Match GREATEST/LEAST function behavior
Change the behavior of the GREATEST / LEAST functions to be similar to
how it is implemented in other databases (as functions instead of
aggregators). The GREATEST/LEAST functions are not in the SQL standard,
but users will expect behavior similar to what other databases provide.
* Match postgres behavior & handle more SQL types
* Fix imports
* Move RowSignature from druid-sql to druid-processing and make use of it.
1) Moved (most of) RowSignature from sql to processing. Left behind the SQL-specific
stuff in a RowSignatures utility class. It also picked up some new convenience
methods along the way.
2) There were a lot of places in the code where Map<String, ValueType> was used to
associate columns with type info. These are now all replaced with RowSignature.
3) QueryToolChest's resultArrayFields method is replaced with resultArraySignature,
and it now provides type info.
* Fix up extensions.
* Various fixes
* Link up row-based datasources to serving layer.
- Add SegmentWrangler interface that allows linking of DataSources to Segments.
- Add LocalQuerySegmentWalker that uses SegmentWranglers to compute queries on
data that is available locally.
- Modify ClientQuerySegmentWalker to use LocalQuerySegmentWalker when the base
datasource is concrete and not a table.
- Add SegmentWranglerModule to the Broker so it has them available and can
properly instantiate . LocalQuerySegmentWalkers.
- Set InlineDataSource and LookupDataSource to concrete, since they can be
directly queried now.
* Fix tests.
* Add SQL GROUPING SETS support.
Built on top of the subtotalsSpec feature in the groupBy query. This also involves
two changes to subtotalsSpec:
- Alter behavior so limitSpec is applied after subtotalsSpec, rather than applied to
each grouping set. This is more in line with SQL standard behavior. I think it is okay
to make this change, since the old behavior was not documented, so users should
hopefully not be depending on it.
- Fix a bug where virtual columns were included in the subtotal queries, but they
should not have been.
Also fixes two bugs in query equality checking:
- BaseQuery: Use getDuration() instead of "duration" in equals and hashCode, since the
latter is lazily initialized and might be null in one query but not the other.
- GroupByQuery: Include subtotalsSpec in equals and hashCode.
* Fix bugs.
* Fix tests.
* PR updates.
* Grouping class hygiene.
* add Expr.stringify which produces parseable expression strings, parser support for null values in arrays, and parser support for empty numeric arrays
* oops, macros are expressions too
* style
* spotbugs
* qualified type arrays
* review stuffs
* simplify grammar
* more permissive array parsing
* reuse expr joiner
* fix it
* Fix timestamp extract fn to match postgres
Update the timestamp extract function so that it matches the PostgreSQL docs.
Examples from the PostgreSQL docs were added as tests for DECADE, CENTURY
and MILLENIUM extraction.
There were bugs in CENTURY and MILLENIUM that were spotted because of intelliJ
inspections - 'Integer division in floating point context'
* Update CalciteQueryTest
* remove useless round
* mark integer division as an error
* Use ExecutorService instead of ScheduledExecutorService where necessary - #9286
* Added inspection rule to prohibit ScheduledExecutorService assignment to ExecutorService
* Remove EasyMock dependency from CalciteTests.
Useful because CalciteTests is used by other modules (e.g. druid-benchmarks)
and we don't want them to have to pull in EasyMock.
* CalciteTests no longer needs curator-x-discovery either.
* implement shell for greatest sql aggregator with hardcoded long values
* implement functional long greatest aggregator for direct access columns
* implement greatest & least sql aggregators for long & double types using abstract base class
* add javadocs, unit tests & handling for floats for greatest/least postaggregations
* minor checkstyle fix
* improve naming for the test cases
* make inner class static
* remove blank lines to retest travis build
* change trivial text to rerun travis build
* implement suggested updates for greatest/least sql aggs & fix checkstyle issues
* fix stale comments in greatest/least sql aggs abstract base
* Update sql.md
* improve sql function definitions for greatest/least sql aggs
* add more tests for greatest/least sql aggs
* add tests to cover invalid greatest/least sql expressions
* rename & reorder greatest least sql tests
* Guicify druid sql module
Break up the SQLModule in to smaller modules and provide a binding that
modules can use to register schemas with druid sql.
* fix some tests
* address code review
* tests compile
* Working tests
* Add all the tests
* fix up licenses and dependencies
* add calcite dependency to druid-benchmarks
* tests pass
* rename the schemas
* SQL join support for lookups.
1) Add LookupSchema to SQL, so lookups show up in the catalog.
2) Add join-related rels and rules to SQL, allowing joins to be planned into
native Druid queries.
* Add two missing LookupSchema calls in tests.
* Fix tests.
* Fix typo.
* Add LookupJoinableFactory.
Enables joins where the right-hand side is a lookup. Includes an
integration test.
Also, includes changes to LookupExtractorFactoryContainerProvider:
1) Add "getAllLookupNames", which will be needed to eventually connect
lookups to Druid's SQL catalog.
2) Convert "get" from nullable to Optional return.
3) Swap out most usages of LookupReferencesManager in favor of the
simpler LookupExtractorFactoryContainerProvider interface.
* Fixes for tests.
* Fix another test.
* Java 11 message fix.
* Fixups.
* Fixup benchmark class.
* intelliJ inspections cleanup
- remove redundant escapes
- performance warnings
- access static member via instance reference
- static method declared final
- inner class may be static
Most of these changes are aesthetic, however, they will allow inspections to
be enabled as part of CI checks going forward
The valuable changes in this delta are:
- using StringBuilder instead of string addition in a loop
indexing-hadoop/.../Utils.java
processing/.../ByteBufferMinMaxOffsetHeap.java
- Use class variables instead of static variables for parameterized test
processing/src/.../ScanQueryLimitRowIteratorTest.java
* Add intelliJ inspection warnings as errors to druid profile
* one more static inner class
* Reconcile terminology and method naming to 'used/unused segments'; Don't use terms 'enable/disable data source'; Rename MetadataSegmentManager to MetadataSegments; Make REST API methods which mark segments as used/unused to return server error instead of an empty response in case of error
* Fix brace
* Import order
* Rename withKillDataSourceWhitelist to withSpecificDataSourcesToKill
* Fix tests
* Fix tests by adding proper methods without interval parameters to IndexerMetadataStorageCoordinator instead of hacking with Intervals.ETERNITY
* More aligned names of DruidCoordinatorHelpers, rename several CoordinatorDynamicConfig parameters
* Rename ClientCompactTaskQuery to ClientCompactionTaskQuery for consistency with CompactionTask; ClientCompactQueryTuningConfig to ClientCompactionTaskQueryTuningConfig
* More variable and method renames
* Rename MetadataSegments to SegmentsMetadata
* Javadoc update
* Simplify SegmentsMetadata.getUnusedSegmentIntervals(), more javadocs
* Update Javadoc of VersionedIntervalTimeline.iterateAllObjects()
* Reorder imports
* Rename SegmentsMetadata.tryMark... methods to mark... and make them to return boolean and the numbers of segments changed and relay exceptions to callers
* Complete merge
* Add CollectionUtils.newTreeSet(); Refactor DruidCoordinatorRuntimeParams creation in tests
* Remove MetadataSegmentManager
* Rename millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments to leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments
* Fix tests, refactor DruidCluster creation in tests into DruidClusterBuilder
* Fix inspections
* Fix SQLMetadataSegmentManagerEmptyTest and rename it to SqlSegmentsMetadataEmptyTest
* Rename SegmentsAndMetadata to SegmentsAndCommitMetadata to reduce the similarity with SegmentsMetadata; Rename some methods
* Rename DruidCoordinatorHelper to CoordinatorDuty, refactor DruidCoordinator
* Unused import
* Optimize imports
* Rename IndexerSQLMetadataStorageCoordinator.getDataSourceMetadata() to retrieveDataSourceMetadata()
* Unused import
* Update terminology in datasource-view.tsx
* Fix label in datasource-view.spec.tsx.snap
* Fix lint errors in datasource-view.tsx
* Doc improvements
* Another attempt to please TSLint
* Another attempt to please TSLint
* Style fixes
* Fix IndexerSQLMetadataStorageCoordinator.createUsedSegmentsSqlQueryForIntervals() (wrong merge)
* Try to fix docs build issue
* Javadoc and spelling fixes
* Rename SegmentsMetadata to SegmentsMetadataManager, address other comments
* Address more comments
Builds on #9235, using the datasource analysis functionality to replace various ad-hoc
approaches. The most interesting changes are in ClientQuerySegmentWalker (brokers),
ServerManager (historicals), and SinkQuerySegmentWalker (indexing tasks).
Other changes related to improving how we analyze queries:
1) Changes TimelineServerView to return an Optional timeline, which I thought made
the analysis changes cleaner to implement.
2) Added QueryToolChest#canPerformSubquery, which is now used by query entry points to
determine whether it is safe to pass a subquery dataSource to the query toolchest.
Fixes an issue introduced in #5471 where subqueries under non-groupBy-typed queries
were silently ignored, since neither the query entry point nor the toolchest did
anything special with them.
3) Removes the QueryPlus.withQuerySegmentSpec method, which was mostly being used in
error-prone ways (ignoring any potential subqueries, and not verifying that the
underlying data source is actually a table). Replaces with a new function,
Queries.withSpecificSegments, that includes sanity checks.
* Add join-related DataSource types, and analysis functionality.
Builds on #9111 and implements the datasource analysis mentioned in #8728. Still can't
handle join datasources, but we're a step closer.
Join-related DataSource types:
1) Add "join", "lookup", and "inline" datasources.
2) Add "getChildren" and "withChildren" methods to DataSource, which will be used
in the future for query rewriting (e.g. inlining of subqueries).
DataSource analysis functionality:
1) Add DataSourceAnalysis class, which breaks down datasources into three components:
outer queries, a base datasource (left-most of the highest level left-leaning join
tree), and other joined-in leaf datasources (the right-hand branches of the
left-leaning join tree).
2) Add "isConcrete", "isGlobal", and "isCacheable" methods to DataSource in order to
support analysis.
Other notes:
1) Renamed DataSource#getNames to DataSource#getTableNames, which I think is clearer.
Also, made it a Set, so implementations don't need to worry about duplicates.
2) The addition of "isCacheable" should work around #8713, since UnionDataSource now
returns false for cacheability.
* Remove javadoc comment.
* Updates reflecting code review.
* Add comments.
* Add more comments.
* null handling for numeric first/last aggregators, refactor to not extend nullable numeric agg since they are complex typed aggs
* initially null or not based on config
* review stuff, make string first/last consistent with null handling of numeric columns, more tests
* docs
* handle nil selectors, revert to primitive first/last types so groupby v1 works...
AggregateCaseToFilterRule was added to Calcite in https://issues.apache.org/jira/browse/CALCITE-3144,
and was originally copied from Druid's CaseFilteredAggregatorRule. So there isn't a good reason to
keep using our version.
* Fix NPE for subquery with limit
* Mark it as unplannable by returning null
* Migrate testcases from SqlResourceTest to CalciteQueryTest
* Throw CannotBuildQueryException
* Fix typo
* Patch comments
* fix bug with sqlOuterLimit, use sqlOuterLimit instead of wrapping sql query for web console
* fixes, refactors, tests
* meh
* better name
* fix comment location
* fix copy and paste