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
* merge druid-core, extendedset, and druid-hll into druid-processing to simplify everything
* fix poms and license stuff
* mockito is evil
* allow reset of JvmUtils RuntimeInfo if tests used static injection to override
* Use an HllSketchHolder object to enable optimized merge
HllSketchAggregatorFactory.combine had been implemented using a
pure pair-wise, "make a union -> add 2 things to union -> get sketch"
algorithm. This algorithm does 2 things that was CPU
1) The Union object always builds an HLL_8 sketch regardless of the
target type. This means that when the target type is not HLL_8, we
spent CPU cycles converting to HLL_8 and back over and over again
2) By throwing away the Union object and converting back to the
HllSketch only to build another Union object, we do lots and lots
of copy+conversions of the HllSketch
This change introduces an HllSketchHolder object which can hold onto
a Union object and delay conversion back into an HllSketch until
it is actually needed. This follows the same pattern as the
SketchHolder object for theta sketches.
* Fallback virtual column
This virtual columns enables falling back to another column if
the original column doesn't exist. This is useful when doing
column migrations and you have some old data with column X,
new data with column Y and you want to use Y if it exists, X
otherwise so that you can run a consistent query against all of
the data.
SQL test framework extensions
* Capture planner artifacts: logical plan, etc.
* Planner test builder validates the logical plan
* Validation for the SQL resut schema (we already have
validation for the Druid row signature)
* Better Guice integration: properties, reuse Guice modules
* Avoid need for hand-coded expr, macro tables
* Retire some of the test-specific query component creation
* Fix query log hook race condition
* we can read where we want to
we can leave your bounds behind
'cause if the memory is not there
we really don't care
and we'll crash this process of mine
* Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH.
These aggregation functions are documented as creating sketches. However,
they are planned into native aggregators that include finalization logic
to convert the sketch to a number of some sort. This creates an
inconsistency: the functions sometimes return sketches, and sometimes
return numbers, depending on where they lie in the native query plan.
This patch changes these SQL aggregators to _never_ finalize, by using
the "shouldFinalize" feature of the native aggregators. It already
existed for theta sketches. This patch adds the feature for hll and
quantiles sketches.
As to impact, Druid finalizes aggregators in two cases:
- When they appear in the outer level of a query (not a subquery).
- When they are used as input to an expression or finalizing-field-access
post-aggregator (not any other kind of post-aggregator).
With this patch, the functions will no longer be finalized in these cases.
The second item is not likely to matter much. The SQL functions all declare
return type OTHER, which would be usable as an input to any other function
that makes sense and that would be planned into an expression.
So, the main effect of this patch is the first item. To provide backwards
compatibility with anyone that was depending on the old behavior, the
patch adds a "sqlFinalizeOuterSketches" query context parameter that
restores the old behavior.
Other changes:
1) Move various argument-checking logic from runtime to planning time in
DoublesSketchListArgBaseOperatorConversion, by adding an OperandTypeChecker.
2) Add various JsonIgnores to the sketches to simplify their JSON representations.
3) Allow chaining of ExpressionPostAggregators and other PostAggregators
in the SQL layer.
4) Avoid unnecessary FieldAccessPostAggregator wrapping in the SQL layer,
now that expressions can operate on complex inputs.
5) Adjust return type to thetaSketch (instead of OTHER) in
ThetaSketchSetBaseOperatorConversion.
* Fix benchmark class.
* Fix compilation error.
* Fix ThetaSketchSqlAggregatorTest.
* Hopefully fix ITAutoCompactionTest.
* Adjustment to ITAutoCompactionTest.
* Refactor Calcite test "framework" for planner tests
Refactors the current Calcite tests to make it a bit easier
to adjust the set of runtime objects used within a test.
* Move data creation out of CalciteTests into TestDataBuilder
* Move "framework" creation out of CalciteTests into
a QueryFramework
* Move injector-dependent functions from CalciteTests
into QueryFrameworkUtils
* Wrapper around the planner factory, etc. to allow
customization.
* Bulk of the "framework" created once per class rather
than once per test.
* Refactor tests to use a test builder
* Change all testQuery() methods to use the test builder.
Move test execution & verification into a test runner.
We introduce two new configuration keys that refine the query context security model controlled by druid.auth.authorizeQueryContextParams. When that value is set to true then two other configuration options become available:
druid.auth.unsecuredContextKeys: The set of query context keys that do not require a security check. Use this for the "white-list" of key to allow. All other keys go through the existing context key security checks.
druid.auth.securedContextKeys: The set of query context keys that do require a security check. Use this when you want to allow all but a specific set of keys: only these keys go through the existing context key security checks.
Both are set using JSON list format:
druid.auth.securedContextKeys=["secretKey1", "secretKey2"]
You generally set one or the other values. If both are set, unsecuredContextKeys acts as exceptions to securedContextKeys.
In addition, Druid defines two query context keys which always bypass checks because Druid uses them internally:
sqlQueryId
sqlStringifyArrays
* KLL sketch
* added documentation
* direct static refs
* direct static refs
* fixed test
* addressed review points
* added KLL sketch related terms
* return a copy from get
* Copy unions when returning them from "get".
* Remove redundant "final".
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
* Refactor SqlLifecycle into statement classes
Create direct & prepared statements
Remove redundant exceptions from tests
Tidy up Calcite query tests
Make PlannerConfig more testable
* Build fixes
* Added builder to SqlQueryPlus
* Moved Calcites system properties to saffron.properties
* Build fix
* Resolve merge conflict
* Fix IntelliJ inspection issue
* Revisions from reviews
Backed out a revision to Calcite tests that didn't work out as planned
* Build fix
* Fixed spelling errors
* Fixed failed test
Prepare now enforces security; before it did not.
* Rebase and fix IntelliJ inspections issue
* Clean up exception handling
* Fix handling of JDBC auth errors
* Build fix
* More tweaks to security messages
* GroupBy: Reduce allocations by reusing entry and key holders.
Two main changes:
1) Reuse Entry objects returned by various implementations of
Grouper.iterator.
2) Reuse key objects contained within those Entry objects.
This is allowed by the contract, which states that entries must be
processed and immediately discarded. However, not all call sites
respected this, so this patch also updates those call sites.
One particularly sneaky way that the old code retained entries too long
is due to Guava's MergingIterator and CombiningIterator. Internally,
these both advance to the next value prior to returning the current
value. So, this patch addresses that in two ways:
1) For merging, we have our own implementation MergeIterator already,
although it had the same problem. So, this patch updates our
implementation to return the current item prior to advancing to the
next item. It also adds a forbidden-api entry to ensure that this
safer implementation is used instead of Guava's.
2) For combining, we address the problem in a different way: by copying
the key when creating the new, combined entry.
* Attempt to fix test.
* Remove unused import.
The query context is a way that the user gives a hint to the Druid query engine, so that they enforce a certain behavior or at least let the query engine prefer a certain plan during query planning. Today, there are 3 types of query context params as below.
Default context params. They are set via druid.query.default.context in runtime properties. Any user context params can be default params.
User context params. They are set in the user query request. See https://druid.apache.org/docs/latest/querying/query-context.html for parameters.
System context params. They are set by the Druid query engine during query processing. These params override other context params.
Today, any context params are allowed to users. This can cause
1) a bad UX if the context param is not matured yet or
2) even query failure or system fault in the worst case if a sensitive param is abused, ex) maxSubqueryRows.
This PR adds an ability to limit context params per user role. That means, a query will fail if you have a context param set in the query that is not allowed to you. To do that, this PR adds a new built-in resource type, QUERY_CONTEXT. The resource to authorize has a name of the context param (such as maxSubqueryRows) and the type of QUERY_CONTEXT. To allow a certain context param for a user, the user should be granted WRITE permission on the context param resource. Here is an example of the permission.
{
"resourceAction" : {
"resource" : {
"name" : "maxSubqueryRows",
"type" : "QUERY_CONTEXT"
},
"action" : "WRITE"
},
"resourceNamePattern" : "maxSubqueryRows"
}
Each role can have multiple permissions for context params. Each permission should be set for different context params.
When a query is issued with a query context X, the query will fail if the user who issued the query does not have WRITE permission on the query context X. In this case,
HTTP endpoints will return 403 response code.
JDBC will throw ForbiddenException.
Note: there is a context param called brokerService that is used only by the router. This param is used to pin your query to run it in a specific broker. Because the authorization is done not in the router, but in the broker, if you have brokerService set in your query without a proper permission, your query will fail in the broker after routing is done. Technically, this is not right because the authorization is checked after the context param takes effect. However, this should not cause any user-facing issue and thus should be OK. The query will still fail if the user doesn’t have permission for brokerService.
The context param authorization can be enabled using druid.auth.authorizeQueryContextParams. This is disabled by default to avoid any hassle when someone upgrades his cluster blindly without reading release notes.
These changes are to use the latest datasketches-java-3.1.0 and also to restore support for quantile and HLL4 sketches to be able to grow larger than a given buffer in a buffer aggregator and move to heap in rare cases. This was discussed in #11544.
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
* rework sql planner expression and virtual column handling
* simplify a bit
* add back and deprecate old methods, more tests, fix multi-value string coercion bug and associated tests
* spotbugs
* fix bugs with multi-value string array expression handling
* javadocs and adjust test
* better
* fix tests
Fixes#12022
### Description
The current implementations of memory estimation in `OnHeapIncrementalIndex` and `StringDimensionIndexer` tend to over-estimate which leads to more persistence cycles than necessary.
This PR replaces the max estimation mechanism with getting the incremental memory used by the aggregator or indexer at each invocation of `aggregate` or `encode` respectively.
### Changes
- Add new flag `useMaxMemoryEstimates` in the task context. This overrides the same flag in DefaultTaskConfig i.e. `druid.indexer.task.default.context` map
- Add method `AggregatorFactory.factorizeWithSize()` that returns an `AggregatorAndSize` which contains
the aggregator instance and the estimated initial size of the aggregator
- Add method `Aggregator.aggregateWithSize()` which returns the incremental memory used by this aggregation step
- Update the method `DimensionIndexer.processRowValsToKeyComponent()` to return the encoded key component as well as its effective size in bytes
- Update `OnHeapIncrementalIndex` to use the new estimations only if `useMaxMemoryEstimates = false`
* Ingestion will fail for HLLSketchBuild instead of creating with incorrect values
* Addressing review comments for HLL< updated error message introduced test case
* Code cleanup from query profile project
* Fix spelling errors
* Fix Javadoc formatting
* Abstract out repeated test code
* Reuse constants in place of some string literals
* Fix up some parameterized types
* Reduce warnings reported by Eclipse
* Reverted change due to lack of tests
Add a "guessAggregatorHeapFootprint" method to AggregatorFactory that
mitigates #6743 by enabling heap footprint estimates based on a specific
number of rows. The idea is that at ingestion time, the number of rows
that go into an aggregator will be 1 (if rollup is off) or will likely
be a small number (if rollup is on).
It's a heuristic, because of course nothing guarantees that the rollup
ratio is a small number. But it's a common case, and I expect this logic
to go wrong much less often than the current logic. Also, when it does
go wrong, users can fix it by lowering maxRowsInMemory or
maxBytesInMemory. The current situation is unintuitive: when the
estimation goes wrong, users get an OOME, but actually they need to
*raise* these limits to fix it.
* add back and deprecate aggregator factory methods so i can say i told you so when i delete these later
* rename to make less ambiguous, fix fill method
* adjust
* add missing json type for ListFilteredVirtualColumn, and tests to try to avoid this happening again
* fixes
* ugly, but maybe this
* oops
* too many mappers
* Remove OffheapIncrementalIndex and clarify aggregator thread-safety needs.
This patch does the following:
- Removes OffheapIncrementalIndex.
- Clarifies that Aggregators are required to be thread safe.
- Clarifies that BufferAggregators and VectorAggregators are not
required to be thread safe.
- Removes thread safety code from some DataSketches aggregators that
had it. (Not all of them did, and that's OK, because it wasn't necessary
anyway.)
- Makes enabling "useOffheap" with groupBy v1 an error.
Rationale for removing the offheap incremental index:
- It is only used in one rare scenario: groupBy v1 (which is non-default)
in "useOffheap" mode (also non-default). So you have to go pretty deep
into the wilderness to get this code to activate in production. It is
never used during ingestion.
- Its existence complicates developer efforts to reason about how
aggregators get used, because the way it uses buffer aggregators is so
different from how every other query engine uses them.
- It doesn't have meaningful testing.
By the way, I do believe that the given way the offheap incremental index
works, it actually didn't require buffer aggregators to be thread-safe.
It synchronizes on "aggregate" and doesn't call "get" until it has
stopped calling "aggregate". Nevertheless, this is a bother to think about,
and for the above reasons I think it makes sense to remove the code anyway.
* Remove things that are now unused.
* Revert removal of getFloat, getLong, getDouble from BufferAggregator.
* OAK-related warnings, suppressions.
* Unused item suppressions.
* Add druid.sql.approxCountDistinct.function property.
The new property allows admins to configure the implementation for
APPROX_COUNT_DISTINCT and COUNT(DISTINCT expr) in approximate mode.
The motivation for adding this setting is to enable site admins to
switch the default HLL implementation to DataSketches.
For example, an admin can set:
druid.sql.approxCountDistinct.function = APPROX_COUNT_DISTINCT_DS_HLL
* Fixes
* Fix tests.
* Remove erroneous cannotVectorize.
* Remove unused import.
* Remove unused test imports.
* SQL: Allow Scans to be used as outer queries.
This has been possible in the native query system for a while, but the capability
hasn't yet propagated into the SQL layer. One example of where this is useful is
a query like:
SELECT * FROM (... LIMIT X) WHERE <filter>
Because this expands the kinds of subquery structures the SQL layer will consider,
it was also necessary to improve the cost calculations. These changes appear in
PartialDruidQuery and DruidOuterQueryRel. The ideas are:
- Attach per-column penalties to the output signature of each query, instead of to
the initial projection that starts a query. This encourages moving projections
into subqueries instead of leaving them on outer queries.
- Only attach penalties to projections if there are actually expressions happening.
So, now, projections that simply reorder or remove fields are free.
- Attach a constant penalty to every outer query. This discourages creating them
when they are not needed.
The changes are generally beneficial to the test cases we have in CalciteQueryTest.
Most plans are unchanged, or are changed in purely cosmetic ways. Two have changed
for the better:
- testUsingSubqueryWithLimit now returns a constant from the subquery, instead of
returning every column.
- testJoinOuterGroupByAndSubqueryHasLimit returns a minimal set of columns from
the innermost subquery; two unnecessary columns are no longer there.
* Fix various DS operator conversions.
These were all implemented as direct conversions, which isn't appropriate
because they do not actually map onto native functions. These are only
usable as post-aggregations.
* Test case adjustment.
* Null handling fixes for DS HLL and Theta sketches.
For HLL, this fixes an NPE when processing a null in a multi-value dimension.
For both, empty strings are now properly treated as nulls (and ignored) in
replace-with-default mode. Behavior in SQL-compatible mode is unchanged.
* Fix expectation.
* add ColumnInspector argument to PostAggregator.getType to allow post-aggs to compute their output type based on input types
* add test for test for coverage
* simplify
* Remove unused imports.
Co-authored-by: Gian Merlino <gian@imply.io>
* latest datasketches-java and datasketches-memory
* updated versions of datasketches-java and datasketches-memory
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
* better type system
* needle in a haystack
* ColumnCapabilities is a TypeSignature instead of having one, INFORMATION_SCHEMA support
* fixup merge
* more test
* fixup
* intern
* fix
* oops
* oops again
* ...
* more test coverage
* fix error message
* adjust interning, more javadocs
* oops
* more docs more better
* Configurable maxStreamLength for doubles sketches
* fix equals/hashcode and it test failure
* fix test
* fix it test
* benchmark
* doc
* grouping key
* fix comment
* dependency check
* Update docs/development/extensions-core/datasketches-quantiles.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* fix count and average SQL aggregators on constant virtual columns
* style
* even better, why are we tracking virtual columns in aggregations at all if we have a virtual column registry
* oops missed a few
* remove unused
* this will fix it
* SQL timeseries no longer skip empty buckets with all granularity
* add comment, fix tests
* the ol switcheroo
* revert unintended change
* docs and more tests
* style
* make checkstyle happy
* docs fixes and more tests
* add docs, tests for array_agg
* fixes
* oops
* doc stuffs
* fix compile, match doc style
* Vectorize the DataSketches quantiles aggregator.
Also removes synchronization for the BufferAggregator and VectorAggregator
implementations, since it is not necessary (similar to #11115).
Extends DoublesSketchAggregatorTest and DoublesSketchSqlAggregatorTest
to run all test cases in vectorized mode.
* Style fix.
* Vectorized versions of HllSketch aggregators.
The patch uses the same "helper" approach as #10767 and #10304, and
extends the tests to run in both vectorized and non-vectorized modes.
Also includes some minor changes to the theta sketch vector aggregator:
- Cosmetic changes to make the hll and theta implementations look
more similar.
- Extends the theta SQL tests to run in vectorized mode.
* Updates post-code-review.
* Fix javadoc.
* use the latest Apache DataSketches release 2.0.0
* updated datasketches version
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
* Vectorized theta sketch aggregator.
Also a refactoring of BufferAggregator and VectorAggregator such that
they share a common interface, BaseBufferAggregator. This allows
implementing both in the same file with an abstract + dual subclass
structure.
* Rework implementation to use composition instead of inheritance.
* Rework things to enable working properly for both complex types and
regular types.
Involved finally moving makeVectorProcessor from DimensionHandlerUtils
into ColumnProcessors and harmonizing the two things.
* Add missing method.
* Style and name changes.
* Fix issues from inspections.
* Fix style issue.
* 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
* new average aggregator
* method to create count aggregator factory
* test everything
* update other usages
* fix style
* fix more tests
* fix datasketches tests
* 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.
* IntelliJ inspections cleanup
* Standard Charset object can be used
* Redundant Collection.addAll() call
* String literal concatenation missing whitespace
* Statement with empty body
* Redundant Collection operation
* StringBuilder can be replaced with String
* Type parameter hides visible type
* fix warnings in test code
* more test fixes
* remove string concatenation inspection error
* fix extra curly brace
* cleanup AzureTestUtils
* fix charsets for RangerAdminClient
* review comments
* 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.
* 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
* Harmonization and bug-fixing for selector and filter behavior on unknown types.
- Migrate ValueMatcherColumnSelectorStrategy to newer ColumnProcessorFactory
system, and set defaultType COMPLEX so unknown types can be dynamically matched.
- Remove ValueGetters in favor of ColumnComparisonFilter doing its own thing.
- Switch various methods to use convertObjectToX when casting to numbers, rather
than ad-hoc and inconsistent logic.
- Fix bug in RowBasedExpressionColumnValueSelector: isBindingArray should return
true even for 0- or 1- element arrays.
- Adjust various javadocs.
* Add throwParseExceptions option to Rows.objectToNumber, switch back to that.
* Update tests.
* Adjust moment sketch tests.
* Add MemoryOpenHashTable, a table similar to ByteBufferHashTable.
With some key differences to improve speed and design simplicity:
1) Uses Memory rather than ByteBuffer for its backing storage.
2) Uses faster hashing and comparison routines (see HashTableUtils).
3) Capacity is always a power of two, allowing simpler design and more
efficient implementation of findBucket.
4) Does not implement growability; instead, leaves that to its callers.
The idea is this removes the need for subclasses, while still giving
callers flexibility in how to handle table-full scenarios.
* Fix LGTM warnings.
* Adjust dependencies.
* Remove easymock from druid-benchmarks.
* Adjustments from review.
* Fix datasketches unit tests.
* Fix checkstyle.
* 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.
* 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
* Parallel indexing single dim partitions
Implements single dimension range partitioning for native parallel batch
indexing as described in #8769. This initial version requires the
druid-datasketches extension to be loaded.
The algorithm has 5 phases that are orchestrated by the supervisor in
`ParallelIndexSupervisorTask#runRangePartitionMultiPhaseParallel()`.
These phases and the main classes involved are described below:
1) In parallel, determine the distribution of dimension values for each
input source split.
`PartialDimensionDistributionTask` uses `StringSketch` to generate
the approximate distribution of dimension values for each input
source split. If the rows are ungrouped,
`PartialDimensionDistributionTask.UngroupedRowDimensionValueFilter`
uses a Bloom filter to skip rows that would be grouped. The final
distribution is sent back to the supervisor via
`DimensionDistributionReport`.
2) The range partitions are determined.
In `ParallelIndexSupervisorTask#determineAllRangePartitions()`, the
supervisor uses `StringSketchMerger` to merge the individual
`StringSketch`es created in the preceding phase. The merged sketch is
then used to create the range partitions.
3) In parallel, generate partial range-partitioned segments.
`PartialRangeSegmentGenerateTask` uses the range partitions
determined in the preceding phase and
`RangePartitionCachingLocalSegmentAllocator` to generate
`SingleDimensionShardSpec`s. The partition information is sent back
to the supervisor via `GeneratedGenericPartitionsReport`.
4) The partial range segments are grouped.
In `ParallelIndexSupervisorTask#groupGenericPartitionLocationsPerPartition()`,
the supervisor creates the `PartialGenericSegmentMergeIOConfig`s
necessary for the next phase.
5) In parallel, merge partial range-partitioned segments.
`PartialGenericSegmentMergeTask` uses `GenericPartitionLocation` to
retrieve the partial range-partitioned segments generated earlier and
then merges and publishes them.
* Fix dependencies & forbidden apis
* Fixes for integration test
* Address review comments
* Fix docs, strict compile, sketch check, rollup check
* Fix first shard spec, partition serde, single subtask
* Fix first partition check in test
* Misc rewording/refactoring to address code review
* Fix doc link
* Split batch index integration test
* Do not run parallel-batch-index twice
* Adjust last partition
* Split ITParallelIndexTest to reduce runtime
* Rename test class
* Allow null values in range partitions
* Indicate which phase failed
* Improve asserts in tests
* IndexerSQLMetadataStorageCoordinator.getTimelineForIntervalsWithHandle() don't fetch abutting intervals; simplify getUsedSegmentsForIntervals()
* Add VersionedIntervalTimeline.findNonOvershadowedObjectsInInterval() method; Propagate the decision about whether only visible segmetns or visible and overshadowed segments should be returned from IndexerMetadataStorageCoordinator's methods to the user logic; Rename SegmentListUsedAction to RetrieveUsedSegmentsAction, SegmetnListUnusedAction to RetrieveUnusedSegmentsAction, and UsedSegmentLister to UsedSegmentsRetriever
* Fix tests
* More fixes
* Add javadoc notes about returning Collection instead of Set. Add JacksonUtils.readValue() to reduce boilerplate code
* Fix KinesisIndexTaskTest, factor out common parts from KinesisIndexTaskTest and KafkaIndexTaskTest into SeekableStreamIndexTaskTestBase
* More test fixes
* More test fixes
* Add a comment to VersionedIntervalTimelineTestBase
* Fix tests
* Set DataSegment.size(0) in more tests
* Specify DataSegment.size(0) in more places in tests
* Fix more tests
* Fix DruidSchemaTest
* Set DataSegment's size in more tests and benchmarks
* Fix HdfsDataSegmentPusherTest
* Doc changes addressing comments
* Extended doc for visibility
* Typo
* Typo 2
* Address comment
* remove select query
* thanks teamcity
* oops
* oops
* add back a SelectQuery class that throws RuntimeExceptions linking to docs
* adjust text
* update docs per review
* deprecated
* Fix dependency analyze warnings
Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports and
updated druid-forbidden-apis to prevent regressions.
* Address review comments
* Adjust scope for org.glassfish.jaxb:jaxb-runtime
* Fix dependencies for hdfs-storage
* Consolidate netty4 versions