* 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
We added compression to the latest/first pair storage, but
the code change was forcing new things to be persisted
with the new format, meaning that any segment created with
the new code cannot be read by the old code. Instead, we
need to default to creating the old format and then remove that default in a future version.
* Add string comparison methods to StringUtils, fix dictionary comparisons.
There are various places in Druid code where we assume that String.compareTo
is consistent with Unicode code-point ordering. Sadly this is not the case.
To help deal with this, this patch introduces the following helpers:
1) compareUnicode: Compares two Strings in Unicode code-point order.
2) compareUtf8: Compares two UTF-8 byte arrays in Unicode code-point order.
Equivalent to comparison as unsigned bytes.
3) compareUtf8UsingJavaStringOrdering: Compares two UTF-8 byte arrays, or
ByteBuffers, in a manner consistent with String.compareTo.
There is no helper for comparing two Strings in a manner consistent
with String.compareTo, because for that we can use compareTo directly.
The patch also fixes an inconsistency between the String and UTF-8
dictionary GenericIndexed flavors of string-typed columns: they were
formerly using incompatible comparators.
* Adjust test.
* FrontCodedIndexed updates.
* Add test.
* Fix comments.
Changes:
- Add a metric for partition-wise kafka/kinesis lag for streaming ingestion.
- Emit lag metrics for streaming ingestion when supervisor is not suspended and state is in {RUNNING, IDLE, UNHEALTHY_TASKS, UNHEALTHY_SUPERVISOR}
- Document metrics
* Compaction: Fetch segments one at a time on main task; skip when possible.
Compact tasks include the ability to fetch existing segments and determine
reasonable defaults for granularitySpec, dimensionsSpec, and metricsSpec.
This is a useful feature that makes compact tasks work well even when the
user running the compaction does not have a clear idea of what they want
the compacted segments to be like.
However, this comes at a cost: it takes time, and disk space, to do all
of these fetches. This patch improves the situation in two ways:
1) When segments do need to be fetched, download them one at a time and
delete them when we're done. This still takes time, but minimizes the
required disk space.
2) Don't fetch segments on the main compact task when they aren't needed.
If the user provides a full granularitySpec, dimensionsSpec, and
metricsSpec, we can skip it.
* Adjustments.
* Changes from code review.
* Fix logic for determining rollup.
* Use lookup memory footprint in MSQ memory computations.
Two main changes:
1) Add estimateHeapFootprint to LookupExtractor.
2) Use this in MSQ's IndexerWorkerContext when determining the total
amount of available memory. It's taken off the top.
This prevents MSQ tasks from running out of memory when there are lookups
defined in the cluster.
* Updates from code review.
* First set of changes for framework
* Second set of changes to move segment map function to data source
* Minot change to server manager
* Removing the createSegmentMapFunction from JoinableFactoryWrapper and moving to JoinDataSource
* Checkstyle fixes
* Patching Eric's fix for injection
* Checkstyle and fixing some CI issues
* Fixing code inspections and some failed tests and one injector for test in avatica
* Another set of changes for CI...almost there
* Equals and hashcode part update
* Fixing injector from Eric + refactoring for broadcastJoinHelper
* Updating second injector. Might revert later if better way found
* Fixing guice issue in JoinableFactory
* Addressing review comments part 1
* Temp changes refactoring
* Revert "Temp changes refactoring"
This reverts commit 9da42a9ef0.
* temp
* Temp discussions
* Refactoring temp
* Refatoring the query rewrite to refer to a datasource
* Refactoring getCacheKey by moving it inside data source
* Nullable annotation check in injector
* Addressing some comments, removing 2 analysis.isJoin() checks and correcting the benchmark files
* Minor changes for refactoring
* Addressing reviews part 1
* Refactoring part 2 with new test cases for broadcast join
* Set for nullables
* removing instance of checks
* Storing nullables in guice to avoid checking on reruns
* Fixing a test case and removing an irrelevant line
* Addressing the atomic reference review comments
* add FrontCodedIndexed for delta string encoding
* now for actual segments
* fix indexOf
* fixes and thread safety
* add bucket size 4, which seems generally better
* fixes
* fixes maybe
* update indexes to latest interfaces
* utf8 support
* adjust
* oops
* oops
* refactor, better, faster
* more test
* fixes
* revert
* adjustments
* fix prefixing
* more chill
* sql nested benchmark too
* refactor
* more comments and javadocs
* better get
* remove base class
* fix
* hot rod
* adjust comments
* faster still
* minor adjustments
* spatial index support
* spotbugs
* add isSorted to Indexed to strengthen indexOf contract if set, improve javadocs, add docs
* fix docs
* push into constructor
* use base buffer instead of copy
* oops
* SQL: Use timestamp_floor when granularity is not safe.
PR #12944 added a check at the execution layer to avoid materializing
excessive amounts of time-granular buckets. This patch modifies the SQL
planner to avoid generating queries that would throw such errors, by
switching certain plans to use the timestamp_floor function instead of
granularities. This applies both to the Timeseries query type, and the
GroupBy timestampResultFieldGranularity feature.
The patch also goes one step further: we switch to timestamp_floor
not just in the ETERNITY + non-ALL case, but also if the estimated
number of time-granular buckets exceeds 100,000.
Finally, the patch modifies the timestampResultFieldGranularity
field to consistently be a String rather than a Granularity. This
ensures that it can be round-trip serialized and deserialized, which is
useful when trying to execute the results of "EXPLAIN PLAN FOR" with
GroupBy queries that use the timestampResultFieldGranularity feature.
* Fix test, address PR comments.
* Fix ControllerImpl.
* Fix test.
* Fix unused import.
We introduce two new configuration keys that refine the query context security model controlled by druid.auth.authorizeQueryContextParams. When that value is set to true then two other configuration options become available:
druid.auth.unsecuredContextKeys: The set of query context keys that do not require a security check. Use this for the "white-list" of key to allow. All other keys go through the existing context key security checks.
druid.auth.securedContextKeys: The set of query context keys that do require a security check. Use this when you want to allow all but a specific set of keys: only these keys go through the existing context key security checks.
Both are set using JSON list format:
druid.auth.securedContextKeys=["secretKey1", "secretKey2"]
You generally set one or the other values. If both are set, unsecuredContextKeys acts as exceptions to securedContextKeys.
In addition, Druid defines two query context keys which always bypass checks because Druid uses them internally:
sqlQueryId
sqlStringifyArrays
This is in preparation for eventually retiring the flag `useMaxMemoryEstimates`,
after which the footprint of a value in the dimension dictionary will always be
estimated using the `estimateSizeOfValue()` method.
* fix json_value sql planning with decimal type, fix vectorized expression math null value handling in default mode
changes:
* json_value 'returning' decimal will now plan to native double typed query instead of ending up with default string typing, allowing decimal vector math expressions to work with this type
* vector math expressions now zero out 'null' values even in 'default' mode (druid.generic.useDefaultValueForNull=false) to prevent downstream things that do not check the null vector from producing incorrect results
* more better
* test and why not vectorize
* more test, more fix
* Expose HTTP Response headers from SqlResource
This change makes the SqlResource expose HTTP response
headers in the same way that the QueryResource exposes them.
Fundamentally, the change is to pipe the QueryResponse
object all the way through to the Resource so that it can
populate response headers. There is also some code
cleanup around DI, as there was a superfluous FactoryFactory
class muddying things up.
changes:
* long and double value columns are now written directly, at the same time as writing out the 'intermediary' dictionaryid column with unsorted ids
* remove reverse value lookup from GlobalDictionaryIdLookup since it is no longer needed
* more consistent expression error messages
* review stuff
* add NamedFunction for Function, ApplyFunction, and ExprMacro to share common stuff
* fixes
* add expression transform name to transformer failure, better parse_json error messaging
-Add classes for writing cell values in LZ4 block compressed format.
Payloads are indexed by element number for efficient random lookup
-update SerializablePairLongStringComplexMetricSerde to use block
compression
-SerializablePairLongStringComplexMetricSerde also uses delta encoding
of the Long by doing 2-pass encoding: buffers first to find min/max
numbers and delta-encodes as integers if possible
Entry points for doing block-compressed storage of byte[] payloads
are the CellWriter and CellReader class. See
SerializablePairLongStringComplexMetricSerde for how these are used
along with how to do full column-based storage (delta encoding here)
which includes 2-pass encoding to compute a column header
* FrameFile: Java 17 compatibility.
DataSketches Memory.map is not Java 17 compatible, and from discussions
with the team, is challenging to make compatible with 17 while also
retaining compatibility with 8 and 11. So, in this patch, we switch away
from Memory.map and instead use the builtin JDK mmap functionality. Since
it only supports maps up to Integer.MAX_VALUE, we also implement windowing
in FrameFile, such that we can still handle large files.
Other changes:
1) Add two new "map" functions to FileUtils, which we use in this patch.
2) Add a footer checksum to the FrameFile format. Individual frames
already have checksums, but the footer was missing one.
* Changes for static analysis.
* wip
* Fixes.
* Fix accounting of bytesAdded in ReadableByteChunksFrameChannel.
Could cause WorkerInputChannelFactory to get into an infinite loop when
reading the footer of a frame file.
* Additional tests.
During ingestion, if a row containing multiple values for a numeric dimension is encountered,
the whole ingestion task fails. Ideally, this should just be registered as a parse exception.
Changes:
- Remove `instanceof List` check from `LongDimensionIndexer`, `FloatDimensionIndexer` and `DoubleDimensionIndexer`.
Any invalid type, including list, throws a parse exception in `DimensionHandlerUtils.convertObjectToXXX`
methods. `ParseException` is already handled in `OnHeapIncrementalIndex` and does not fail the entire task.
* json_value adjustments
changes:
* native json_value expression now has optional 3rd argument to specify type, which will cast all values to the specified type
* rework how JSON_VALUE is wired up in SQL. Now we are using a custom convertlet to translate JSON_VALUE(... RETURNING type) into dedicated JSON_VALUE_BIGINT, JSON_VALUE_DOUBLE, JSON_VALUE_VARCHAR, JSON_VALUE_ANY instead of using the calcite StandardConvertletTable that wraps JSON_VALUE_ANY in a CAST, so that we preserve the typing of JSON_VALUE to pass down to the native expression as the 3rd argument
* fix json_value_any to be usable by humans too, coverage
* fix bug
* checkstyle
* checkstyle
* review stuff
* validate that options to json_value are the supported options rather than ignore them
* remove more legacy undocumented functions
* KLL sketch
* added documentation
* direct static refs
* direct static refs
* fixed test
* addressed review points
* added KLL sketch related terms
* return a copy from get
* Copy unions when returning them from "get".
* Remove redundant "final".
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
The method wasn't following its contract, leading to pollution of the
overall planner context, when really we just want to create a new
context for a specific query.
* Error handling improvements for frame channels.
Two changes:
1) Send errors down in-memory channels (BlockingQueueFrameChannel) on
failure. This ensures that in situations where a chain of processors
has been set up on a single machine, all processors see the root
cause error. In particular, this means the final processor in the
chain reports the root cause error, which ensures that someone with
a handle to the final processor will get the proper error.
2) Update FrameFileHttpResponseHandler to expect that the final fetch,
rather than being simply empty, is also empty with a special header.
This ensures that the handler is able to tell the difference between
an empty fetch due to being at EOF, and an empty fetch due to a
truncated HTTP response (after the 200 OK and headers are sent down,
but before any content appears).
* Fix tests, imports.
* Checkstyle!
* Refactor SqlLifecycle into statement classes
Create direct & prepared statements
Remove redundant exceptions from tests
Tidy up Calcite query tests
Make PlannerConfig more testable
* Build fixes
* Added builder to SqlQueryPlus
* Moved Calcites system properties to saffron.properties
* Build fix
* Resolve merge conflict
* Fix IntelliJ inspection issue
* Revisions from reviews
Backed out a revision to Calcite tests that didn't work out as planned
* Build fix
* Fixed spelling errors
* Fixed failed test
Prepare now enforces security; before it did not.
* Rebase and fix IntelliJ inspections issue
* Clean up exception handling
* Fix handling of JDBC auth errors
* Build fix
* More tweaks to security messages
* Introduce defaultOnDiskStorage config for groupBy
* add debug log to groupby query config
* Apply config change suggestion from review
* Remove accidental new lines
* update default value of new default disk storage config
* update debug log to have more descriptive text
* Make maxOnDiskStorage and defaultOnDiskStorage HumanRedadableBytes
* improve test coverage
* Provide default implementation to new default method on advice of reviewer
In the current druid code base, we have the interface DataSegmentPusher which allows us to push segments to the appropriate deep storage without the extension being worried about the semantics of how to push too deep storage.
While working on #12262, whose some part of the code will go as an extension, I realized that we do not have an interface that allows us to do basic "write, get, delete, deleteAll" operations on the appropriate deep storage without let's say pulling the s3-storage-extension dependency in the custom extension.
Hence, the idea of StorageConnector was born where the storage connector sits inside the druid core so all extensions have access to it.
Each deep storage implementation, for eg s3, GCS, will implement this interface.
Now with some Jackson magic, we bind the implementation of the correct deep storage implementation on runtime using a type variable.
* Adjust "in" filter null behavior to match "selector".
Now, both of them match numeric nulls if constructed with a "null" value.
This is consistent as far as native execution goes, but doesn't match
the behavior of SQL = and IN. So, to address that, this patch also
updates the docs to clarify that the native filters do match nulls.
This patch also updates the SQL docs to describe how Boolean logic is
handled in addition to how NULL values are handled.
Fixes#12856.
* Fix test.
* Frame processing and channels.
Follow-up to #12745. This patch adds three new concepts:
1) Frame channels are interfaces for doing nonblocking reads and writes
of frames.
2) Frame processors are interfaces for doing nonblocking processing of
frames received from input channels and sent to output channels.
3) Cluster-by keys, which can be used for sorting or partitioning.
The patch also adds SuperSorter, a user of these concepts, both to
illustrate how they are used, and also because it is going to be useful
in future work.
Central classes:
- ReadableFrameChannel. Implementations include
BlockingQueueFrameChannel (in-memory channel that implements both interfaces),
ReadableFileFrameChannel (file-based channel),
ReadableByteChunksFrameChannel (byte-stream-based channel), and others.
- WritableFrameChannel. Implementations include BlockingQueueFrameChannel
and WritableStreamFrameChannel (byte-stream-based channel).
- ClusterBy, a sorting or partitioning key.
- FrameProcessor, nonblocking processor of frames. Implementations include
FrameChannelBatcher, FrameChannelMerger, and FrameChannelMuxer.
- FrameProcessorExecutor, an executor service that runs FrameProcessors.
- SuperSorter, a class that uses frame channels and processors to
do parallel external merge sort of any amount of data (as long as there
is enough disk space).
* Additional tests, fixes.
* Changes from review.
* Better implementation for ReadableInputStreamFrameChannel.
* Rename getFrameFileReference -> newFrameFileReference.
* Add InterruptedException to runIncrementally; add more tests.
* Cancellation adjustments.
* Review adjustments.
* Refactor BlockingQueueFrameChannel, rename doneReading and doneWriting to close.
* Additional changes from review.
* Additional changes.
* Fix test.
* Adjustments.
* Adjustments.
* Refactor Guice initialization
Builders for various module collections
Revise the extensions loader
Injector builders for server startup
Move Hadoop init to indexer
Clean up server node role filtering
Calcite test injector builder
* Revisions from review comments
* Build fixes
* Revisions from review comments
* Improved Java 17 support and Java runtime docs.
1) Add a "Java runtime" doc page with information about supported
Java versions, garbage collection, and strong encapsulation..
2) Update asm and equalsverifier to versions that support Java 17.
3) Add additional "--add-opens" lines to surefire configuration, so
tests can pass successfully under Java 17.
4) Switch openjdk15 tests to openjdk17.
5) Update FrameFile to specifically mention Java runtime incompatibility
as the cause of not being able to use Memory.map.
6) Update SegmentLoadDropHandler to log an error for Errors too, not
just Exceptions. This is important because an IllegalAccessError is
encountered when the correct "--add-opens" line is not provided,
which would otherwise be silently ignored.
7) Update example configs to use druid.indexer.runner.javaOptsArray
instead of druid.indexer.runner.javaOpts. (The latter is deprecated.)
* Adjustments.
* Use run-java in more places.
* Add run-java.
* Update .gitignore.
* Exclude hadoop-client-api.
Brought in when building on Java 17.
* Swap one more usage of java.
* Fix the run-java script.
* Fix flag.
* Include link to Temurin.
* Spelling.
* Update examples/bin/run-java
Co-authored-by: Xavier Léauté <xl+github@xvrl.net>
Co-authored-by: Xavier Léauté <xl+github@xvrl.net>
add NumericRangeIndex interface and BoundFilter support
changes:
* NumericRangeIndex interface, like LexicographicalRangeIndex but for numbers
* BoundFilter now uses NumericRangeIndex if comparator is numeric and there is no extractionFn
* NestedFieldLiteralColumnIndexSupplier.java now supports supplying NumericRangeIndex for single typed numeric nested literal columns
* better faster stronger and (ever so slightly) more understandable
* more tests, fix bug
* fix style
* Automatic sizing for GroupBy dictionary sizes.
Merging and selector dictionary sizes currently both default to 100MB.
This is not optimal, because it can lead to OOM on small servers and
insufficient resource utilization on larger servers. It also invites
end users to try to tune it when queries run out of dictionary space,
which can make things worse if the end user sets it to too high.
So, this patch:
- Adds automatic tuning for selector and merge dictionaries. Selectors
use up to 15% of the heap and merge buffers use up to 30% of the heap
(aggregate across all queries).
- Updates out-of-memory error messages to emphasize enabling disk
spilling vs. increasing memory parameters. With the memory parameters
automatically sized, it is more likely that an end user will get
benefit from enabling disk spilling.
- Removes the query context parameters that allow lowering of configured
dictionary sizes. These complicate the calculation, and I don't see a
reasonable use case for them.
* Adjust tests.
* Review adjustments.
* Additional comment.
* Remove unused import.
When we return DISK_FULL to a processing thread, it skips the rest of
the segment and the query is canceled. However, it's possible that the
next segment starts processing before cancellation can kick in. We want
that one, if it occurs, to see DISK_FULL too.
* Preserve column order in DruidSchema, SegmentMetadataQuery.
Instead of putting columns in alphabetical order. This is helpful
because it makes query order better match ingestion order. It also
allows tools, like the reindexing flow in the web console, to more
easily do follow-on ingestions using a column order that matches the
pre-existing column order.
We prefer the order from the latest segments. The logic takes all
columns from the latest segments in the order they appear, then adds
on columns from older segments after those.
* Additional test adjustments.
* Adjust imports.
* Frame format for data transfer and short-term storage.
As we move towards query execution plans that involve more transfer
of data between servers, it's important to have a data format that
provides for doing this more efficiently than the options available to
us today.
This patch adds:
- Columnar frames, which support fast querying.
- Row-based frames, which support fast sorting via memory comparison
and fast whole-row copies via memory copying.
- Frame files, a container format that can be stored on disk or
transferred between servers.
The idea is we should use row-based frames when data is expected to
be sorted, and columnar frames when data is expected to be queried.
The code in this patch is not used in production yet. Therefore, the
patch involves minimal changes outside of the org.apache.druid.frame
package. The main ones are adjustments to SqlBenchmark to add benchmarks
for queries on frames, and the addition of a "forEach" method to Sequence.
* Fixes based on tests, static analysis.
* Additional fixes.
* Skip DS mapping tests on JDK 14+
* Better JDK checking in tests.
* Fix imports.
* Additional comment.
* Adjustments from code review.
* Update test case.
* jvm gc to mxbeans
* add zgc and shenandoah #12476
* remove tryCreateGcCounter
* separate the space collector
* blend GcGenerationCollector into GcCollector
* add jdk surefire argLine
* Poison StupidPool and fix resource leaks
There are various resource leaks from test setup as well as some
corners in query processing. We poison the StupidPool to start failing
tests when the leaks come and fix any issues uncovered from that so
that we can start from a clean baseline.
Unfortunately, because of how poisoning works,
we can only fail future checkouts from the same pool,
which means that there is a natural race between a
leak happening -> GC occurs -> leak detected -> pool poisoned.
This race means that, depending on interleaving of tests,
if the very last time that an object is checked out
from the pool leaks, then it won't get caught.
At some point in the future, something will catch it,
however and from that point on it will be deterministic.
* Remove various things left over from iterations
* Clean up FilterAnalysis and add javadoc on StupidPool
* Revert changes to .idea/misc.xml that accidentally got pushed
* Style and test branches
* Stylistic woes
True, false, and null have different meanings: true/false mean "legacy"
and "not legacy"; null means use the default set by ScanQueryConfig.
So, we need to respect this in the JsonIgnore setup.
* Fix self-referential shape inspection in BaseExpressionColumnValueSelector.
The new test would throw StackOverflowError on the old code.
* Restore prior test.
* Remove null and empty fields from native queries
* Test fixes
* Attempted IT fix.
* Revisions from review comments
* Build fixes resulting from changes suggested by reviews
* IT fix for changed segment size
* Clean up query contexts
Uses constants in place of literal strings for context keys.
Moves some QueryContext methods to QueryContexts for reuse.
* Revisions from review comments
Often users are submitting queries, and ingestion specs that work only if the relevant extension is not loaded. However, the error is too technical for the users and doesn't suggest them to check for missing extensions. This PR modifies the error message so users can at least check their settings before assuming that the error is because of a bug.
fixes an issue caused by a test modification in #12408 that was closing buffers allocated by the compression strategy instead of allowing the closer to do it
RowBasedColumnSelectorFactory inherited strange behavior from
Rows.objectToStrings for nulls that appear in lists: instead of being
left as a null, it is replaced with the string "null". Some callers may
need compatibility with this strange behavior, but it should be opt-in.
Query-time call sites are changed to opt-out of this behavior, since it
is not consistent with query-time expectations. The IncrementalIndex
ingestion-time call site retains the old behavior, as this is traditionally
when Rows.objectToStrings would be used.
* Add RowIdSupplier to ColumnSelectorFactory.
This enables virtual columns to cache their outputs in case they are
called multiple times on the same underlying row. This is common for
numeric selectors, where the common pattern is to call isNull() and
then follow with getLong(), getFloat(), or getDouble(). Here, output
caching reduces the number of expression evals by half.
* Fix tests.
* Adding zstandard compression library
* 1. Took @clintropolis's advice to have ZStandard decompressor use the byte array when the buffers are not direct.
2. Cleaned up checkstyle issues.
* Fixing zstandard version to latest stable version in pom's and updating license files
* Removing zstd from benchmarks and adding to processing (poms)
* fix the intellij inspection issue
* Removing the prefix v for the version in the license check for ztsd
* Fixing license checks
Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
Adds a default implementation of getQueryContext, which was added to the Query interface in #12396. Query is marked with @ExtensionPoint, and lately we have been trying to be less volatile on these interfaces by providing default implementations to be more chill for extension writers.
The way this default implementation is done in this PR is a bit strange due to the way that getQueryContext is used (mutated with system default and system generated keys); the default implementation has a specific object that it returns, and I added another temporary default method isLegacyContext that checks if the getQueryContext returns that object or not. If not, callers fall back to using getContext and withOverriddenContext to set these default and system values.
I am open to other ideas as well, but this way should work at least without exploding, and added some tests to ensure that it is wired up correctly for QueryLifecycle, including the context authorization stuff.
The added test shows the strange behavior if query context authorization is enabled, mainly that the system default and system generated query context keys also need to be granted as permissions for things to function correctly. This is not great, so I mentioned it in the javadocs as well. Not sure if it needs to be called out anywhere else.
Description
Fixes a bug when running q's like
SELECT cntarray,
Count(*)
FROM (SELECT dim1,
dim2,
Array_agg(cnt) AS cntarray
FROM (SELECT dim1,
dim2,
dim3,
Count(*) AS cnt
FROM foo
GROUP BY 1,
2,
3)
GROUP BY 1,
2)
GROUP BY 1
This generates an error:
org.apache.druid.java.util.common.ISE: Unable to convert type [Ljava.lang.Object; to org.apache.druid.segment.data.ComparableList
at org.apache.druid.segment.DimensionHandlerUtils.convertToList(DimensionHandlerUtils.java:405) ~[druid-xx]
Because it's an array of numbers it looks like it does the convertToList call, which looks like:
@Nullable
public static ComparableList convertToList(Object obj)
{
if (obj == null) {
return null;
}
if (obj instanceof List) {
return new ComparableList((List) obj);
}
if (obj instanceof ComparableList) {
return (ComparableList) obj;
}
throw new ISE("Unable to convert type %s to %s", obj.getClass().getName(), ComparableList.class.getName());
}
I.e. it doesn't know about arrays. Added the array handling as part of this PR.
* Emit state of replace and append for native batch tasks
* Emit count of one depending on batch ingestion mode (APPEND, OVERWRITE, REPLACE)
* Add metric to compaction job
* Avoid null ptr exc when null emitter
* Coverage
* Emit tombstone & segment counts
* Tasks need a type
* Spelling
* Integrate BatchIngestionMode in batch ingestion tasks functionality
* Typos
* Remove batch ingestion type from metric since it is already in a dimension. Move IngestionMode to AbstractTask to facilitate having mode as a dimension. Add metrics to streaming. Add missing coverage.
* Avoid inner class referenced by sub-class inspection. Refactor computation of IngestionMode to make it more robust to null IOConfig and fix test.
* Spelling
* Avoid polluting the Task interface
* Rename computeCompaction methods to avoid ambiguous java compiler error if they are passed null. Other minor cleanup.