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