This PR adds a new property druid.router.sql.enable which allows the
Router to handle SQL queries when set to true.
This change does not affect Avatica JDBC requests and they are still routed
by hashing the Connection ID.
To allow parsing of the request object as a SqlQuery (contained in module druid-sql),
some classes have been moved from druid-server to druid-services with
the same package name.
This change allows the selection of a specific broker service (or broker tier) by the Router.
The newly added ManualTieredBrokerSelectorStrategy works as follows:
Check for the parameter brokerService in the query context. If this is a valid broker service, use it.
Check if the field defaultManualBrokerService has been set in the strategy. If this is a valid broker service, use it.
Move on to the next strategy
* Add a new metric query/segments/count that is not emitted by default
* docs
* test the default implementation of the metric
* fix spelling error in docs
* document the fact that query retries will result in additional metric emissions
* update using recommended text from @jihoonson
This PR refactors the code related to segment loading specifically SegmentLoader and SegmentLoaderLocalCacheManager. SegmentLoader is marked UnstableAPI which means, it can be extended outside core druid in custom extensions. Here is a summary of changes
SegmentLoader returns an instance of ReferenceCountingSegment instead of Segment. Earlier, SegmentManager was wrapping Segment objects inside ReferenceCountingSegment. That is now moved to SegmentLoader. With this, a custom implementation can track the references of segments. It also allows them to create custom ReferenceCountingSegment implementations. For this reason, the constructor visibility in ReferenceCountingSegment is changed from private to protected.
SegmentCacheManager has two additional methods called - reserve(DataSegment) and release(DataSegment). These methods let the caller reserve or release space without calling SegmentLoader#getSegment. We already had similar methods in StorageLocation and now they are available in SegmentCacheManager too which wraps multiple locations.
Refactoring to simplify the code in SegmentCacheManager wherever possible. There is no change in the functionality.
* improve groupBy query granularity translation when issued from sql layer
* fix style
* use virtual column to determine timestampResult granularity
* dont' apply postaggregators on compute nodes
* relocate constants
* fix order by correctness issue
* fix ut
* use more easier understanding code in DefaultLimitSpec
* address comment
* rollback use virtual column to determine timestampResult granularity
* fix style
* fix style
* address the comment
* add more detail document to explain the tradeoff
* address the comment
* address the comment
* add single input string expression dimension vector selector and better expression planning
* better
* fixes
* oops
* rework how vector processor factories choose string processors, fix to be less aggressive about vectorizing
* oops
* javadocs, renaming
* more javadocs
* benchmarks
* use string expression vector processor with vector size 1 instead of expr.eval
* better logging
* javadocs, surprising number of the the
* more
* simplify
This PR refactors the code for QueryRunnerFactory#mergeRunners to accept a new interface called QueryProcessingPool instead of ExecutorService for concurrent execution of query runners. This interface will let custom extensions inject their own implementation for deciding which query-runner to prioritize first. The default implementation is the same as today that takes the priority of query into account. QueryProcessingPool can also be used as a regular executor service. It has a dedicated method for accepting query execution work so implementations can differentiate between regular async tasks and query execution tasks. This dedicated method also passes the QueryRunner object as part of the task information. This hook will let custom extensions carry any state from QuerySegmentWalker to QueryProcessingPool#mergeRunners which is not possible currently.
* upgrade error-prone to 2.7.1 and support checks with Java 11+
- upgrade error-prone to 2.7.1
- support running error-prone with Java 11 and above using -Xplugin
instead of custom compiler
- add compiler arguments to ignore warnings/errors in Java 15/16
- introduce strictCompile property to enable strict profiles since we
now need multiple strict profiles for Java 8
- properly exclude all generated source files from error-prone
- fix druid-processing overriding annotation processors from parent pom
- fix druid-core disabling most non-default checks
- align plugin and annotation errorprone versions
- fix / suppress additional issues found by error-prone:
* fix bug in SeekableStreamSupervisor initializing ArrayList size with
the taskGroupdId
* fix missing @Override annotations
- remove outdated compiler plugin in benchmarks
- remove deleted ParameterPackage error-prone rule
- re-enable checks on benchmark module as well
* fix IntelliJ inspections
* disable LongFloatConversion due to bug in error-prone with JDK 8
* add comment about InsecureCrypto
* enrich expression cache key information to support expressions which depend on external state such as lookups
* cache rules everything around me
* low carb
* rename
* Fix is null selector returning incorrect value for Long data type
* Fix style errors
* Refactor getObject method to also cache null column values
* Make lastInput variable nullable
* Refactor unit test
* Use new boolean lastInputIsNull instead of Long for lastInput to avoid boxing
* Refactor to remove Long for input variable
* Make a separate null caching variable
* Cleaner null caching implementation
* 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
* Fix vectorized cardinality bug on certain string columns.
Fixes a bug introduced in #11182, related to the fact that in some cases,
ColumnProcessors.makeVectorProcessor will call "makeObjectProcessor"
instead of "makeSingleValueDimensionProcessor" or
"makeMultiValueDimensionProcessor". CardinalityVectorProcessorFactory
improperly ignored calls to "makeObjectProcessor".
In addition to fixing the bug, I added this detail to the javadocs for
VectorColumnProcessorFactory, to prevent others from running into the
same thing in the future. They do not currently call out this case.
* Improve test coverage.
* Additional fixes.
* ARRAY_AGG sql aggregator function
* add javadoc
* spelling
* review stuff, return null instead of empty when nil input
* review stuff
* Update sql.md
* use type inference for finalize, refactor some things
* Vectorize the cardinality aggregator.
Does not include a byRow implementation, so if byRow is true then
the aggregator still goes through the non-vectorized path.
Testing strategy:
- New tests that exercise both styles of "aggregate" for supported types.
- Some existing tests have also become active (note the deleted
"cannotVectorize" lines).
* Adjust whitespace.
* 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.
* Add a way to retrieve UTF-8 bytes directly via DimensionDictionarySelector.
The idea is that certain operations (like count distinct on strings) will
be faster if they are able to run directly on UTF-8 bytes instead of on
Java Strings decoded by "lookupName".
* Add license header.
* Updates suggested by robots.
Lexicographic ordering of UTF-8 byte sequences and in-memory UTF-16
strings are equivalent. So, we can skip the (expensive) conversion and
get an equivalent result. Thank you, Unicode!
The prior code did not include separation between values, and encoded
null ambiguously. This patch fixes both of those issues by encoding
strings as length + value instead of just value.
I think cache key computation was OK prior to #9800. Prior to that
patch, the cache key was computed using CacheKeyBuilder.appendStrings,
which encodes strings as UTF-8 and inserts a separator byte (0xff)
between them that cannot appear in a UTF-8 stream.
* InDimFilter: Fix NPE involving certain Set types.
Normally, InDimFilters that come from JSON have HashSets for "values".
However, programmatically-generated filters (like the ones from #11068)
may use other set types. Some set types, like TreeSets with natural
ordering, will throw NPE on "contains(null)", which causes the
InDimFilter's ValueMatcher to throw NPE if it encounters a null value.
This patch adds code to detect if the values set can support
contains(null), and if not, wrap that in a null-checking lambda.
Also included:
- Remove unneeded NullHandling.needsEmptyToNull method.
- Update IndexedTableJoinable to generate a TreeSet that does not
require lambda-wrapping. (This particular TreeSet is how I noticed
the bug in the first place.)
* Test fixes.
* Improve test coverage
* add experimental expression aggregator
* add test
* fix lgtm
* fix test
* adjust test
* use not null constant
* array_set_concat docs
* add equals and hashcode and tostring
* fix it
* spelling
* do multi-value magic for expression agg, more javadocs, tests
* formatting
* fix inspection
* more better
* nullable
* Enable rewriting certain inner joins as filters.
The main logic for doing the rewrite is in JoinableFactoryWrapper's
segmentMapFn method. The requirements are:
- It must be an inner equi-join.
- The right-hand columns referenced by the condition must not contain any
duplicate values. (If they did, the inner join would not be guaranteed
to return at most one row for each left-hand-side row.)
- No columns from the right-hand side can be used by anything other than
the join condition itself.
HashJoinSegmentStorageAdapter is also modified to pass through to
the base adapter (even allowing vectorization!) in the case where 100%
of join clauses could be rewritten as filters.
In support of this goal:
- Add Query getRequiredColumns() method to help us figure out whether
the right-hand side of a join datasource is being used or not.
- Add JoinConditionAnalysis getRequiredColumns() method to help us
figure out if the right-hand side of a join is being used by later
join clauses acting on the same base.
- Add Joinable getNonNullColumnValuesIfAllUnique method to enable
retrieving the set of values that will form the "in" filter.
- Add LookupExtractor canGetKeySet() and keySet() methods to support
LookupJoinable in its efforts to implement the new Joinable method.
- Add "enableRewriteJoinToFilter" feature flag to
JoinFilterRewriteConfig. The default is disabled.
* Test improvements.
* Test fixes.
* Avoid slow size() call.
* Remove invalid test.
* Fix style.
* Fix mistaken default.
* Small fixes.
* Fix logic error.
PR #10936 renamed BitmapBenchmark, the parent of a couple of bitmap tests, to
BitmapOperationTest. This patch renames it to BitmapOperationTestBase so JUnit
doesn't pick it up as a test case. When JUnit picks it up, it becomes a flaky
test, since its behavior and correctness depends on whether it runs before
or after its subclasses.
Regression introduced in #11004 due to overzealous optimization. Even though
we replaced stateful usage of ByteBuffer with stateless usage of Memory, we
still need to create a new object on "duplicate" due to semantics of setBuffer.
* fix nested groupby got empty result when using virtual column
* move to query.getVirtualColumns().wrap instead of new VirtualizedColumnSelectorFactory
* move test to GroupByQueryRunnerTest
* Update processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java
Co-authored-by: huagnhui.bigrey <huanghui.bigrey@bytedance.com>
Co-authored-by: Jihoon Son <jihoonson@apache.org>
* Vectorize LongDeserializers.
Also, add many more tests.
* more faster
* more more faster
* more cleanup
* fixes
* forbidden
* benchmark style
* idk why
* adjust
* add preconditions for value >= 0 for writers
* add 64 bit exception
Co-authored-by: Gian Merlino <gian@imply.io>
* DruidInputSource: Fix issues in column projection, timestamp handling.
DruidInputSource, DruidSegmentReader changes:
1) Remove "dimensions" and "metrics". They are not necessary, because we
can compute which columns we need to read based on what is going to
be used by the timestamp, transform, dimensions, and metrics.
2) Start using ColumnsFilter (see below) to decide which columns we need
to read.
3) Actually respect the "timestampSpec". Previously, it was ignored, and
the timestamp of the returned InputRows was set to the `__time` column
of the input datasource.
(1) and (2) together fix a bug in which the DruidInputSource would not
properly read columns that are used as inputs to a transformSpec.
(3) fixes a bug where the timestampSpec would be ignored if you attempted
to set the column to something other than `__time`.
(1) and (3) are breaking changes.
Web console changes:
1) Remove "Dimensions" and "Metrics" from the Druid input source.
2) Set timestampSpec to `{"column": "__time", "format": "millis"}` for
compatibility with the new behavior.
Other changes:
1) Add ColumnsFilter, a new class that allows input readers to determine
which columns they need to read. Currently, it's only used by the
DruidInputSource, but it could be used by other columnar input sources
in the future.
2) Add a ColumnsFilter to InputRowSchema.
3) Remove the metric names from InputRowSchema (they were unused).
4) Add InputRowSchemas.fromDataSchema method that computes the proper
ColumnsFilter for given timestamp, dimensions, transform, and metrics.
5) Add "getRequiredColumns" method to TransformSpec to support the above.
* Various fixups.
* Uncomment incorrectly commented lines.
* Move TransformSpecTest to the proper module.
* Add druid.indexer.task.ignoreTimestampSpecForDruidInputSource setting.
* Fix.
* Fix build.
* Checkstyle.
* Misc fixes.
* Fix test.
* Move config.
* Fix imports.
* Fixup.
* Fix ShuffleResourceTest.
* Add import.
* Smarter exclusions.
* Fixes based on tests.
Also, add TIME_COLUMN constant in the web console.
* Adjustments for tests.
* Reorder test data.
* Update docs.
* Update docs to say Druid 0.22.0 instead of 0.21.0.
* Fix test.
* Fix ITAutoCompactionTest.
* Changes from review & from merging.
* expression filter support for vectorized query engines
* remove unused codes
* more tests
* refactor, more tests
* suppress
* more
* more
* more
* oops, i was wrong
* comment
* remove decorate, object dimension selector, more javadocs
* style
* fix SQL issue for group by queries with time filter that gets optimized to false
* short circuit always false in CombineAndSimplifyBounds
* adjust
* javadocs
* add preconditions for and/or filters to ensure they have children
* add comments, remove preconditions
* where filter left first draft
* Revert changes in calcite test
* Refactor a bit
* Fixing the Tests
* Changes
* Adding tests
* Add tests for correlated queries
* Add comment
* Fix typos
* Fix runtime error when IndexedTableJoinMatcher matches long selector to unique string index.
The issue arises when matching against a long selector on the left-hand side to a string
typed Index on the right-hand side, and when that Index also returns true from areKeysUnique.
In this case, IndexedTableJoinMatcher would generate a ConditionMatcher that implements
matchSingleRow by calling findUniqueLong on the Index. This is inappropriate because the Index
is actually string typed. The fix is to check the type of the Index before deciding how to
implement the ConditionMatcher.
The patch adds "testMatchSingleRowToUniqueStringIndex" to IndexedTableJoinMatcherTest, which
explores this case.
* Update tests.
* Granularity: Introduce primitive-typed bucketStart, increment methods.
Saves creation of unnecessary DateTime objects in timestamp_floor and
timestamp_ceil expressions.
* Fix style.
* Amp up the test coverage.
They all use Long.compare, but they don't need to. Changing to
regular comparisons simplifies the code and also removes branches.
(Internally, Long.compare has two branches.)
* 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.
* Retain order of AND, OR filter children.
If we retain the order, it enables short-circuiting. People can put a
more selective filter earlier in the list and lower the chance that
later filters will need to be evaluated.
Short-circuiting was working before #9608, which switched to unordered
sets to solve a different problem. This patch tries to solve that
problem a different way.
This patch moves filter simplification logic from "optimize" to
"toFilter", because that allows the code to be shared with Filters.and
and Filters.or. The simplification has become more complicated and so
it's useful to share it.
This patch also removes code from CalciteCnfHelper that is no longer
necessary because Filters.and and Filters.or are now doing the work.
* Fixes for inspections.
* Fix tests.
* Back to a Set.
* ready to test
* tested on dev cluster
* tested
* code review
* add UTs
* add UTs
* ut passed
* ut passed
* opti imports
* done
* done
* fix checkstyle
* modify uts
* modify logs
* changing the package of SegmentLazyLoadFailCallback.java to org.apache.druid.segment
* merge from master
* modify import orders
* merge from master
* merge from master
* modify logs
* modify docs
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
* OrFilter: Properly handle child matchers that return the original mask.
This happens when a child matcher is literally true (for example,
BooleanVectorValueMatcher). In this case, OrFilter would throw this
exception from its call to removeAll while processing the next filter:
java.lang.IllegalStateException: 'other' must be a different instance from 'this'
Also update the javadocs for VectorValueMatcher to call out that the
returned object may be the same as the input mask.
* Fix style.
No existing code relies on being able to call these methods in this way.
The new tests exhaustively test all vectors up to size 7, and also test
behavior the run-on-self behavior that has been adjusted by this patch.
* Tidy up query error codes
* fix tests
* Restore query exception type in JsonParserIterator
* address review comments; add a comment explaining the ugly switch
* fix test
* cache expression selector results by associating vector expression bindings to underlying vector offset
* better coverage, fix floats
* style
* stupid bot
* stupid me
* more test
* intellij threw me under the bus when it generated those junit methods
* narrow interface instead of passing around offset
* javascript filter result convert to java boolean
* use type convert replace script convert, and add more unit test
Co-authored-by: qinzhen <qinzhen@kuaishou.com>
* Remove redundant IncrementalIndex.Builder
* Parametrize incremental index tests and benchmarks
- Reveal and fix a bug in OffheapIncrementalIndex
* Fix forbiddenapis error: Forbidden method invocation: java.lang.String#format(java.lang.String,java.lang.Object[]) [Uses default locale]
* Fix Intellij errors: declared exception is never thrown
* Add documentation and validate before closing objects on tearDown.
* Add documentation to OffheapIncrementalIndexTestSpec
* Doc corrections and minor changes.
* Add logging for generated rows.
* Refactor new tests/benchmarks.
* Improve IncrementalIndexCreator documentation
* Add required tests for DataGenerator
* Revert "rollupOpportunity" to be a string
* Multiphase merge for IndexMergerV9
* JSON fix
* Cleanup temp files
* Docs
* Address logging and add IT
* Fix spelling and test unloader datasource name
* First draft of grouping_id function
* Add more tests and documentation
* Add calcite tests
* Fix travis failures
* bit of a change
* Add documentation
* Fix typos
* typo fix
* Add context dimension to DefaultQueryMetrics
* remove redundant addition of context dimension from DruidMetrics now that QueryMetrics adds it by default
* update SearchQueryMetrics to reflect the same pattern as other default dimensions in QueryMetrics
* add PublicApi annotation for context in QueryMetrics Interface
* Add new coordinator metrics for duty runtimes
* fix spelling for a constant variable value
* add comment clarifying why the global runtime metric is emitted where it is
* Remove duty alias in lieu of using the class name for metrics
* fix docs
* CoordinatorStats tests + add duty stats to accumulate() logic
* support multi-line text
* add test cases
* split json text into lines case by case
* improve exception handle
* fix CI
* use IntermediateRowParsingReader as base of JsonReader
* update doc
* ignore the non-immutable field in test case
* add more test cases
* mark `lineSplittable` as final
* fix testcases
* fix doc
* add a test case for SqlReader
* return all raw columns when exception occurs
* fix CI
* fix test cases
* resolve review comments
* handle ParseException returned by index.add
* apply Iterables.getOnlyElement
* fix CI
* fix test cases
* improve code in more graceful way
* fix test cases
* fix test cases
* add a test case to check multiple json string in one text block
* fix inspection check
* support for vectorizing expressions with non-existent inputs, more consistent type handling for non-vectorized expressions
* inspector
* changes
* more test
* clean
* Introduce a Configurable Index Type
* Change to @UnstableApi
* Add AppendableIndexSpecTest
* Update doc
* Add spelling exception
* Add tests coverage
* Revert some of the changes to reduce diff
* Minor fixes
* Update getMaxBytesInMemoryOrDefault() comment
* Fix typo, remove redundant interface
* Remove off-heap spec (postponed to a later PR)
* Add javadocs to AppendableIndexSpec
* Describe testCreateTask()
* Add tests for AppendableIndexSpec within TuningConfig
* Modify hashCode() to conform with equals()
* Add comment where building incremental-index
* Add "EqualsVerifier" tests
* Revert some of the API back to AppenderatorConfig
* Don't use multi-line comments
* Remove knob documentation (deferred)
* Proposed changes for making joins cacheable
* Add unit tests
* Fix tests
* simplify logic
* Pull empty byte array logic out of CachingQueryRunner
* remove useless null check
* Minor refactor
* Fix tests
* Fix segment caching on Broker
* Move join cache key computation in Broker
Move join cache key computation in Broker from ResultLevelCachingQueryRunner to CachingClusteredClient
* Fix compilation
* Review comments
* Add more tests
* Fix inspection errors
* Pushed condition analysis to JoinableFactory
* review comments
* Disable join caching for broker and add prefix key to BroadcastSegmentIndexedTable
* Remove commented lines
* Fix populateCache
* Disable caching for selective datasources
Refactored the code so that we can decide at the data source level, whether to enable cache for broker or data nodes
* RowBasedIndexedTable: Add specialized index types for long keys.
Two new index types are added:
1) Use an int-array-based index in cases where the difference between
the min and max values isn't too large, and keys are unique.
2) Use a Long2ObjectOpenHashMap (instead of the prior Java HashMap) in
all other cases.
In addition:
1) RowBasedIndexBuilder, a new class, is responsible for picking which
index implementation to use.
2) The IndexedTable.Index interface is extended to support using
unboxed primitives in the unique-long-keys case, and callers are
updated to use the new functionality.
Other key types continue to use indexes backed by Java HashMaps.
* Fixup logic.
* Add tests.
* vectorize remaining math expressions
* fixes
* remove cannotVectorize() where no longer true
* disable vectorized groupby for numeric columns with nulls
* fixes
* Store hash partition function in dataSegment and allow segment pruning only when hash partition function is provided
* query context
* fix tests; add more test
* javadoc
* docs and more tests
* remove default and hadoop tests
* consistent name and fix javadoc
* spelling and field name
* default function for partitionsSpec
* other comments
* address comments
* fix tests and spelling
* test
* doc
* Include Sequence-building time in CPU time metric.
Meaningful work can be done while building Sequences, and we should
count this work. On the Broker, this includes subquery processing
work done by the mergeResults call of the GroupByQueryQueryToolChest.
* Add test.
* push down ValueType to ExprType conversion, tidy up
* determine expr output type for given input types
* revert unintended name change
* add nullable
* tidy up
* fixup
* more better
* fix signatures
* naming things is hard
* fix inspection
* javadoc
* make default implementation of Expr.getOutputType that returns null
* rename method
* more test
* add output for contains expr macro, split operation and function auto conversion
* 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>
* Fix handling of 'join' on top of 'union' datasources.
The problem is that unions are typically rewritten into a series of
individual queries on the underlying tables, but this isn't done when
the union is wrapped in a join.
The main changes are in UnionQueryRunner:
1) Replace an instanceof UnionQueryRunner check with DataSourceAnalysis.
2) Replace a "query.withDataSource" call with a new function, "Queries.withBaseDataSource".
Together, these enable UnionQueryRunner to "see through" a join.
* Tests.
* Adjust heap sizes for integration tests.
* Different approach, more tests.
* Tweak.
* Styling.
* Add support for all partitioing schemes for auto compaction
* annotate last compaction state for multi phase parallel indexing
* fix build and tests
* test
* better home
* 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
* Make NUMERIC_HASHING_THRESHOLD configurable
Change the default numeric hashing threshold to 1 and make it configurable.
Benchmarks attached to this PR show that binary searches are not more faster
than doing a set contains check. The attached flamegraph shows the amount of
time a query spent in the binary search. Given the benchmarks, we can expect
to see roughly a 2x speed up in this part of the query which works out to
~ a 10% faster query in this instance.
* Remove NUMERIC_HASHING_THRESHOLD
* Remove stale docs
Previously, this was disallowed, because expressions treated multi-values
as nulls. But now, if there's a single multi-value column that can be
mapped over, it's okay to use the index. Expression selectors already do
this.
* Optimize large InDimFilters
For large InDimFilters, in default mode, the filter does a linear check of the
set to see if it contains either an empty or null. If it does, the empties are
converted to nulls by passing through the entire list again.
Instead of this, in default mode, we attempt to remove an empty string from the
values that are passed to the InDimFilter. If an empty string was removed, we
add null to the set
* code review
* Revert "code review"
This reverts commit 61fe33ebf7.
* code review - less brittle
* 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
* 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.
1) lookupId could return IDs beyond maxId if called with a recently added value.
2) getRow could return an ID for null beyond maxId, if null was recently
encountered in a dimension that initially didn't appear at all. (In this case,
the dictionary ID for null can be > 0).
Also add a comment explaining how this stuff is supposed to work.
* fix bug with realtime expressions on sparse string columns
* fix test
* add comment back
* push capabilities for dimensions to dimension indexers since they know things
* style
* style
* fixes
* getting a bit carried away
* missed one
* fix it
* benchmark build fix
* review stuffs
* javadoc and comments
* add comment
* more strict check
* fix missed usaged of impl instead of interface
* 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.
* Combine InDimFilter, InFilter.
There are two motivations:
1. Ensure that when HashJoinSegmentStorageAdapter compares its Filter
to the original one, and it is an "in" type, the comparison is by
reference and does not need to check deep equality. This is useful
when the "in" filter is very large.
2. Simplify things. (There isn't a great reason for the DimFilter and
Filter logic to be separate, and combining them reduces some
duplication.)
* Fix test.
* 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.
* support unit suffix on byte-related properties
* add doc
* change default value of byte-related properites in example files
* fix coding style
* fix doc
* fix CI
* suppress spelling errors
* improve code according to comments
* rename Bytes to HumanReadableBytes
* add getBytesInInt to get value safely
* improve doc
* fix problem reported by CI
* fix problem reported by CI
* resolve code review comments
* improve error message
* improve code & doc according to comments
* fix CI problem
* improve doc
* suppress spelling check errors
* 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
* 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
* Fix RetryQueryRunner to actually do the job
* more javadoc
* fix test and checkstyle
* don't combine for testing
* address comments
* fix unit tests
* always initialize response context in cachingClusteredClient
* fix subquery
* address comments
* fix test
* query id for builders
* make queryId optional in the builders and ClusterQueryResult
* fix test
* suppress tests and unused methods
* exclude groupBy builder
* fix jacoco exclusion
* add tests for builders
* address comments
* don't truncate
* Join filter pre-analysis simplifications and sanity checks.
- At pre-analysis time, only compute pre-analysis for the innermost
root query, since this is the one that will run on the join that involves
the base datasource. Previously, pre-analyses were computed for multiple
levels of the query, some of which were unnecessary.
- Remove JoinFilterPreAnalysisGroup and join query level gathering code,
since they existed to support precomputation of multiple pre-analyses.
- Embed JoinFilterPreAnalysisKey into JoinFilterPreAnalysis and use it to
sanity check at processing time that the correct pre-analysis was done.
Tangentially related changes:
- Remove prioritizeAndLaneQuery functionality from LocalQuerySegmentWalker.
The computed priority and lanes were not being used.
- Add "getBaseQuery" method to DataSourceAnalysis to support identification
of the proper subquery for filter pre-analysis.
* Fix compilation errors.
* Adjust tests.
* Filter http requests by http method
Add a config that allows a user which http methods to allow against their
Druid server.
Druid will only accept http requests with the method: GET, PUT, POST, DELETE
and OPTIONS.
If a Druid admin wants to allow other methods, they can do so by using the
ServerConfig#allowedHttpMethods config.
If a Druid user would like to disallow OPTIONS, this can be done by changing
the AuthConfig#allowUnauthenticatedHttpOptions config
* Exclude OPTIONS from always supported HTTP methods
Add HEAD as an allowed method for web console e2e tests
* fix docs
* fix security IT
* Actually fix the web console e2e tests
* Ignore icode coverage for nitialization classes
* code review