* IncrementalIndex#add is no longer thread-safe.
Following #14866, there is no longer a reason for IncrementalIndex#add
to be thread-safe.
It turns out it already was not using its selectors in a thread-safe way,
as exposed by #15615 making `testMultithreadAddFactsUsingExpressionAndJavaScript`
in `IncrementalIndexIngestionTest` flaky. Note that this problem isn't
new: Strings have been stored in the dimension selectors for some time,
but we didn't have a test that checked for that case; we only have
this test that checks for concurrent adds involving numeric selectors.
At any rate, this patch changes OnheapIncrementalIndex to no longer try
to offer a thread-safe "add" method. It also improves performance a bit
by adding a row ID supplier to the selectors it uses to read InputRows,
meaning that it can get the benefit of caching values inside the selectors.
This patch also:
1) Adds synchronization to HyperUniquesAggregator and CardinalityAggregator,
which the similar datasketches versions already have. This is done to
help them adhere to the contract of Aggregator: concurrent calls to
"aggregate" and "get" must be thread-safe.
2) Updates OnHeapIncrementalIndexBenchmark to use JMH and moves it to the
druid-benchmarks module.
* Spelling.
* Changes from static analysis.
* Fix javadoc.
* Clear "lineSplittable" for JSON when using KafkaInputFormat.
JsonInputFormat has a "withLineSplittable" method that can be used to
control whether JSON is read line-by-line, or as a whole. The intent
is that in streaming ingestion, "lineSplittable" is false (although it
can be overridden by "assumeNewlineDelimited"), and in batch ingestion,
lineSplittable is true.
When a "json" format is wrapped by a "kafka" format, this isn't set
properly. This patch updates KafkaInputFormat to set this on an
underlying "json" format.
The tests for KafkaInputFormat were overriding the "lineSplittable"
parameter explicitly, which wasn't really fair, because that made them
unrealistic to what happens in production. Now they omit the parameter
and get the production behavior.
* Add test.
* Fix test coverage.
* Faster parsing: reduce String usage, list-based input rows.
Three changes:
1) Reworked FastLineIterator to optionally avoid generating Strings
entirely, and reduce copying somewhat. Benefits the line-oriented
JSON, CSV, delimited (TSV), and regex formats.
2) In the delimited (TSV) format, when the delimiter is a single byte,
split on UTF-8 bytes directly.
3) In CSV and delimited (TSV) formats, use list-based input rows when
the column list is provided upfront by the user.
* Fix style.
* Fix inspections.
* Restore validation.
* Remove fastutil-extra.
* Exception type.
* Fixes for error messages.
* Fixes for null handling.
This PR fixes the summary iterator to add aggregators in the correct position. The summary iterator is used when dims are not present, therefore the new change is identical to the old one, but seems more correct while reading.
* support groups windowing mode; which is a close relative of ranges (but not in the standard)
* all windows with range expressions will be executed wit it groups
* it will be 100% correct in case for both bounds its true that: isCurrentRow() || isUnBounded()
* this covers OVER ( ORDER BY COL )
* for other cases it will have some chances of getting correct results...
* Cache value selectors in RowBasedColumnSelectorFactory.
There was already caching for dimension selectors. This patch adds caching
for value (object and number) selectors. It's helpful when the same field is
read multiple times during processing of a single row (for example, by being
an input to both MIN and MAX aggregations).
* Fix typing.
* Fix logic.
* Add SpectatorHistogram extension
* Clarify documentation
Cleanup comments
* Use ColumnValueSelector directly
so that we support being queried as a Number using longSum or doubleSum aggregators as well as a histogram.
When queried as a Number, we're returning the count of entries in the histogram.
* Apply suggestions from code review
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
* Fix references
* Fix spelling
* Update docs/development/extensions-contrib/spectator-histogram.md
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
---------
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
* Add ImmutableLookupMap for static lookups.
This patch adds a new ImmutableLookupMap, which comes with an
ImmutableLookupExtractor. It uses a fastutil open hashmap plus two
lists to store its data in such a way that forward and reverse
lookups can both be done quickly. I also observed footprint to be
somewhat smaller than Java HashMap + MapLookupExtractor for a 1 million
row lookup.
The main advantage, though, is that reverse lookups can be done much
more quickly than MapLookupExtractor (which iterates the entire map
for each call to unapplyAll). This speeds up the recently added
ReverseLookupRule (#15626) during SQL planning with very large lookups.
* Use in one more test.
* Fix benchmark.
* Object2ObjectOpenHashMap
* Fixes, and LookupExtractor interface update to have asMap.
* Remove commented-out code.
* Fix style.
* Fix import order.
* Add fastutil.
* Avoid storing Map entries.
* Reverse, pull up lookups in the SQL planner.
Adds two new rules:
1) ReverseLookupRule, which eliminates calls to LOOKUP by doing
reverse lookups.
2) AggregatePullUpLookupRule, which pulls up calls to LOOKUP above
GROUP BY, when the lookup is injective.
Adds configs `sqlReverseLookup` and `sqlPullUpLookup` to control whether
these rules fire. Both are enabled by default.
To minimize the chance of performance problems due to many keys mapping to
the same value, ReverseLookupRule refrains from reversing a lookup if there
are more keys than `inSubQueryThreshold`. The rationale for using this setting
is that reversal works by generating an IN, and the `inSubQueryThreshold`
describes the largest IN the user wants the planner to create.
* Add additional line.
* Style.
* Remove commented-out lines.
* Fix tests.
* Add test.
* Fix doc link.
* Fix docs.
* Add one more test.
* Fix tests.
* Logic, test updates.
* - Make FilterDecomposeConcatRule more flexible.
- Make CalciteRulesManager apply reduction rules til fixpoint.
* Additional tests, simplify code.
* Faster k-way merging using tournament trees, 8-byte key strides.
Two speedups for FrameChannelMerger (which does k-way merging in MSQ):
1) Replace the priority queue with a tournament tree, which does fewer
comparisons.
2) Compare keys using 8-byte strides, rather than 1 byte at a time.
* Adjust comments.
* Fix style.
* Adjust benchmark and test.
* Add eight-list test (power of two).
This PR fixes a bug with the long string pair serde where null and empty strings are treated equivalently, and the return value is always null. When 'useDefaultValueForNull' was set to true by default, this wasn't a commonly seen issue, because nulls were equivalent to empty strings. However, since the default has changed to false, this can create incorrect results when the long string pairs are serded, where the empty strings are incorrectly converted to nulls.
changes:
* ColumnIndexSelector now extends ColumnSelector. The only real implementation of ColumnIndexSelector, ColumnSelectorColumnIndexSelector, already has a ColumnSelector, so this isn't very disruptive
* removed getColumnNames from ColumnSelector since it was not used
* VirtualColumns and VirtualColumn getIndexSupplier method now needs argument of ColumnIndexSelector instead of ColumnSelector, which allows expression virtual columns to correctly recognize other virtual columns, fixing an issue which would incorrectly handle other virtual columns as non-existent columns instead
* fixed a bug with sql planner incorrectly not using expression filter for equality filters on columns with extractionFn and no virtual column registry
* overhaul DruidPredicateFactory to better handle 3VL
fixes some bugs caused by some limitations of the original design of how DruidPredicateFactory interacts with 3-value logic. The primary impacted area was with how filters on values transformed with expressions or extractionFn which turn non-null values into nulls, which were not possible to be modelled with the 'isNullInputUnknown' method
changes:
* adds DruidObjectPredicate to specialize string, array, and object based predicates instead of using guava Predicate
* DruidPredicateFactory now uses DruidObjectPredicate
* introduces DruidPredicateMatch enum, which all predicates returned from DruidPredicateFactory now use instead of booleans to indicate match. This means DruidLongPredicate, DruidFloatPredicate, DruidDoublePredicate, and the newly added DruidObjectPredicate apply methods all now return DruidPredicateMatch. This allows matchers and indexes
* isNullInputUnknown has been removed from DruidPredicateFactory
* rename, fix test
* adjust
* style
* npe
* more test
* fix default value mode to not match new test
* Reverse lookup fixes and enhancements.
1) Add a "mayIncludeUnknown" parameter to DimFilter#optimize. This is important
because otherwise the reverse-lookup optimization is done improperly when
the "in" filter appears under a "not", and the lookup extractionFn may return
null for some possible values of the filtered column. The "includeUnknown" test
cases in InDimFilterTest illustrate the difference in behavior.
2) Enhance InDimFilter#optimizeLookup to handle "mayIncludeUnknown", and to be able
to do a reverse lookup in a wider variety of cases.
3) Make "unapply" protected in LookupExtractor, and move callers to "unapplyAll".
The main reason is that MapLookupExtractor, a common implementation, lacks a
reverse mapping and therefore does a scan of the map for each call to "unapply".
For performance sake these calls need to be batched.
* Remove optimize call from BloomDimFilter.
* Follow the law.
* Fix tests.
* Fix imports.
* Switch function.
* Fix tests.
* More tests.
* New handling for COALESCE, SEARCH, and filter optimization.
COALESCE is converted by Calcite's parser to CASE, which is largely
counterproductive for us, because it ends up duplicating expressions.
In the current code we end up un-doing it in our CaseOperatorConversion.
This patch has a different approach:
1) Add CaseToCoalesceRule to convert CASE back to COALESCE earlier, before
the Volcano planner runs, using CaseToCoalesceRule.
2) Add FilterDecomposeCoalesceRule to decompose calls like
"f(COALESCE(x, y))" into "(x IS NOT NULL AND f(x)) OR (x IS NULL AND f(y))".
This helps use indexes when available on x and y.
3) Add CoalesceLookupRule to push COALESCE into the third arg of LOOKUP.
4) Add a native "coalesce" function so we can convert 3+ arg COALESCE.
The advantage of this approach is that by un-doing the CASE to COALESCE
conversion earlier, we have flexibility to do more stuff with
COALESCE (like decomposition and pushing into LOOKUP).
SEARCH is an operator used internally by Calcite to represent matching
an argument against some set of ranges. This patch improves our handling
of SEARCH in two ways:
1) Expand NOT points (point "holes" in the range set) from SEARCH as
`!(a || b)` rather than `!a && !b`, which makes it possible to convert
them to a "not" of "in" filter later.
2) Generate those nice conversions for NOT points even if the SEARCH
is not composed of 100% NOT points. Without this change, a SEARCH
for "x NOT IN ('a', 'b') AND x < 'm'" would get converted like
"x < 'a' OR (x > 'a' AND x < 'b') OR (x > 'b' AND x < 'm')".
One of the steps we take when generating Druid queries from Calcite
plans is to optimize native filters. This patch improves this step:
1) Extract common ANDed predicates in ConvertSelectorsToIns, so we can
convert "(a && x = 'b') || (a && x = 'c')" into "a && x IN ('b', 'c')".
2) Speed up CombineAndSimplifyBounds and ConvertSelectorsToIns on
ORs with lots of children by adjusting the logic to avoid calling
"indexOf" and "remove" on an ArrayList.
3) Refactor ConvertSelectorsToIns to reduce duplicated code between the
handling for "selector" and "equals" filters.
* Not so final.
* Fixes.
* Fix test.
* Fix test.
* Fix ColumnSelectorColumnIndexSelector#getColumnCapabilities.
It was using virtualColumns.getColumnCapabilities, which only returns
capabilities for virtual columns, not regular columns. The effect of this
is that expression filters (and in some cases, arrayContainsElement filters)
would build value matchers rather than use indexes.
I think this has been like this since #12315, which added the
getColumnCapabilities method to BitmapIndexSelector, and included the same
implementation as exists in the code today.
This error is easy to make due to the design of virtualColumns.getColumnCapabilities,
so to help avoid it in the future, this patch renames the method to
getColumnCapabilitiesWithoutFallback to emphasize that it does not return
capabilities for regular columns.
* Make getColumnCapabilitiesWithoutFallback package-private.
* Fix expression filter bitmap usage.
The PR: #13947 introduced a function evalDimension() in the interface RowFunction.
There was no default implementation added for this interface which causes all the implementations and custom transforms to fail and require to implement their own version of evalDimension method. This PR adds a default implementation in the interface which allows the evalDimension to return value as a Singleton array of eval result.
Fixes#15072
Before this modification , the third parameter (timezone) require to be a Literal, it will throw a error when this parameter is column Identifier.
Changes
- Add `log` implementation for `AuditManager` alongwith `SQLAuditManager`
- `LoggingAuditManager` simply logs the audit event. Thus, it returns empty for
all `fetchAuditHistory` calls.
- Add new config `druid.audit.manager.type` which can take values `log`, `sql` (default)
- Add new config `druid.audit.manager.logLevel` which can take values `DEBUG`, `INFO`, `WARN`.
This gets activated only if `type` is `log`.
- Remove usage of `ConfigSerde` from `AuditManager` as audit is not just limited to configs
- Add `AuditSerdeHelper` for a single implementation of serialization/deserialization of
audit payload and other utility methods.
* Allow for kafka emitter producer secrets to be masked in logs instead of being visible
This change will allow for kafka producer config values that should be secrets to not show up in the logs.
This will enhance the security of the people who use the kafka emitter to use this if they want to.
This is opt in and will not affect prior configs for this emitter
* fix checkstyle issue
* change property name
I was looking into a query which was performing a bit poorly because the case_searched was touching more than 1 columns (if there is only 1 column there is a cache based evaluator).
While I was doing that I've noticed that there are a few simple things which could help a bit:
use a static TRUE/FALSE instead of creating a new object every time
create the ExprEval early for ConstantExpr -s (except the one for BigInteger which seem to have some odd contract)
return early from type autodetection
these changes mostly reduce the amount of garbage the query creates during case_searched evaluation; although ExpressionSelectorBenchmark shows some improvements ~15% - but my manual trials on the taxi dataset with 60M rows showed more improvements - probably due to the fact that these changes mostly only reduce gc pressure.
* Add initial draft of MarkDanglingTombstonesAsUnused duty.
* Use overshadowed segments instead of all used segments.
* Add unit test for MarkDanglingSegmentsAsUnused duty.
* Add mock call
* Simplify code.
* Docs
* shorter lines formatting
* metric doc
* More tests, refactor and fix up some logic.
* update javadocs; other review comments.
* Make numCorePartitions as 0 in the TombstoneShardSpec.
* fix up test
* Add tombstone core partition tests
* Update docs/design/coordinator.md
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
* review comment
* Minor cleanup
* Only consider tombstones with 0 core partitions
* Need to register the test shard type to make jackson happy
* test comments
* checkstyle
* fixup misc typos in comments
* Update logic to use overshadowed segments
* minor cleanup
* Rename duty to eternity tombstone instead of dangling. Add test for full eternity tombstone.
* Address review feedback.
---------
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
Query with lookups in FilteredAggregator fails with this exception in router,
Cannot construct instance of `org.apache.druid.query.aggregation.FilteredAggregatorFactory`, problem: Lookup [campaigns_lookup[campaignId][is_sold][autodsp]] not found at [Source: (org.eclipse.jetty.server.HttpInputOverHTTP); line: 1, column: 913] (through reference chain: org.apache.druid.query.groupby.GroupByQuery["aggregations"]->java.util.ArrayList[1])
T
he problem is that constructor of FilteredAggregatorFactory is actually validating if the lookup exists in this statement dimFilter.toFilter().
This is failing on the router, which is to be expected, because, the router isn’t assigned any lookups.
The fix is to move to a lazy initialisation of the filter object in the constructor.
It wasn't checking the column name, so it would return a domain regardless
of the input column. This means that null filters on data sources with range
partitioning would lead to excessive pruning of segments, and therefore
missing results.
I think this is a problem as it discards the false return value when the putToKeyBuffer can't store the value because of the limit
Not forwarding the return value at that point may lead to the normal continuation here regardless something was not added to the dictionary like here
* Make numCorePartitions as 0 in the TombstoneShardSpec.
* fix up test
* Add tombstone core partition tests
* review comment
* Need to register the test shard type to make jackson happy
Fixed the following flaky tests:
org.apache.druid.math.expr.ParserTest#testApplyFunctions
org.apache.druid.math.expr.ParserTest#testSimpleMultiplicativeOp1
org.apache.druid.math.expr.ParserTest#testFunctions
org.apache.druid.math.expr.ParserTest#testSimpleLogicalOps1
org.apache.druid.math.expr.ParserTest#testSimpleAdditivityOp1
org.apache.druid.math.expr.ParserTest#testSimpleAdditivityOp2
The above mentioned tests have been reported as flaky (tests assuming deterministic implementation of a non-deterministic specification ) when ran against the NonDex tool.
The tests contain assertions (Assertion 1 & Assertion 2) that compare an ArrayList created from a HashSet using the ArrayList() constructor with another List. However, HashSet does not guarantee the ordering of elements and thus resulting in these flaky tests that assume deterministic implementation of HashSet. Thus, when the NonDex tool shuffles the HashSet elements, it results in the test failures:
Co-authored-by: ythorat2 <ythorat2@illinois.edu>
* MSQ generates tombstones honoring the query's granularity.
This change tweaks to only account for the infinite-interval tombstones.
For finite-interval tombstones, the MSQ query granualrity will be used
which is consistent with how MSQ works.
* more tests and some cleanup.
* checkstyle
* comment edits
* Throw TooManyBuckets fault based on review; add more tests.
* Add javadocs for both methods on reconciling the methods.
* review: Move testReplaceTombstonesWithTooManyBucketsThrowsException to MsqFaultsTest
* remove unused imports.
* Move TooManyBucketsException to indexing package for shared exception handling.
* lower max bucket for tests and fixup count
* Advance and count the iterator.
* checkstyle
In the current design, brokers query both data nodes and tasks to fetch the schema of the segments they serve. The table schema is then constructed by combining the schemas of all segments within a datasource. However, this approach leads to a high number of segment metadata queries during broker startup, resulting in slow startup times and various issues outlined in the design proposal.
To address these challenges, we propose centralizing the table schema management process within the coordinator. This change is the first step in that direction. In the new arrangement, the coordinator will take on the responsibility of querying both data nodes and tasks to fetch segment schema and subsequently building the table schema. Brokers will now simply query the Coordinator to fetch table schema. Importantly, brokers will still retain the capability to build table schemas if the need arises, ensuring both flexibility and resilience.
* Use filters for pruning properly for hash-joins.
Native used them too aggressively: it might use filters for the RHS
to prune the LHS. MSQ used them not at all. Now, both use them properly,
pruning based on base (LHS) columns only.
* Fix tests.
* Fix style.
* Clear filterFields too.
* Update.
* Add system fields to input sources.
Main changes:
1) The SystemField enum defines system fields "__file_uri", "__file_path",
and "__file_bucket". They are associated with each input entity.
2) The SystemFieldInputSource interface can be added to any InputSource
to make it system-field-capable. It sets up serialization of a list
of configured "systemFields" in the JSON form of the input source, and
provides a method getSystemFieldValue for computing the value of each
system field. Cloud object, HDFS, HTTP, and Local now have this.
* Fix various LocalInputSource calls.
* Fix style stuff.
* Fixups.
* Fix tests and coverage.
* better documentation for the differences between arrays and mvds
* add outputType to ExpressionPostAggregator to make docs true
* add output coercion if outputType is defined on ExpressionPostAgg
* updated post-aggregations.md to be consistent with aggregations.md and filters.md and use tables
* Frames: consider writing singly-valued column when input column hasMultipleValues is UNKNOWN.
Prior to this patch, columnar frames would always write multi-valued columns if
the input column had hasMultipleValues = UNKNOWN. This had the effect of flipping
UNKNOWN to TRUE when copying data into frames, which is problematic because TRUE
causes expressions to assume that string inputs must be treated as arrays.
We now avoid this by flipping UNKNOWN to FALSE if no multi-valuedness
is encountered, and flipping it to TRUE if multi-valuedness is encountered.
* Add regression test case.
Currently advance function in postJoinCursor calls advanceUninterruptibly which in turn keeps calling baseCursor.advanceUninterruptibly until the post join condition matches, without checking for interrupts. This causes the CPU to hit 100% without getting a chance for query to be cancelled.
With this change, the call flow of advance and advanceUninterruptibly is separated out so that they call baseCursor.advance and baseCursor.advanceUninterruptibly in them, respectively, giving a chance for interrupts in the former case between successive calls to baseCursor.advance.
* Fix error assuming a Complex Type that is a Number is a double
In the case where a complex type is a number, it may not be castable to double. It can safely be case as Number first to get to the doubleValue.
- adds a new query build path: DruidQuery#toScanAndSortQuery which:
- builds a ScanQuery without considering the current ordering
- builds an operator to execute the sort
- fixes a null string to "null" literal string conversion in the frame serializer code
- fixes some DrillWindowQueryTest cases
- fix NPE in NaiveSortOperator in case there was no input
- enables back CoreRules.AGGREGATE_REMOVE
- adds a processing level OffsetLimit class and uses that instead of just the limit in the rac parts
- earlier window expressions on top of a subquery with an offset may have ignored the offset
* provide function name when unknown exceptions are encountered
* fix keywords/etc
* fix keywrod order - regex excercise
* add test
* add check&fix keywords
* decoupledIgnore
* Revert "decoupledIgnore"
This reverts commit e922c820a7.
* unpatch Function
* move to a different location
* checkstyle
for some exotic queries like:
SELECT
'_'||dim1,
MIN(cast(0 as double)) OVER (),
MIN(cast((cnt||cnt) as bigint)) OVER ()
FROM foo
the compilation have resulted in NPE -s mostly because VirtualColumn -s were not handled properly
* add native filters for "(filter) is true" and "(filter) is false"
changes:
* add IsTrueDimFilter, IsFalseDimFilter, and abstract IsBooleanDimFilter for native json filter implementations of `(filter) IS TRUE` and `(filter) IS FALSE`
* add IsBooleanFilter for actual filtering logic for these filters, which ignore includeUnknown to always use matches with false for true and !matches with true for false
* fix test incorrectly adjusted to wrong answer in #15058
* add tests for default value mode
* sql compatible tri-state native logical filters when druid.expressions.useStrictBooleans=true and druid.generic.useDefaultValueForNull=false, and new druid.generic.useThreeValueLogicForNativeFilters=true
* log.warn if non-default configurations are used to guide operators towards SQL complaint behavior
* fixes
* check for latest rewrite place
* Revert "check for latest rewrite place"
This reverts commit 5cf1e2c1ca.
* some stuff
(cherry picked from commit ab346d4373ea888eb8ef6115e018e7fb0d27407f)
* update test output
* updates to test ouptuts
* some stuff
* move validator
* cleanup
* fix
* change test slightly
* add apidoc cleanup warnings
* cleanup/etc
* instead of telling the story; add a fail with some reason whats the issue
* lead-lag fix
* add test
* remove unnecessary throw
* druidexception-trial
* Revert "druidexception-trial"
This reverts commit 8fa06644bc.
* undo changes to no_grouping; add no_grouping2
* add missing assert on resultcount
* rename method; update
* introduce enum/etc
* make resultmatchmode accessible from TestBuilder#expectedResults
* fix dump results to use log
* fix
* handle null correctly
* disable feature type based things for MSQ
* fix varianssqlaggtest
* use eps in other test
* fix intellij error
* add final
* addrss review
* update test/string/etc
* write concat in 3 lines :D
This patch changes the thread name of the processing pool of the indexers/peons/historicals from query.getType() + "_" + query.getDataSource() + "_" + query.getIntervals() to query.getId()
* add a bunch of tests with array typed columns to CalciteArraysQueryTest
* fix a bug with unnest filter pushdown when filtering on unnested array columns
This PR aims to add the capabilities to:
1. Fetch the realtime segment metadata from the coordinator server view,
2. Adds the ability for workers to query indexers, similar to how brokers do the same for native queries.
* Fix IndexerWorkerClient#fetchChannelData when response has data and error.
When a channel data response from a worker includes some data and then
some I/O error, then when the call is retried, we will re-read the set
of data that was read by the previous connection and add it to the
local channel again. This causes the local channel to become corrupted.
The patch fixes this case by skipping data that has already been read.
* Updating plans when using joins with unnest on the left
* Correcting segment map function for hashJoin
* The changes done here are not reflected into MSQ yet so these tests might not run in MSQ
* native tests
* Self joins with unnest data source
* Making this pass
* Addressing comments by adding explanation and new test
Code relying on monomorphic processing on JDK8 doesn't work correctly, since it tries to reference getArrayLength using method handles, which might have been accidentally removed here since it seems unused. This PR adds the method back as is.
Fixes a bug caused by #14919, which was just using the column name as part of a temp file name, which.. isn't very cool, my bad. Switched to use StringUtils.urlEncode so that ugly chars don't explode stuff. The modified test fails without the changes in this PR.
Row-based frames, and by extension, MSQ now supports numeric array types. This means that all queries consuming or producing arrays would also work with MSQ. Numeric arrays can also be ingested via MSQ. Post this patch, queries like, SELECT [1, 2] would work with MSQ since they consume a numeric array, instead of failing with an unsupported column type exception.
When merging analyses, lenient merging sets unmergeable aggregators
to null. Merging such a null aggregator record into a nonnull record
would potentially lead to NPE in getMergingFactory.
The new code only calls getMergingFactory if both the old and new
aggregators are nonnull; else, if either is null, then the merged
aggregator is also set to null.
This patch introduces "processor managers" to processor factories, as a replacement for the sequence of processors. Processor managers can use the results of earlier processors to influence the creation of later processors, which provides us with the building block we need to ensure that broadcast join data is only read once.
In particular, when broadcast join is happening, the BaseFrameProcessorFactory now uses a ChainedProcessorManager to first run BroadcastJoinSegmentMapFnProcessor (in a single thread), and then run all of the regular processors (possibly multithreaded).
When moving timestamps by an offset using org.joda.time.chrono.ISOChronology library, if the new timestamp falls in Daylight Savings Time (DST) transition period, the library rounds it off to the nearest valid time. This can lead to incorrect final timestamp when calculated using intermediate offsets landing in DST transition, for e.g. +21D arrived at using +14D and +7D offset, where +14D lands in DST transition period. Since bucketStart values are calculated using this library, this behaviour can lead to incorrect bucketStart times.
This change updates dependencies as needed and fixes tests to remove code incompatible with Java 21
As a result all unit tests now pass with Java 21.
* update maven-shade-plugin to 3.5.0 and follow-up to #15042
* explain why we need to override configuration when specifying outputFile
* remove configuration from dependency management in favor of explicit overrides in each module.
* update to mockito to 5.5.0 for Java 21 support when running with Java 11+
* continue using latest mockito 4.x (4.11.0) when running with Java 8
* remove need to mock private fields
* exclude incorrectly declared mockito dependency from pac4j-oidc
* remove mocking of ByteBuffer, since sealed classes can no longer be mocked in Java 21
* add JVM options workaround for system-rules junit plugin not supporting Java 18+
* exclude older versions of byte-buddy from assertj-core
* fix for Java 19 changes in floating point string representation
* fix missing InitializedNullHandlingTest
* update easymock to 5.2.0 for Java 21 compatibility
* update animal-sniffer-plugin to 1.23
* update nl.jqno.equalsverifier to 3.15.1
* update exec-maven-plugin to 3.1.0
This change is meant to fix a issue where passing too large of a task payload to the mm-less task runner will cause the peon to fail to startup because the payload is passed (compressed) as a environment variable (TASK_JSON). In linux systems the limit for a environment variable is commonly 128KB, for windows systems less than this. Setting a env variable longer than this results in a bunch of "Argument list too long" errors.
The aggregators had incorrect types for getResultType when shouldFinalze
is false. They had the finalized type, but they should have had the
intermediate type.
Also includes a refactor of how ExprMacroTable is handled in tests, to make
it easier to add tests for this to the MSQ module. The bug was originally
noticed because the incorrect result types caused MSQ queries with DS_HLL
to behave erratically.
These were added in #14977, but the implementations are incorrect, because they return null when the input arg is null. They should return false when the input is null. Remove them for now, rather than fixing them, since they're so new that they might as well never have existed.
Changes:
- Add task context parameter `taskLockType`. This determines the type of lock used by a batch task.
- Add new task actions for transactional replace and append of segments
- Add methods StorageCoordinator.commitAppendSegments and commitReplaceSegments
- Upgrade segments to appropriate versions when performing replace and append
- Add new metadata table `upgradeSegments` to track segments that need to be upgraded
- Add tests
* Adding new function decode_base64_utf8 and expr macro
* using BaseScalarUnivariateMacroFunctionExpr
* Print stack trace in case of debug in ChainedExecutionQueryRunner
* fix static check
* update RoaringBitmap to 0.9.49
update RoaringBitmap from 0.9.0 to 0.9.49
Many optimizations and improvements have gone into recent releases of
RoaringBitmap. It seems worthwhile to incorporate those.
* implement workaround for BatchIterator interface change
* add test case for BatchIteratorAdapter.advanceIfNeeded
* Add IS [NOT] DISTINCT FROM to SQL and join matchers.
Changes:
1) Add "isdistinctfrom" and "notdistinctfrom" native expressions.
2) Add "IS [NOT] DISTINCT FROM" to SQL. It uses the new native expressions
when generating expressions, and is treated the same as equals and
not-equals when generating native filters on literals.
3) Update join matchers to have an "includeNull" parameter that determines
whether we are operating in "equals" mode or "is not distinct from"
mode.
* Main changes:
- Add ARRAY handling to "notdistinctfrom" and "isdistinctfrom".
- Include null in pushed-down filters when using "notdistinctfrom" in a join.
Other changes:
- Adjust join filter analyzer to more explicitly use InDimFilter's ValuesSets,
relying less on remembering to get it right to avoid copies.
* Remove unused "wrap" method.
* Fixes.
* Remove methods we do not need.
* Fix bug with INPUT_REF.
This is due to the recursive filter creation in unnest storage adapter not performing correctly in case of an empty children. This PR addresses the issue
* Fix for latest agg to handle nulls in time column. Also adding optimization for dictionary encoded string columns
* One minor fix
* Adding more tests for the new class
* Changing the init to a putInt
* Fix for schema mismatch to go down using the non vectorize path till we update the vectorized aggs properly
* Fixing a failed test
* Updating numericNilAgg
* Moving to use default values in case of nil agg
* Adding the same for first agg
* Fixing a test
* fixing vectorized string agg for last/first with cast if numeric
* Updating tests to remove mockito and cover the case of string first/last on non string columns
* Updating a test to vectorize
* Addressing review comments: Name change to NilVectorAggregator and using static variables now
* fixing intellij inspections
Changes:
- Move following configs from `CliCoordinator` to `DruidCoordinatorConfig`:
- `druid.coordinator.kill.on`
- `druid.coordinator.kill.pendingSegments.on`
- `druid.coordinator.kill.supervisors.on`
- `druid.coordinator.kill.rules.on`
- `druid.coordinator.kill.audit.on`
- `druid.coordinator.kill.datasource.on`
- `druid.coordinator.kill.compaction.on`
- In the Coordinator style used by historical management duties, always instantiate all
the metadata cleanup duties but execute only if enabled. In the existing code, they are
instantiated only when enabled by using optional binding with Guice.
- Add a wrapper `MetadataManager` which contains handles to all the different
metadata managers for rules, supervisors, segments, etc.
- Add a `CoordinatorConfigManager` to simplify read and update of coordinator configs
- Remove persistence related methods from `CoordinatorCompactionConfig` and
`CoordinatorDynamicConfig` as these are config classes.
- Remove annotations `@CoordinatorIndexingServiceDuty`,
`@CoordinatorMetadataStoreManagementDuty`
changes:
* add back nested column v4 serializers
* 'json' schema by default still uses the newer 'nested common format' used by 'auto', but now has an optional 'formatVersion' property which can be specified to override format versions on native ingest jobs
* add system config to specify default column format stuff, 'druid.indexing.formats', and property 'druid.indexing.formats.nestedColumnFormatVersion' to specify system level preferred nested column format for friendly rolling upgrades from versions which do not support the newer 'nested common format' used by 'auto'
* update test
* update test
* format
* test
* fix0
* Revert "fix0"
This reverts commit 44992cb393.
* ok resultset
* add plan
* update test
* before rewind
* test
* fix toString/compare/test
* move test
* add timeColumn to hashCode
Changes:
- Add new metric `kill/pendingSegments/count` with dimension `dataSource`
- Add tests for `KillStalePendingSegments`
- Reduce no-op logs that spit out for each datasource even when no pending
segments have been deleted. This can get particularly noisy at low values of `indexingPeriod`.
- Refactor the code in `KillStalePendingSegments` for readability and add javadocs
A new monitor SubqueryCountStatsMonitor which emits the metrics corresponding to the subqueries and their execution is now introduced. Moreover, the user can now also use the auto mode to automatically set the number of bytes available per query for the inlining of its subquery's results.
* Vectorizing earliest for numeric
* Vectorizing earliest string aggregator
* checkstyle fix
* Removing unnecessary exceptions
* Ignoring tests in MSQ as earliest is not supported for numeric there
* Fixing benchmarks
* Updating tests as MSQ does not support earliest for some cases
* Addressing review comments by adding the following:
1. Checking capabilities first before creating selectors
2. Removing mockito in tests for numeric first aggs
3. Removing unnecessary tests
* Addressing issues for dictionary encoded single string columns where we can use the dictionary ids instead of the entire string
* Adding a flag for multi value dimension selector
* Addressing comments
* 1 more change
* Handling review comments part 1
* Handling review comments and correctness fix for latest_by when the time expression need not be in sorted order
* Updating numeric first vector agg
* Revert "Updating numeric first vector agg"
This reverts commit 4291709901.
* Updating code for correctness issues
* fixing an issue with latest agg
* Adding more comments and removing an unnecessary check
* Addressing null checks for tie selector and only vectorize false for quantile sketches
Changes:
- Make ServiceMetricEvent.Builder extend ServiceEventBuilder<ServiceMetricEvent>
and thus convert it to a plain builder rather than a builder of builder.
- Add methods setCreatedTime , setMetricAndValue to the builder
Currently we have an error handler for https connection attempts, but
not for plaintext connection attempts. This leads to warnings like the
following for plaintext connection errors:
EXCEPTION, please implement org.jboss.netty.handler.codec.http.HttpContentDecompressor.exceptionCaught() for proper handling.
This happens because if we don't add our own error handler, the last
handler in the chain during a connection attempt is HttpContentDecompressor,
which doesn't handle errors.
The new error handler for plaintext doesn't do much: it just closes
the channel.
This patch fixes a few issues toward #14858
1. some phony classes were added to enable maven to track the compilation of those classes
2. cyclonedx 2.7.9 seem to handle incremental compilation better; it had a PR relating to that
3. needed to update root pom to 25
4. update antlr to 4.5.3 older one didn't really worked incrementally; 4.5.3 works much better
Currently, Druid is using Guava 16.0.1 version. This upgrade to 31.1-jre fixes the following issues.
CVE-2018-10237 (Unbounded memory allocation in Google Guava 11.0 through 24.x before 24.1.1 allows remote attackers to conduct denial of service attacks against servers that depend on this library and deserialize attacker-provided data because the AtomicDoubleArray class (when serialized with Java serialization) and the CompoundOrdering class (when serialized with GWT serialization) perform eager allocation without appropriate checks on what a client has sent and whether the data size is reasonable). We don't use Java or GWT serializations. Despite being false positive they're causing red security scans on Druid distribution.
Latest version of google-client-api is incompatible with the existing Guava version. This PR unblocks Update google client apis to latest version #14414
Follow up changes to #12599
Changes:
- Rename column `used_flag_last_updated` to `used_status_last_updated`
- Remove new CLI tool `UpdateTables`.
- We already have a `CreateTables` with similar functionality, which should be able to
handle update cases too.
- Any user running the cluster for the first time should either just have `connector.createTables`
enabled or run `CreateTables` which should create tables at the latest version.
- For instance, the `UpdateTables` tool would be inadequate when a new metadata table has
been added to Druid, and users would have to run `CreateTables` anyway.
- Remove `upgrade-prep.md` and include that info in `metadata-init.md`.
- Fix log messages to adhere to Druid style
- Use lambdas
* Add new configurable buffer period to create gap between mark unused and kill of segment
* Changes after testing
* fixes and improvements
* changes after initial self review
* self review changes
* update sql statement that was lacking last_used
* shore up some code in SqlMetadataConnector after self review
* fix derby compatibility and improve testing/docs
* fix checkstyle violations
* Fixes post merge with master
* add some unit tests to improve coverage
* ignore test coverage on new UpdateTools cli tool
* another attempt to ignore UpdateTables in coverage check
* change column name to used_flag_last_updated
* fix a method signature after column name switch
* update docs spelling
* Update spelling dictionary
* Fixing up docs/spelling and integrating altering tasks table with my alteration code
* Update NULL values for used_flag_last_updated in the background
* Remove logic to allow segs with null used_flag_last_updated to be killed regardless of bufferPeriod
* remove unneeded things now that the new column is automatically updated
* Test new background row updater method
* fix broken tests
* fix create table statement
* cleanup DDL formatting
* Revert adding columns to entry table by default
* fix compilation issues after merge with master
* discovered and fixed metastore inserts that were breaking integration tests
* fixup forgotten insert by using pattern of sharing now timestamp across columns
* fix issue introduced by merge
* fixup after merge with master
* add some directions to docs in the case of segment table validation issues
* Update to Calcite 1.35.0
* Update from.ftl for Calcite 1.35.0.
* Fixed tests in Calcite upgrade by doing the following:
1. Added a new rule, CoreRules.PROJECT_FILTER_TRANSPOSE_WHOLE_PROJECT_EXPRESSIONS, to Base rules
2. Refactored the CorrelateUnnestRule
3. Updated CorrelateUnnestRel accordingly
4. Fixed a case with selector filters on the left where Calcite was eliding the virtual column
5. Additional test cases for fixes in 2,3,4
6. Update to StringListAggregator to fail a query if separators are not propagated appropriately
* Refactored for testcases to pass after the upgrade, introduced 2 new data sources for handling filters and select projects
* Added a literalSqlAggregator as the upgraded Calcite involved changes to subquery remove rule. This corrected plans for 2 queries with joins and subqueries by replacing an useless literal dimension with a post agg. Additionally a test with COUNT DISTINCT and FILTER which was failing with Calcite 1.21 is added here which passes with 1.35
* Updated to latest avatica and updated code as SqlUnknownTimeStamp is now used in Calcite which needs to be resolved to a timestamp literal
* Added a wrapper segment ref to use for unnest and filter segment reference
The Azure connector is introduced and MSQ's fault tolerance and durable storage can now be used with Microsoft Azure's blob storage. Also, the results of newly introduced queries from deep storage can now store and fetch the results from Azure's blob storage.
The current version of jackson-databind is flagged for vulnerabilities CVE-2020-28491 (Although cbor format is not used in druid), CVE-2020-36518 (Seems genuine as deeply nested json in can cause resource exhaustion). Updating the dependency to the latest version 2.12.7 to fix these vulnerabilities.
* Minimize PostAggregator computations
Since a change back in 2014, the topN query has been computing
all PostAggregators on all intermediate responses from leaf nodes
to brokers. This generates significant slow downs for queries
with relatively expensive PostAggregators. This change rewrites
the query that is pushed down to only have the minimal set of
PostAggregators such that it is impossible for downstream
processing to do too much work. The final PostAggregators are
applied at the very end.
Fixes a case I missed in #14688 when the return type is STRING but its coming from a top level array typed column instead of a nested array column while making a vector object selector.
Also while here I noticed that the internal JSON_VALUE functions for array types were named inconsistently with the non-array functions, so I renamed them. These are not documented so it should not be disruptive in any way, since they are only used internally for rewrites while planning to make the correctly virtual column.
JSON_VALUE_RETURNING_ARRAY_VARCHAR -> JSON_VALUE_ARRAY_VARCHAR
JSON_VALUE_RETURNING_ARRAY_BIGINT -> JSON_VALUE_ARRAY_BIGINT
JSON_VALUE_RETURNING_ARRAY_DOUBLE -> JSON_VALUE_ARRAY_DOUBLE
The internal non-array functions are JSON_VALUE_VARCHAR, JSON_VALUE_BIGINT, and JSON_VALUE_DOUBLE.
This PR has fixes a bug in the SqlStatementAPI where if the task is not found on the overlord, the response status is 500.
This changes the response to invalid input since the queryID passed is not valid.
* fix issue with nested virtual column array element vector selectors when input is numeric array but output is non-numeric
* add vector value selector for mixed numeric type variant and nested variant fields, tests
* Save a metadata call when reading files from CloudObjectInputSource.
The call to createSplits(inputFormat, null) in formattableReader would
use the default split hint spec, MaxSizeSplitHintSpec, which makes
getObjectMetadata calls in order to compute its splits. This isn't
necessary; we're just trying to unpack the files inside the input
source.
To fix this, use FilePerSplitHintSpec to extract files without any
funny business.
* Adjust call.
* Fix constant.
* Test coverage.
* Frames support for string arrays that are null.
The row format represents null arrays as 0x0001, which older readers
would interpret as an empty array. This provides compatibility with
older readers, which is useful during updates.
The column format represents null arrays by writing -(actual length) - 1
instead of the length, and using FrameColumnWriters.TYPE_STRING_ARRAY for
the type code for string arrays generally. Older readers will report this
as an unrecognized type code. Column format is only used by the operator
query, which is currently experimental, so the impact isn't too severe.
* Remove unused import.
* Return Object[] instead of List from frame array selectors.
Update MSQSelectTest and MSQInsertTest to reflect the fact that null
arrays are possible.
Add a bunch of javadocs to object selectors describing expected behavior,
including the requirement that array selectors return Object[].
* update test case.
* Update test cases.
* fix issues with equality and range filters matching double values to long typed inputs
* adjust to ensure we never homogenize null, [], and [null] into [null] for expressions on real array columns
* allow for batched delete of segments instead of deleting segment data one by one
create new batchdelete method in datasegment killer that has default functionality
of iterating through all segments and calling delete on them. This will enable
a slow rollout of other deepstorage implementations to move to a batched delete
on their own time
* cleanup batchdelete segments
* batch delete with the omni data deleter
cleaned up code
just need to add tests and docs for this functionality
* update java doc to explain how it will try to use batch if function is overwritten
* rename killBatch to kill
add unit tests
* add omniDataSegmentKillerTest for deleting multiple segments at a time. fix checkstyle
* explain test peculiarity better
* clean up batch kill in s3.
* remove unused return value. cleanup comments and fix checkstyle
* default to batch delete. more specific java docs. list segments that couldn't be deleted
if there was a client error or server error
* simplify error handling
* add tests where an exception is thrown when killing multiple s3 segments
* add test for failing to delete two calls with the s3 client
* fix javadoc for kill(List<DataSegment> segments) clean up tests remove feature flag
* fix typo in javadocs
* fix test failure
* fix checkstyle and improve tests
* fix intellij inspections issues
* address comments, make delete multiple segments not assume same bucket
* fix test errors
* better grammar and punctuation. fix test. and better logging for exception
* remove unused code
* avoid extra arraylist instantiation
* fix broken test
* fix broken test
* fix tests to use assert.throws
* Use OverlordClient for all Overlord RPCs.
Continuing the work from #12696, this patch removes HttpIndexingServiceClient
and the IndexingService flavor of DruidLeaderClient completely. All remaining
usages are migrated to OverlordClient.
Supporting changes include:
1) Add a variety of methods to OverlordClient.
2) Update MetadataTaskStorage to skip the complete-task lookup when
the caller requests zero completed tasks. This helps performance of
the "get active tasks" APIs, which don't want to see complete ones.
* Use less forbidden APIs.
* Fixes from CI.
* Add test coverage.
* Two more tests.
* Fix test.
* Updates from CR.
* Remove unthrown exceptions.
* Refactor to improve testability and test coverage.
* Add isNil tests.
* Remove unnecessary "deserialize" methods.
* Add ingest/input/bytes metric and Kafka consumer metrics.
New metrics:
1) ingest/input/bytes. Equivalent to processedBytes in the task reports.
2) kafka/consumer/bytesConsumed: Equivalent to the Kafka consumer
metric "bytes-consumed-total". Only emitted for Kafka tasks.
3) kafka/consumer/recordsConsumed: Equivalent to the Kafka consumer
metric "records-consumed-total". Only emitted for Kafka tasks.
* Fix anchor.
* Fix KafkaConsumerMonitor.
* Interface updates.
* Doc changes.
* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java
Co-authored-by: Benedict Jin <asdf2014@apache.org>
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
* remove extractionFn from equality, null, and range filters
changes:
* EqualityFilter, NullFilter, and RangeFilter no longer support extractionFn
* SQL planner will use ExpressionFilter in the small number of cases where an extractionFn would have been used if sqlUseBoundsAndSelectors is set to false instead of equality/null/range filters
* fix bugs and add tests with serde, equals, and cache key for null, equality, and range filters
* test coverage fixes bugs
* adjust
* adjust again
* so persnickety
* Add EARLIEST aggregator merge strategy.
- More unit tests.
- Include the aggregators analysis type by default in tests.
* Docs.
* Some comments and a test
* Collapse into individual code blocks.
changes:
* new filters that preserve match value typing to better handle filtering different column types
* sql planner uses new filters by default in sql compatible null handling mode
* remove isFilterable from column capabilities
* proper handling of array filtering, add array processor to column processors
* javadoc for sql test filter functions
* range filter support for arrays, tons more tests, fixes
* add dimension selector tests for mixed type roots
* support json equality
* rename semantic index maker thingys to mostly have plural names since they typically make many indexes, e.g. StringValueSetIndex -> StringValueSetIndexes
* add cooler equality index maker, ValueIndexes
* fix missing string utf8 index supplier
* expression array comparator stuff
This adds a new contrib extension: druid-iceberg-extensions which can be used to ingest data stored in Apache Iceberg format. It adds a new input source of type iceberg that connects to a catalog and retrieves the data files associated with an iceberg table and provides these data file paths to either an S3 or HDFS input source depending on the warehouse location.
Two important dependencies associated with Apache Iceberg tables are:
Catalog : This extension supports reading from either a Hive Metastore catalog or a Local file-based catalog. Support for AWS Glue is not available yet.
Warehouse : This extension supports reading data files from either HDFS or S3. Adapters for other cloud object locations should be easy to add by extending the AbstractInputSourceAdapter.
MSQ engine returns correct error codes for invalid user inputs in the query context. Also, using DruidExceptions for MSQ related errors happening in the Broker with improved error messages.
* Add aggregatorMergeStrategy property to SegmentMetadaQuery.
- Adds a new property aggregatorMergeStrategy to segmentMetadata query.
aggregatorMergeStrategy currently supports three types of merge strategies -
the legacy strict and lenient strategies, and the new latest strategy.
- The latest strategy considers the latest aggregator from the latest segment
by time order when there's a conflict when merging aggregators from different
segments.
- Deprecate lenientAggregatorMerge property; The API validates that both the new
and old properties are not set, and returns an exception.
- When merging segments as part of segmentMetadata query, the segments have a more
elaborate id -- <datasource>_<interval>_merged_<partition_number> format, similar to
the name format that segments usually contain. Previously it was simply "merged".
- Adjust unit tests to test the latest strategy, to assert the returned complete
SegmentAnalysis object instead of just the aggregators for completeness.
* Don't explicitly set strict strategy in tests
* Apply suggestions from code review
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* Update docs/querying/segmentmetadataquery.md
* Apply suggestions from code review
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
---------
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
Uses a custom continusou jfr profiler.
Modifies the github actions for tests to do profiling only in the case
of jdk17, as the profiler requires jdk17+ to use the JFR streaming API
plus a few other language features in the code.
Continuous Profiling service is provided to the Apache Druid project
free of charge by Imply and any committer can request free access to
the UI.
* Fix a resource leak with Window processing
Additionally, in order to find the leak, there were
adjustments to the StupidPool to track leaks a bit better.
It would appear that the pool objects get GC'd during testing
for some reason which was causing some incorrect identification
of leaks from objects that had been returned but were GC'd along
with the pool.
* Suppress unused warning
* Add ZooKeeper connection state alerts and metrics.
- New metric "zk/connected" is an indicator showing 1 when connected,
0 when disconnected.
- New metric "zk/disconnected/time" measures time spent disconnected.
- New alert when Curator connection state enters LOST or SUSPENDED.
* Use right GuardedBy.
* Test fixes, coverage.
* Adjustment.
* Fix tests.
* Fix ITs.
* Improved injection.
* Adjust metric name, add tests.
Two changes:
1) Intern DecompressingByteBufferObjectStrategy. Saves ~32 bytes per column.
2) Split GenericIndexed into GenericIndexed.V1 and GenericIndexed.V2. The
major benefit here is isolating out the ByteBuffers that are only needed
for V2. This saves ~80 bytes for V1 (one buffer instead of two).
There are two ways of estimating heap footprint of an Aggregator:
1) AggregatorFactory#guessAggregatorHeapFootprint
2) AggregatorFactory#factorizeWithSize + Aggregator#aggregateWithSize
When the second path is used, the default implementation of factorizeWithSize
is now updated to delegate to guessAggregatorHeapFootprint, making these equivalent.
The old logic used getMaxIntermediateSize, which is less accurate.
Also fixes a bug where, when using the second path, calling factorizeWithSize
on PassthroughAggregatorFactory would fail because getMaxIntermediateSize was
not implemented. (There is no buffer aggregator, so there would be no need.)
Cache is disabled for GroupByStrategyV2 on broker since the pr #3820 [groupBy v2: Results not fully merged when caching is enabled on the broker]. But we can enable the result-level cache on broker for GroupByStrategyV2 and keep the segment-level cache disabled.
* Claim full support for Java 17.
No production code has changed, except the startup scripts.
Changes:
1) Allow Java 17 without DRUID_SKIP_JAVA_CHECK.
2) Include the full list of opens and exports on both Java 11 and 17.
3) Document that Java 17 is both supported and preferred.
4) Switch some tests from Java 11 to 17 to get better coverage on the
preferred version.
* Doc update.
* Update errorprone.
* Update docker_build_containers.sh.
* Update errorprone in licenses.yaml.
* Add some more run-javas.
* Additional run-javas.
* Update errorprone.
* Suppress new errorprone error.
* Add exports and opens in ForkingTaskRunner for Java 11+.
Test, doc changes.
* Additional errorprone updates.
* Update for errorprone.
* Restore old fomatting in LdapCredentialsValidator.
* Copy bin/ too.
* Fix Java 15, 17 build line in docker_build_containers.sh.
* Update busybox image.
* One more java command.
* Fix interpolation.
* IT commandline refinements.
* Switch to busybox 1.34.1-glibc.
* POM adjustments, build and test one IT on 17.
* Additional debugging.
* Fix silly thing.
* Adjust command line.
* Add exports and opens one more place.
* Additional harmonization of strong encapsulation parameters.
One of the most requested features in druid is to have an ability to download big result sets.
As part of #14416 , we added an ability for MSQ to be queried via a query friendly endpoint. This PR builds upon that work and adds the ability for MSQ to write select results to durable storage.
We write the results to the durable storage location <prefix>/results/<queryId> in the druid frame format. This is exposed to users by
/v2/sql/statements/:queryId/results.
* Fix ColumnSignature error message and jdk17 test issue.
On jdk17, the "problem" part of the error message could change from
NullPointerException to:
Cannot invoke "String.length()" because "s" is null
Due to the new more-helpful NPEs in Java 17. This broke the expectation
and led to test failures on this case.
This patch fixes the problem by improving the error message so it isn't
a generic NullPointerException.
* Fix format.
This commit borrows some test definitions from Drill's test suite
and tries to use them to flesh out the full validation of window
function capbilities.
In order to be able to run these tests, we also add the ability to
run a Scan operation against segments, which also meant an
implementation of RowsAndColumns for frames.
UniformGranularityTest's test to test a large number of intervals
runs through 10 years of 1 second intervals. This pushes a lot of
stuff through IntervalIterator and shows up in terms of test
runtime as one of the hottest tests. Most of the time is going to
constructing jodatime objects because it is doing things with
DateTime objects instead of millis. Change the calls to use
millis instead and things go faster.
If a server is removed during `HttpServerInventoryView.serverInventoryInitialized`,
the initialization gets stuck as this server is never synced. The method eventually times
out (default 250s).
Fix: Mark a server as stopped if it is removed. `serverInventoryInitialized` only waits for
non-stopped servers to sync.
Other changes:
- Add new metrics for better debugging of slow broker/coordinator startup
- `segment/serverview/sync/healthy`: whether the server view is syncing properly with a server
- `segment/serverview/sync/unstableTime`: time for which sync with a server has been unstable
- Clean up logging in `HttpServerInventoryView` and `ChangeRequestHttpSyncer`
- Minor refactor for readability
- Add utility class `Stopwatch`
- Add tests and stubs
* combine string column implementations
changes:
* generic indexed, front-coded, and auto string columns now all share the same column and index supplier implementations
* remove CachingIndexed implementation, which I think is largely no longer needed by the switch of many things to directly using ByteBuffer, avoiding the cost of creating Strings
* remove ColumnConfig.columnCacheSizeBytes since CachingIndexed was the only user
* Add "stringEncoding" parameter to DataSketches HLL.
Builds on the concept from #11172 and adds a way to feed HLL sketches
with UTF-8 bytes.
This must be an option rather than always-on, because prior to this
patch, HLL sketches used UTF-16LE encoding when hashing strings. To
remain compatible with sketch images created prior to this patch -- which
matters during rolling updates and when reading sketches that have been
written to segments -- we must keep UTF-16LE as the default.
Not currently documented, because I'm not yet sure how best to expose
this functionality to users. I think the first place would be in the SQL
layer: we could have it automatically select UTF-8 or UTF-16LE when
building sketches at query time. We need to be careful about this, though,
because UTF-8 isn't always faster. Sometimes, like for the results of
expressions, UTF-16LE is faster. I expect we will sort this out in
future patches.
* Fix benchmark.
* Fix style issues, improve test coverage.
* Put round back, to make IT updates easier.
* Fix test.
* Fix issue with filtered aggregators and add test.
* Use DS native update(ByteBuffer) method. Improve test coverage.
* Add another suppression.
* Fix ITAutoCompactionTest.
* Update benchmarks.
* Updates.
* Fix conflict.
* Adjustments.
In these other cases, stick to plain "filter". This simplifies lots of
logic downstream, and doesn't hurt since we don't have intervals-specific
optimizations outside of tables.
Fixes an issue where we couldn't properly filter on a column from an
external datasource if it was named __time.
* Properly read SQL-compatible segments in default-value mode.
Main changes:
1) Dictionary-encoded and front-coded string columns: in default-value
mode, detect cases where a dictionary has the empty string in it, then
either combine it with null (if null is present) or replace it with
null (if null is not present).
2) Numeric nullable columns: in default-value mode, ignore the null
value bitmap. This causes all null numbers to be read as zeroes.
Testing strategy:
1) Add a mmappedWithSqlCompatibleNulls case to BaseFilterTest that
writes segments under SQL-compatible mode, and reads them under
default-value mode.
2) Unit tests for the new wrapper classes (CombineFirstTwoEntriesIndexed,
CombineFirstTwoValuesColumnarInts, CombineFirstTwoValuesColumnarMultiInts,
CombineFirstTwoValuesIndexedInts).
* Fix a mistake, use more singlethreadedness.
* WIP
* Tests, improvements.
* Style.
* See Spot bug.
* Remove unused method.
* Address review comments.
1) Read bitmaps even if we don't retain them.
2) Combine StringFrontCodedDictionaryEncodedColumn and ScalarStringDictionaryEncodedColumn.
* Add missing tests.
This PR aims to expose a new API called
"@path("/druid/v2/sql/statements/")" which takes the same payload as the current "/druid/v2/sql" endpoint and allows users to fetch results in an async manner.
* Fix another infinite loop and remove Mockito usage
The ConfigManager objects were `started()` without ever being
stopped. This scheduled a poll call that never-ended, to make
matters worse, the poll interval was set to 0 ms, making an
infinite poll with 0 sleep, i.e. an infinite loop.
Also introduce test classes and remove usage of mocks
* Checkstyle
Adds support for automatic cleaning of a "query-results" directory in durable storage. This directory will be cleaned up only if the task id is not known to the overlord. This will allow the storage of query results after the task has finished running.
* Cache parsed expressions and binding analysis in more places.
Main changes:
1) Cache parsed and analyzed expressions within PlannerContext for a
single SQL query.
2) Cache parsed expressions together with input binding analysis using
a new class AnalyzeExpr.
This speeds up SQL planning, because SQL planning involves parsing
analyzing the same expression strings over and over again.
* Fixes.
* Fix style.
* Fix test.
* Simplify: get rid of AnalyzedExpr, focus on caching.
* Rename parse -> parseExpression.
Changes:
- Throw an `InsertCannotAllocateSegmentFault` if the allocated segment is not aligned with
the requested granularity.
- Tests to verify new behaviour
Users can now add a guardrail to prevent subquery’s results from exceeding the set number of bytes by setting druid.server.http.maxSubqueryRows in Broker's config or maxSubqueryRows in the query context. This feature is experimental for now and would default back to row-based limiting in case it fails to get the accurate size of the results consumed by the query.
Recently, we have seen flakiness in these two tests, apparently due to
computations based on Runtime.getRuntime().maxMemory() differing during
static initialization and in the actual tests. I can't think of a reason
why this would be happening, but anyway, this patch switches the tests to
use the statics instead of recomputing Runtime.getRuntime().maxMemory().
* Fix compatibility issue with SqlTaskResource
The DruidException changes broke the response format
for errors coming back from the SqlTaskResource, so fix those
Added a new monitor SysMonitorOshi to replace SysMonitor. The new monitor has a wider support for different machine architectures including ARM instances. Please switch to SysMonitorOshi as SysMonitor is now deprecated and will be removed in future releases.
This commit does a complete revamp of the coordinator to address problem areas:
- Stability: Fix several bugs, add capabilities to prioritize and cancel load queue items
- Visibility: Add new metrics, improve logs, revamp `CoordinatorRunStats`
- Configuration: Add dynamic config `smartSegmentLoading` to automatically set
optimal values for all segment loading configs such as `maxSegmentsToMove`,
`replicationThrottleLimit` and `maxSegmentsInNodeLoadingQueue`.
Changed classes:
- Add `StrategicSegmentAssigner` to make assignment decisions for load, replicate and move
- Add `SegmentAction` to distinguish between load, replicate, drop and move operations
- Add `SegmentReplicationStatus` to capture current state of replication of all used segments
- Add `SegmentLoadingConfig` to contain recomputed dynamic config values
- Simplify classes `LoadRule`, `BroadcastRule`
- Simplify the `BalancerStrategy` and `CostBalancerStrategy`
- Add several new methods to `ServerHolder` to track loaded and queued segments
- Refactor `DruidCoordinator`
Impact:
- Enable `smartSegmentLoading` by default. With this enabled, none of the following
dynamic configs need to be set: `maxSegmentsToMove`, `replicationThrottleLimit`,
`maxSegmentsInNodeLoadingQueue`, `useRoundRobinSegmentAssignment`,
`emitBalancingStats` and `replicantLifetime`.
- Coordinator reports richer metrics and produces cleaner and more informative logs
- Coordinator uses an unlimited load queue for all serves, and makes better assignment decisions
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.
Description:
Druid allows a configuration of load rules that may cause a used segment to not be loaded
on any historical. This status is not tracked in the sys.segments table on the broker, which
makes it difficult to determine if the unavailability of a segment is expected and if we should
not wait for it to be loaded on a server after ingestion has finished.
Changes:
- Track replication factor in `SegmentReplicantLookup` during evaluation of load rules
- Update API `/druid/coordinator/v1metadata/segments` to return replication factor
- Add column `replication_factor` to the sys.segments virtual table and populate it in
`MetadataSegmentView`
- If this column is 0, the segment is not assigned to any historical and will not be loaded.
* fix kafka input format reader schema discovery and partial schema discovery to actually work right, by re-using dimension filtering logic of MapInputRowParser
changes:
* auto columns no longer participate in generic 'null column' handling, this was a mistake to try to support and caused ingestion failures due to mismatched ColumnFormat, and will be replaced in the future with nested common format constant column functionality (not in this PR)
* fix bugs with auto columns which contain empty objects, empty arrays, or primitive types mixed with either of these empty constructs
* fix bug with bound filter when upper is null equivalent but is strict
Changes
- Add a `DruidException` which contains a user-facing error message, HTTP response code
- Make `EntryExistsException` extend `DruidException`
- If metadata store max_allowed_packet limit is violated while inserting a new task, throw
`DruidException` with response code 400 (bad request) to prevent retries
- Add `SQLMetadataConnector.isRootCausePacketTooBigException` with impl for MySQL
The class apparently only exists to add a toString()
method to Indexes, which basically just crashes any debugger
on any meaningfully sized index. It's a pointless
abstract class that basically only causes pain.
In this PR, we are enhancing KafkaEmitter, to emit metadata about published segments (SegmentMetadataEvent) into a Kafka topic. This segment metadata information that gets published into Kafka, can be used by any other downstream services to query Druid intelligently based on the segments published. The segment metadata gets published into kafka topic in json string format similar to other events.
### Description
This change allows for consideration of the input format and compression when computing how to split the input files among available tasks, in MSQ ingestion, when considering the value of the `maxInputBytesPerWorker` query context parameter. This query parameter allows users to control the maximum number of bytes, with granularity of input file / object, that ingestion tasks will be assigned to ingest. With this change, this context parameter now denotes the estimated weighted size in bytes of the input to split on, with consideration for input format and compression format, rather than the actual file size, reported by the file system. We assume uncompressed newline delimited json as a baseline, with scaling factor of `1`. This means that when computing the byte weight that a file has towards the input splitting, we take the file size as is, if uncompressed json, 1:1. It was found during testing that gzip compressed json, and parquet, has scale factors of `4` and `8` respectively, meaning that each byte of data is weighted 4x and 8x respectively, when computing input splits. This weighted byte scaling is only considered for MSQ ingestion that uses either LocalInputSource or CloudObjectInputSource at the moment. The default value of the `maxInputBytesPerWorker` query context parameter has been updated from 10 GiB, to 512 MiB
This PR adds a new interface to control how SegmentMetadataCache chooses ColumnType when faced with differences between segments for SQL schemas which are computed, exposed as druid.sql.planner.metadataColumnTypeMergePolicy and adds a new 'least restrictive type' mode to allow choosing the type that data across all segments can best be coerced into and sets this as the default behavior.
This is a behavior change around when segment driven schema migrations take effect for the SQL schema. With latestInterval, the SQL schema will be updated as soon as the first job with the new schema has published segments, while using leastRestrictive, the schema will only be updated once all segments are reindexed to the new type. The benefit of leastRestrictive is that it eliminates a bunch of type coercion errors that can happen in SQL when types are varied across segments with latestInterval because the newest type is not able to correctly represent older data, such as if the segments have a mix of ARRAY and number types, or any other combinations that lead to odd query plans.
* Expr getCacheKey now delegates to children
* Removed the LOOKUP_EXPR_CACHE_KEY as we do not need it
* Adding an unit test
* Update processing/src/main/java/org/apache/druid/math/expr/Expr.java
Co-authored-by: Clint Wylie <cjwylie@gmail.com>
---------
Co-authored-by: Clint Wylie <cjwylie@gmail.com>
* Fixing an issue with filtering on a single dimension by converting In filter to a selector filter as needed with Filters.toFilter
* Adding a test so that any future refactoring does not break this behavior
* Made comment a bit more meaningful
* Be able to load segments on Peons
This change introduces a new config on WorkerConfig
that indicates how many bytes of each storage
location to use for storage of a task. Said config
is divided up amongst the locations and slots
and then used to set TaskConfig.tmpStorageBytesPerTask
The Peons use their local task dir and
tmpStorageBytesPerTask as their StorageLocations for
the SegmentManager such that they can accept broadcast
segments.
Changes:
- Replace `OverlordHelper` with `OverlordDuty` to align with `CoordinatorDuty`
- Each duty has a `run()` method and defines a `Schedule` with an initial delay and period.
- Update existing duties `TaskLogAutoCleaner` and `DurableStorageCleaner`
- Add utility class `Configs`
- Update log, error messages and javadocs
- Other minor style improvements
Changes:
- Do not allow retention rules for any datasource or cluster to be null
- Allow empty rules at the datasource level but not at the cluster level
- Add validation to ensure that `druid.manager.rules.defaultRule` is always set correctly
- Minor style refactors
* fix issues with filtering nulls on values coerced to numeric types
* fix issues with 'auto' type numeric columns in default value mode
* optimize variant typed columns without nested data
* more tests for 'auto' type column ingestion
This PR fixes an issue that could occur if druid.query.scheduler.numThreads is configured and any exception occurs after QueryScheduler.run has been called to create a Sequence. This would result in total and/or lane specific locks being acquired, but because the sequence was not actually being evaluated, the "baggage" which typically releases these locks was not being executed. An example of how this can happen is if a group-by having filter, which wraps and transforms this sequence happens to explode while wrapping the sequence. The end result is that the locks are acquired, but never released, eventually halting the ability to execute any queries.
This PR fixes an issue when using 'auto' encoded LONG typed columns and the 'vectorized' query engine. These columns use a delta based bit-packing mechanism, and errors in the vectorized reader would cause it to incorrectly read column values for some bit sizes (1 through 32 bits). This is a regression caused by #11004, which added the optimized readers to improve performance, so impacts Druid versions 0.22.0+.
While writing the test I finally got sad enough about IndexSpec not having a "builder", so I made one, and switched all the things to use it. Apologies for the noise in this bug fix PR, the only real changes are in VSizeLongSerde, and the tests that have been modified to cover the buggy behavior, VSizeLongSerdeTest and ExpressionVectorSelectorsTest. Everything else is just cleanup of IndexSpec usage.
* Make LoggingEmitter more useful
* Skip code coverage for facade classes
* fix spellcheck
* code review
* fix dependency
* logging.md
* fix checkstyle
* Add back jacoco version to main pom
* TimeBoundary: Use cursor when datasource is not a regular table.
Fixes a bug where TimeBoundary could return incorrect results with
INNER Join or inline data.
* Addl Javadocs.
* Fix two concurrency issues with segment fetching.
1) SegmentLocalCacheManager: Fix a concurrency issue where certain directory
cleanup happened outside of directoryWriteRemoveLock. This created the
possibility that segments would be deleted by one thread, while being
actively downloaded by another thread.
2) TaskDataSegmentProcessor (MSQ): Fix a concurrency issue when two stages
in the same process both use the same segment. For example: a self-join
using distributed sort-merge. Prior to this change, the two stages could
delete each others' segments.
3) ReferenceCountingResourceHolder: increment() returns a new ResourceHolder,
rather than a Releaser. This allows it to be passed to callers without them
having to hold on to both the original ResourceHolder *and* a Releaser.
4) Simplify various interfaces and implementations by using ResourceHolder
instead of Pair and instead of split-up fields.
* Add test.
* Fix style.
* Remove Releaser.
* Updates from master.
* Add some GuardedBys.
* Use the correct GuardedBy.
* Adjustments.
* Improved handling for zero-length intervals.
1) Return an empty list from VersionedIntervalTimeline.lookup when
provided with an empty interval. (The logic doesn't quite work when
intervals are empty, which led to #14129.)
2) Don't return zero-length intervals from JodaUtils.condenseIntervals.
3) Detect "incorrect" comparator in JodaUtils.condenseIntervals, and
recreate the SortedSet if needed. (Not strictly related to the theme
of this patch. Just another thing in the same file.)
4) Remove unused method JodaUtils.containOverlappingIntervals.
Fixes#14129.
* Fix TimewarpOperatorTest.
* MSQ: Subclass CalciteJoinQueryTest, other supporting changes.
The main change is the new tests: we now subclass CalciteJoinQueryTest
in CalciteSelectJoinQueryMSQTest twice, once for Broadcast and once for
SortMerge.
Two supporting production changes for default-value mode:
1) InputNumberDataSource is marked as concrete, to allow leftFilter to
be pushed down to it.
2) In default-value mode, numeric frame field readers can now return nulls.
This is necessary when stacking joins on top of joins: nulls must be
preserved for semantics that match broadcast joins and native queries.
3) In default-value mode, StringFieldReader.isNull returns true on empty
strings in addition to nulls. This is more consistent with the behavior
of the selectors, which map empty strings to null as well in that mode.
As an effect of change (2), the InsertTimeNull change from #14020 (to
replace null timestamps with default timestamps) is reverted. IMO, this
is fine, as either behavior is defensible, and the change from #14020
hasn't been released yet.
* Adjust tests.
* Style fix.
* Additional tests.
* return task status reported by peon
* Write TaskStatus to file in AbstractTask.cleanUp
* Get TaskStatus from task log
* Fix merge conflicts in AbstractTaskTest
* Add unit tests for TaskLogPusher, TaskLogStreamer, NoopTaskLogs to satisfy code coverage
* Add license headerss
* Fix style
* Remove unknown exception declarations
* Allow for Log4J to be configured for peons but still ensure console logging is enforced
This change will allow for log4j to be configured for peons but require console logging is still
configured for them to ensure peon logs are saved to deep storage.
Also fixed the test ConsoleLoggingEnforcementTest to use a valid appender for the non console
Config as the previous config was incorrect and would never return a logger.
* fix checkstyle
* add warning to logger when it overwrites all loggers to be console
* optimize calls for altering logging config for ConsoleLoggingEnforcementConfigurationFactory
add getName to the druid logger class
* update docs, and error message
* edit docs to be more clear
* fix checkstyle issues
* CI fixes - LoggerTest code coverage and fix spelling issue for logging docs
* Updating segment map function for QueryDataSource to ensure group by of group by of join data source gets into proper segment map function path
* Adding unit tests for the failed case
* There you go coverage bot, be happy now
* MSQ: Support for querying lookup and inline data directly.
Main changes:
1) Add of LookupInputSpec and DataSourcePlan.forLookup.
2) Add InlineInputSpec, and modify of DataSourcePlan.forInline to use
this instead of an ExternalInputSpec with JSON. This allows the inline
data to act as the right-hand side of a join, if needed.
Supporting changes:
1) Modify JoinDataSource's leftFilter validation to be a little less
strict: it's now OK with leftFilter being attached to any concrete
leaf (no children) datasource, rather than requiring it be a table.
This allows MSQ to create JoinDataSource with InputNumberDataSource
as the base.
2) Add SegmentWranglerModule to CliIndexer, CliPeon. This allows them to
query lookups and inline data directly.
* Updates based on CI.
* Additional tests.
* Style fix.
* Remove unused import.
* MSQ: Support multiple result columns with the same name.
This is allowed in SQL, and is supported by the regular SQL endpoint.
We retain a validation that INSERT ... SELECT does not allow multiple
columns with the same name, because column names in segments must be
unique.
changes:
* adds support for boolean inputs to the classic long dimension indexer, which plays nice with LONG being the semi official boolean type in Druid, and even nicer when druid.expressions.useStrictBooleans is set to true, since the sampler when using the new 'auto' schema when 'useSchemaDiscovery' is specified on the dimensions spec will call the type out as LONG
* fix bugs with sampler response and new schema discovery stuff incorrectly using classic 'json' type for the logical schema instead of the new 'auto' type
* Frames: Ensure nulls are read as default values when appropriate.
Fixes a bug where LongFieldWriter didn't write a properly transformed
zero when writing out a null. This had no meaningful effect in SQL-compatible
null handling mode, because the field would get treated as a null anyway.
But it does have an effect in default-value mode: it would cause Long.MIN_VALUE
to get read out instead of zero.
Also adds NullHandling checks to the various frame-based column selectors,
allowing reading of nullable frames by servers in default-value mode.
Fixes#13837.
### Description
This change allows for input source type security in the native task layer.
To enable this feature, the user must set the following property to true:
`druid.auth.enableInputSourceSecurity=true`
The default value for this property is false, which will continue the existing functionality of needing authorization to write to the respective datasource.
When this config is enabled, the users will be required to be authorized for the following resource action, in addition to write permission on the respective datasource.
`new ResourceAction(new Resource(ResourceType.EXTERNAL, {INPUT_SOURCE_TYPE}, Action.READ`
where `{INPUT_SOURCE_TYPE}` is the type of the input source being used;, http, inline, s3, etc..
Only tasks that provide a non-default implementation of the `getInputSourceResources` method can be submitted when config `druid.auth.enableInputSourceSecurity=true` is set. Otherwise, a 400 error will be thrown.
* smarter nested column index utilization
changes:
* adds skipValueRangeIndexScale and skipValuePredicateIndexScale to ColumnConfig (e.g. DruidProcessingConfig) available as system config via druid.processing.indexes.skipValueRangeIndexScale and druid.processing.indexes.skipValuePredicateIndexScale
* NestedColumnIndexSupplier uses skipValueRangeIndexScale and skipValuePredicateIndexScale to multiply by the total number of rows to be processed to determine the threshold at which we should no longer consider using bitmap indexes because it will be too many operations
* Default values for skipValueRangeIndexScale and skipValuePredicateIndexScale have been initially set to 0.08, but are separate to allow independent tuning
* these are not documented on purpose yet because they are kind of hard to explain, the mainly exist to help conduct larger scale experiments than the jmh benchmarks used to derive the initial set of values
* these changes provide a pretty sweet performance boost for filter processing on nested columns
* Always use file sizes when determining batch ingest splits.
Main changes:
1) Update CloudObjectInputSource and its subclasses (S3, GCS,
Azure, Aliyun OSS) to use SplitHintSpecs in all cases. Previously, they
were only used for prefixes, not uris or objects.
2) Update ExternalInputSpecSlicer (MSQ) to consider file size. Previously,
file size was ignored; all files were treated as equal weight when
determining splits.
A side effect of these changes is that we'll make additional network
calls to find the sizes of objects when users specify URIs or objects
as opposed to prefixes. IMO, this is worth it because it's the only way
to respect the user's split hint and task assignment settings.
Secondary changes:
1) S3, Aliyun OSS: Use getObjectMetadata instead of listObjects to get
metadata for a single object. This is a simpler call that is also
expected to be less expensive.
2) Azure: Fix a bug where getBlobLength did not populate blob
reference attributes, and therefore would not actually retrieve the
blob length.
3) MSQ: Align dynamic slicing logic between ExternalInputSpecSlicer and
TableInputSpecSlicer.
4) MSQ: Adjust WorkerInputs to ensure there is always at least one
worker, even if it has a nil slice.
* Add msqCompatible to testGroupByWithImpossibleTimeFilter.
* Fix tests.
* Add additional tests.
* Remove unused stuff.
* Remove more unused stuff.
* Adjust thresholds.
* Remove irrelevant test.
* Fix comments.
* Fix bug.
* Updates.
changes:
* introduce ColumnFormat to separate physical storage format from logical type. ColumnFormat is now used instead of ColumnCapabilities to get column handlers for segment creation
* introduce new 'auto' type indexer and merger which produces a new common nested format of columns, which is the next logical iteration of the nested column stuff. Essentially this is an automatic type column indexer that produces the most appropriate column for the given inputs, making either STRING, ARRAY<STRING>, LONG, ARRAY<LONG>, DOUBLE, ARRAY<DOUBLE>, or COMPLEX<json>.
* revert NestedDataColumnIndexer, NestedDataColumnMerger, NestedDataColumnSerializer to their version pre #13803 behavior (v4) for backwards compatibility
* fix a bug in RoaringBitmapSerdeFactory if anything actually ever wrote out an empty bitmap using toBytes and then later tried to read it (the nerve!)
* select sum(c) on an unnested column now does not return 'Type mismatch' error and works properly
* Making sure an inner join query works properly
* Having on unnested column with a group by now works correctly
* count(*) on an unnested query now works correctly
While using intermediateSuperSorterStorageMaxLocalBytes the super sorter was retaining references of the memory allocator.
The fix clears the current outputChannel when close() is called on the ComposingWritableFrameChannel.java
* Reworking s3 connector with
1. Adding retries
2. Adding max fetch size
3. Using s3Utils for most of the api's
4. Fixing bugs in DurableStorageCleaner
5. Moving to Iterator for listDir call
array columns!
changes:
* add support for storing nested arrays of string, long, and double values as specialized nested columns instead of breaking them into separate element columns
* nested column type mimic behavior means that columns ingested with only root arrays of primitive values will be ARRAY typed columns
* neat test refactor stuff
* add v4 segment test
* add array element indexes
* add tests for unnest and array columns
* fix unnest column value selector cursor handling of null and empty arrays
* Refactoring and bug fixes on top of unnest. The filter now is passed inside the unnest cursors. Added tests for scenarios such as
1. filter on unnested column which involves a left filter rewrite
2. filter on unnested virtual column which pushes the filter to the right only and involves no rewrite
3. not filters
4. SQL functions applied on top of unnested column
5. null present in first row of the column to be unnested
changes:
* fixes inconsistent handling of byte[] values between ExprEval.bestEffortOf and ExprEval.ofType, which could cause byte[] values to end up as java toString values instead of base64 encoded strings in ingest time transforms
* improved ExpressionTransform binding to re-use ExprEval.bestEffortOf when evaluating a binding instead of throwing it away
* improved ExpressionTransform array handling, added RowFunction.evalDimension that returns List<String> to back Row.getDimension and remove the automatic coercing of array types that would typically happen to expression transforms unless using Row.getDimension
* added some tests for ExpressionTransform with array inputs
* improved ExpressionPostAggregator to use partial type information from decoration
* migrate some test uses of InputBindings.forMap to use other methods
* Adds new implementation of 'frontCoded' string encoding strategy, which writes out a v1 FrontCodedIndexed which stores buckets on a prefix of the previous value instead of the first value in the bucket
* Refactoring and bug fixes on top of unnest. The filter now is passed inside the unnest cursors. Added tests for scenarios such as
1. filter on unnested column which involves a left filter rewrite
2. filter on unnested virtual column which pushes the filter to the right only and involves no rewrite
3. not filters
4. SQL functions applied on top of unnested column
5. null present in first row of the column to be unnested
* Various changes and fixes to UNNEST.
Native changes:
1) UnnestDataSource: Replace "column" and "outputName" with "virtualColumn".
This enables pushing expressions into the datasource. This in turn
allows us to do the next thing...
2) UnnestStorageAdapter: Logically apply query-level filters and virtual
columns after the unnest operation. (Physically, filters are pulled up,
when possible.) This is beneficial because it allows filters and
virtual columns to reference the unnested column, and because it is
consistent with how the join datasource works.
3) Various documentation updates, including declaring "unnest" as an
experimental feature for now.
SQL changes:
1) Rename DruidUnnestRel (& Rule) to DruidUnnestRel (& Rule). The rel
is simplified: it only handles the UNNEST part of a correlated join.
Constant UNNESTs are handled with regular inline rels.
2) Rework DruidCorrelateUnnestRule to focus on pulling Projects from
the left side up above the Correlate. New test testUnnestTwice verifies
that this works even when two UNNESTs are stacked on the same table.
3) Include ProjectCorrelateTransposeRule from Calcite to encourage
pushing mappings down below the left-hand side of the Correlate.
4) Add a new CorrelateFilterLTransposeRule and CorrelateFilterRTransposeRule
to handle pulling Filters up above the Correlate. New tests
testUnnestWithFiltersOutside and testUnnestTwiceWithFilters verify
this behavior.
5) Require a context feature flag for SQL UNNEST, since it's undocumented.
As part of this, also cleaned up how we handle feature flags in SQL.
They're now hooked into EngineFeatures, which is useful because not
all engines support all features.
With SuperSorter using the PartitionedOutputChannels for sorting, it might OOM on inputs of reasonable size because the channel consists of both the writable frame channel and the frame allocator, both of which are not required once the output channel has been written to.
This change adds a readOnly to the output channel which contains only the readable channel, due to which unnecessary memory references to the writable channel and the memory allocator are lost once the output channel has been written to, preventing the OOM.
* Window planning: use collation traits, improve subquery logic.
SQL changes:
1) Attach RelCollation (sorting) trait to any PartialDruidQuery
that ends in AGGREGATE or AGGREGATE_PROJECT. This allows planning to
take advantage of the fact that Druid sorts by dimensions when
doing aggregations.
2) Windowing: inspect RelCollation trait from input, and insert naiveSort
if, and only if, necessary.
3) Windowing: add support for Project after Window, when the Project
is a simple mapping. Helps eliminate subqueries.
4) DruidRules: update logic for considering subqueries to reflect that
subqueries are not required to be GroupBys, and that we have a bunch
of new Stages now. With all of this evolution that has happened, the
old logic didn't quite make sense.
Native changes:
1) Use merge sort (stable) rather than quicksort when sorting
RowsAndColumns. Makes it easier to write test cases for plans that
involve re-sorting the data.
* Changes from review.
* Mark the bad test as failing.
* Additional update.
* Fix failingTest.
* Fix tests.
* Mark a var final.
* Improve memory efficiency of WrappedRoaringBitmap.
Two changes:
1) Use an int[] for sizes 4 or below.
2) Remove the boolean compressRunOnSerialization. Doesn't save much
space, but it does save a little, and it isn't adding a ton of value
to have it be configurable. It was originally configurable in case
anything broke when enabling it, but it's been a while and nothing
has broken.
* Slight adjustment.
* Adjust for inspection.
* Updates.
* Update snaps.
* Update test.
* Adjust test.
* Fix snaps.
* use custom case operator conversion instead of direct operator conversion, to produce native nvl expression for SQL NVL and 2 argument COALESCE, and add optimization for certain case filters from coalesce and nvl statements
* Sort-merge join and hash shuffles for MSQ.
The main changes are in the processing, multi-stage-query, and sql modules.
processing module:
1) Rename SortColumn to KeyColumn, replace boolean descending with KeyOrder.
This makes it nicer to model hash keys, which use KeyOrder.NONE.
2) Add nullability checkers to the FieldReader interface, and an
"isPartiallyNullKey" method to FrameComparisonWidget. The join
processor uses this to detect null keys.
3) Add WritableFrameChannel.isClosed and OutputChannel.isReadableChannelReady
so callers can tell which OutputChannels are ready for reading and which
aren't.
4) Specialize FrameProcessors.makeCursor to return FrameCursor, a random-access
implementation. The join processor uses this to rewind when it needs to
replay a set of rows with a particular key.
5) Add MemoryAllocatorFactory, which is embedded inside FrameWriterFactory
instead of a particular MemoryAllocator. This allows FrameWriterFactory
to be shared in more scenarios.
multi-stage-query module:
1) ShuffleSpec: Add hash-based shuffles. New enum ShuffleKind helps callers
figure out what kind of shuffle is happening. The change from SortColumn
to KeyColumn allows ClusterBy to be used for both hash-based and sort-based
shuffling.
2) WorkerImpl: Add ability to handle hash-based shuffles. Refactor the logic
to be more readable by moving the work-order-running code to the inner
class RunWorkOrder, and the shuffle-pipeline-building code to the inner
class ShufflePipelineBuilder.
3) Add SortMergeJoinFrameProcessor and factory.
4) WorkerMemoryParameters: Adjust logic to reserve space for output frames
for hash partitioning. (We need one frame per partition.)
sql module:
1) Add sqlJoinAlgorithm context parameter; can be "broadcast" or
"sortMerge". With native, it must always be "broadcast", or it's a
validation error. MSQ supports both. Default is "broadcast" in
both engines.
2) Validate that MSQs do not use broadcast join with RIGHT or FULL join,
as results are not correct for broadcast join with those types. Allow
this in native for two reasons: legacy (the docs caution against it,
but it's always been allowed), and the fact that it actually *does*
generate correct results in native when the join is processed on the
Broker. It is much less likely that MSQ will plan in such a way that
generates correct results.
3) Remove subquery penalty in DruidJoinQueryRel when using sort-merge
join, because subqueries are always required, so there's no reason
to penalize them.
4) Move previously-disabled join reordering and manipulation rules to
FANCY_JOIN_RULES, and enable them when using sort-merge join. Helps
get to better plans where projections and filters are pushed down.
* Work around compiler problem.
* Updates from static analysis.
* Fix @param tag.
* Fix declared exception.
* Fix spelling.
* Minor adjustments.
* wip
* Merge fixups
* fixes
* Fix CalciteSelectQueryMSQTest
* Empty keys are sortable.
* Address comments from code review. Rename mux -> mix.
* Restore inspection config.
* Restore original doc.
* Reorder imports.
* Adjustments
* Fix.
* Fix imports.
* Adjustments from review.
* Update header.
* Adjust docs.
This function is notorious for causing memory exhaustion and excessive
CPU usage; so much so that it was valuable to work around it in the
SQL planner in #13206. Hopefully, a warning comment will encourage
developers to stay away and come up with solutions that do not involve
computing all possible buckets.
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
The FiniteFirehoseFactory and InputRowParser classes were deprecated in 0.17.0 (#8823) in favor of InputSource & InputFormat. This PR removes the FiniteFirehoseFactory and all its implementations along with classes solely used by them like Fetcher (Used by PrefetchableTextFilesFirehoseFactory). Refactors classes including tests using FiniteFirehoseFactory to use InputSource instead.
Removing InputRowParser may not be as trivial as many classes that aren't deprecated depends on it (with no alternatives), like EventReceiverFirehoseFactory. Hence FirehoseFactory, EventReceiverFirehoseFactory, and Firehose are marked deprecated.
* move numeric null value coercion out of expression processing engine
* add ExprEval.valueOrDefault() to allow consumers to automatically coerce to default values
* rename Expr.buildVectorized as Expr.asVectorProcessor more consistent naming with Function and ApplyFunction; javadocs for some stuff
* Speed up composite key joins on IndexedTable.
Prior to this patch, IndexedTable indexes are sorted IntList. This works
great when we have a single-column join key: we simply retrieve the list
and we know what rows match. However, when we have a composite key, we
need to merge the sorted lists. This is inefficient when one is very dense
and others are very sparse.
This patch switches from sorted IntList to IntSortedSet, and changes
to the following intersection algorithm:
1) Initialize the intersection set to the smallest matching set from the
various parts of the composite key.
2) For each element in that smallest set, check other sets for that element.
If any do *not* include it, then remove the element from the intersection
set.
This way, complexity scales with the size of the smallest set, not the
largest one.
* RangeIntSet stuff.
* 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
* fix array_agg to work with complex types and bugs with expression aggregator complex array handling
* more consistent handling of array expressions, numeric arrays more consistently honor druid.generic.useDefaultValueForNull, fix array_ordinal sql output type
* Allow users to add additional metadata to ingestion metrics
When submitting an ingestion spec, users may pass a map of metadata
in the ingestion spec config that will be added to ingestion metrics.
This will make it possible for operators to tag metrics with other
metadata that doesn't necessarily line up with the existing tags
like taskId.
Druid clusters that ingest these metrics can take advantage of the
nested data columns feature to process this additional metadata.
* rename to tags
* docs
* tests
* fix test
* make code cov happy
* checkstyle
changes:
* modified druid schema column type compution to special case COMPLEX<json> handling to choose COMPLEX<json> if any column in any segment is COMPLEX<json>
* NestedFieldVirtualColumn can now work correctly on any type of column, returning either a column selector if a root path, or nil selector if not
* fixed a random bug with NilVectorSelector when using a vector size larger than the default and druid.generic.useDefaultValueForNull=false would have the nulls vector set to all false instead of true
* fixed an overly aggressive check in ExprEval.ofType when handling complex types which would try to treat any string as base64 without gracefully falling back if it was not in fact base64 encoded, along with special handling for complex<json>
* added ExpressionVectorSelectors.castValueSelectorToObject and ExpressionVectorSelectors.castObjectSelectorToNumeric as convience methods to cast vector selectors using cast expressions without the trouble of constructing an expression. the polymorphic nature of the non-vectorized engine (and significantly larger overhead of non-vectorized expression processing) made adding similar methods for non-vectorized selectors less attractive and so have not been added at this time
* fix inconsistency between nested column indexer and serializer in handling values (coerce non primitive and non arrays of primitives using asString)
* ExprEval best effort mode now handles byte[] as string
* added test for ExprEval.bestEffortOf, and add missing conversion cases that tests uncovered
* more tests more better
* 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.
* Adjust Operators to be Pausable
This enables "merge" style operations that
combine multiple streams.
This change includes a naive implementation
of one such merge operator just to provide
concrete evidence that the refactoring is
effective.
* adds the SQL component of the native unnest functionality in Druid to unnest SQL queries on a table dimension, virtual column or a constant array and convert them into native Druid queries
* unnest in SQL is implemented as a combination of Correlate (the comma join part) and Uncollect (the unnest part)
* discover nested columns when using nested column indexer for schemaless
* move useNestedColumnIndexerForSchemaDiscovery from AppendableIndexSpec to DimensionsSpec
* Semantic Implementations for ArrayListRAC
This adds implementations of semantic interfaces
to optimize (eliminate object creation) the
window processing on top of an ArrayListSegment.
Tests are also added to cover the interplay
between the semantic interfaces that are expected
for this use case
* Kinesis: More robust default fetch settings.
1) Default recordsPerFetch and recordBufferSize based on available memory
rather than using hardcoded numbers. For this, we need an estimate
of record size. Use 10 KB for regular records and 1 MB for aggregated
records. With 1 GB heaps, 2 processors per task, and nonaggregated
records, recordBufferSize comes out to the same as the old
default (10000), and recordsPerFetch comes out slightly lower (1250
instead of 4000).
2) Default maxRecordsPerPoll based on whether records are aggregated
or not (100 if not aggregated, 1 if aggregated). Prior default was 100.
3) Default fetchThreads based on processors divided by task count on
Indexers, rather than overall processor count.
4) Additionally clean up the serialized JSON a bit by adding various
JsonInclude annotations.
* Updates for tests.
* Additional important verify.
* single typed "root" only nested columns now mimic "regular" columns of those types
* incremental index can now use nested column indexer instead of string indexer for discovered columns
* Addition of NaiveSortMaker and Default implementation
Add the NaiveSortMaker which makes a sorter
object and a default implementation of the
interface.
This also allows us to plan multiple different window
definitions on the same query.
* Validate response headers and fix exception logging
A class of QueryException were throwing away their
causes making it really hard to determine what's
going wrong when something goes wrong in the SQL
planner specifically. Fix that and adjust tests
to do more validation of response headers as well.
We allow 404s and 307s to be returned even without
authorization validated, but others get converted to 403
* Unify the handling of HTTP between SQL and Native
The SqlResource and QueryResource have been
using independent logic for things like error
handling and response context stuff. This
became abundantly clear and painful during a
change I was making for Window Functions, so
I unified them into using the same code for
walking the response and serializing it.
Things are still not perfectly unified (it would
be the absolute best if the SqlResource just
took SQL, planned it and then delegated the
query run entirely to the QueryResource), but
this refactor doesn't take that fully on.
The new code leverages async query processing
from our jetty container, the different
interaction model with the Resource means that
a lot of tests had to be adjusted to align with
the async query model. The semantics of the
tests remain the same with one exception: the
SqlResource used to not log requests that failed
authorization checks, now it does.
This PR expands `StringDimensionIndexer` to handle conversion of `byte[]` to base64 encoded strings, rather than the current behavior of calling java `toString`.
This issue was uncovered by a regression of sorts introduced by #13519, which updated the protobuf extension to directly convert stuff to java types, resulting in `bytes` typed values being converted as `byte[]` instead of a base64 string which the previous JSON based conversion created. While outputting `byte[]` is more consistent with other input formats, and preferable when the bytes can be consumed directly (such as complex types serde), when fed to a `StringDimensionIndexer`, it resulted in an ugly java `toString` because `processRowValsToUnsortedEncodedKeyComponent` is fed the output of `row.getRaw(..)`. Converting `byte[]` to a base64 string within `StringDimensionIndexer` is consistent with the behavior of calling `row.getDimension(..)` which does do this coercion (and why many tests on binary types appeared to be doing the expected thing).
I added some protobuf `bytes` tests, but they don't really hit the new `StringDimensionIndexer` behavior because they operate on the `InputRow` directly, and call `getDimension` to validate stuff. The parser based version still uses the old conversion mechanisms, so when not using a flattener incorrectly calls `toString` on the `ByteString`. I have encoded this behavior in the test for now, if we either update the parser to use the new flattener or just .. remove parsers we can remove this test stuff.
* bump nested column format version
changes:
* nested field files are now named by their position in field paths list, rather than directly by the path itself. this fixes issues with valid json properties with commas and newlines breaking the csv file meta.smoosh
* update StructuredDataProcessor to deal in NestedPathPart to be consistent with other abstract path handling rather than building JQ syntax strings directly
* add v3 format segment and test
This commit fixes a bug with nested column "value set" indexes caused by not properly
validating that the globalId looked up for value is present in the global dictionary prior to
looking it up in the local dictionary, which when "adjusting" the global ids for value type
can cause incorrect selection of value indexes.
To use an example of a variant typed nested column with 3 values `["1", null, -2]`.
The string dictionary is `[null, "1"]`, the long dictionary is `[-2]` and our local dictionary is `[0, 1, 2]`.
The code for variant typed indexes checks if the value is present in all global dictionaries
and returns indexes for all matches. So in this case, we first lookup "1" in the string dictionary,
find it at global id 1, all is good. Now, we check the long dictionary for `1`, which due to
`-(insertionpoint + 1)` gives us `-(1 + 2) = -2`. Since the global id space is actually stacked
dictionaries, global ids for long and double values must be "adjusted" by the size of string
dictionary, and size of string + size of long for doubles.
Prior to this patch we were not checking that the globalId is 0 or larger, we then immediately
looked up the `localDictionary.indexOf(-2 + adjustLong) = localDictionary.indexOf(-2 + 2) = localDictionary.indexOf(0)` ... which is an actual value contained in the dictionary! The fix is
to skip the longs completely since there were no global matches.
On to doubles, `-(insertionPoint + 1)` gives us `-(0 + 1) = -1`. The double adjust value is '3'
since 2 strings and 1 long, so `localDictionary.indexOf(-1 + 3)` = `localDictionary.indexOf(2)`
which is also a real value in our local dictionary that is definitely not '1'.
So in this one case, looking for '1' incorrectly ended up matching every row.
* Support Framing for Window Aggregations
This adds support for framing over ROWS
for window aggregations.
Still not implemented as yet:
1. RANGE frames
2. Multiple different frames in the same query
3. Frames on last/first functions
This commit adds a new class `InputStats` to track the total bytes processed by a task.
The field `processedBytes` is published in task reports along with other row stats.
Major changes:
- Add class `InputStats` to track processed bytes
- Add method `InputSourceReader.read(InputStats)` to read input rows while counting bytes.
> Since we need to count the bytes, we could not just have a wrapper around `InputSourceReader` or `InputEntityReader` (the way `CountableInputSourceReader` does) because the `InputSourceReader` only deals with `InputRow`s and the byte information is already lost.
- Classic batch: Use the new `InputSourceReader.read(inputStats)` in `AbstractBatchIndexTask`
- Streaming: Increment `processedBytes` in `StreamChunkParser`. This does not use the new `InputSourceReader.read(inputStats)` method.
- Extend `InputStats` with `RowIngestionMeters` so that bytes can be exposed in task reports
Other changes:
- Update tests to verify the value of `processedBytes`
- Rename `MutableRowIngestionMeters` to `SimpleRowIngestionMeters` and remove duplicate class
- Replace `CacheTestSegmentCacheManager` with `NoopSegmentCacheManager`
- Refactor `KafkaIndexTaskTest` and `KinesisIndexTaskTest`
Refactor DataSource to have a getAnalysis method()
This removes various parts of the code where while loops and instanceof
checks were being used to walk through the structure of DataSource objects
in order to build a DataSourceAnalysis. Instead we just ask the DataSource
for its analysis and allow the stack to rebuild whatever structure existed.
* Processors for Window Processing
This is an initial take on how to use Processors
for Window Processing. A Processor is an interface
that transforms RowsAndColumns objects.
RowsAndColumns objects are essentially combinations
of rows and columns.
The intention is that these Processors are the start
of a set of operators that more closely resemble what
DB engineers would be accustomed to seeing.
* Wire up windowed processors with a query type that
can run them end-to-end. This code can be used to
actually run a query, so yay!
* Wire up windowed processors with a query type that
can run them end-to-end. This code can be used to
actually run a query, so yay!
* Some SQL tests for window functions. Added wikipedia
data to the indexes available to the
SQL queries and tests validating the windowing
functionality as it exists now.
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
* Moving all unnest cursor code atop refactored code for unnest
* Updating unnest cursor
* Removing dedup and fixing up some null checks
* AllowList changes
* Fixing some NPEs
* Using bitset for allowlist
* Updating the initialization only when cursor is in non-done state
* Updating code to skip rows not in allow list
* Adding a flag for cases when first element is not in allowed list
* Updating for a null in allowList
* Splitting unnest cursor into 2 subclasses
* Intercepting some apis with columnName for new unnested column
* Adding test cases and renaming some stuff
* checkstyle fixes
* Moving to an interface for Unnest
* handling null rows in a dimension
* Updating cursors after comments part-1
* Addressing comments and adding some more tests
* Reverting a change to ScanQueryRunner and improving a comment
* removing an unused function
* Updating cursors after comments part 2
* One last fix for review comments
* Making some functions private, deleting some comments, adding a test for unnest of unnest with allowList
* Adding an exception for a case
* Closure for unnest data source
* Adding some javadocs
* One minor change in makeDimSelector of columnarCursor
* Updating an error message
* Update processing/src/main/java/org/apache/druid/segment/DimensionUnnestCursor.java
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Unnesting on virtual columns was missing an object array, adding that to support virtual columns unnesting
* Updating exceptions to use UOE
* Renamed files, added column capability test on adapter, return statement and made unnest datasource not cacheable for the time being
* Handling for null values in dim selector
* Fixing a NPE for null row
* Updating capabilities
* Updating capabilities
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
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
* fixes BlockLayoutColumnarLongs close method to nullify internal buffer.
* fixes other BlockLayoutColumnar supplier close methods to nullify internal buffers.
* fix spotbugs
* 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