Changes
- Add `log` implementation for `AuditManager` alongwith `SQLAuditManager`
- `LoggingAuditManager` simply logs the audit event. Thus, it returns empty for
all `fetchAuditHistory` calls.
- Add new config `druid.audit.manager.type` which can take values `log`, `sql` (default)
- Add new config `druid.audit.manager.logLevel` which can take values `DEBUG`, `INFO`, `WARN`.
This gets activated only if `type` is `log`.
- Remove usage of `ConfigSerde` from `AuditManager` as audit is not just limited to configs
- Add `AuditSerdeHelper` for a single implementation of serialization/deserialization of
audit payload and other utility methods.
The PR addresses 2 things:
Add MSQ durable storage connector for GCS
Change GCS client library from the old Google API Client Library to the recommended Google Cloud Client Library. Ref: https://cloud.google.com/apis/docs/client-libraries-explained
* Upgrade org.pac4j:pac4j-oidc to 4.5.5 to address CVE-2021-44878
* add CVE suppression and notes, since vulnerability scan still shows this CVE
* Add tests to improve coverage
Update of direct dependencies:
* kubernetes java-client to 19.0.0
* docker-java-bom to 3.3.4
In order to update transitive dependencies:
* okio to 3.6.0
* bcjava to 1.76
To address CVES:
- CVE-2023-3635 in okio
- CVE-2023-33201 in bcjava
---------
Co-authored-by: Xavier Léauté <xvrl@apache.org>
Fixes a potential NPE which could occur while folding the HllSketchAggregator. If the sketch is null, druid could return a null HllSketchHolder object. Adding a null check here could help here
Resolves a null pointer exception in HllSketchAggregatorFactory
This change completes the change introduced in #15461
and unifies the version of gson dependency used between all the modules.
gson is used by kubernetes-extension, avro-extensions, ranger-security,
and as a test dependency in several core modules.
---------
Co-authored-by: Xavier Léauté <xl+github@xvrl.net>
* Excluding jackson-jaxrs dependency from ranger-plugin-common to address CVE regression introduced by ranger-upgrade: CVE-2019-10202, CVE-2019-10172
* remove the reference to outdated ranger 2.0 from the docs
---------
Co-authored-by: Xavier Léauté <xl+github@xvrl.net>
Recent upgrade of ranger introduced CVE regressions due to outdated elasticsearch components.
Druid-ranger-plugin does not elasticsearch components , and they have been explicitly removed.
Update woodstox-core to 6.4.0 to address GHSA-3f7h-mf4q-vrm4
This PR revives #14978 with a few more bells and whistles. Instead of an unconditional cross-join, we will now split the join condition such that some conditions are now evaluated post-join. To decide what sub-condition goes where, I have refactored DruidJoinRule class to extract unsupported sub-conditions. We build a postJoinFilter out of these unsupported sub-conditions and push to the join.
* update confluent's dependencies to common, supported version
Update io.confluent.* dependencies to common, updated version 6.2.12
currently used versions are EOL
* move version definition to the top level pom
Changes:
- Fix log `Got end of partition marker for partition [%s] from task [%s] in discoverTasks`
by fixing order of args
- Simplify in-line classes by using lambda
- Update kill task message from `Task [%s] failed to respond to [set end offsets]
in a timely manner, killing task` to `Failed to set end offsets, killing task`
- Clean up tests
There is a problem with Quantiles sketches and KLL Quantiles sketches.
Queries using the histogram post-aggregator fail if:
- the sketch contains at least one value, and
- the values in the sketch are all equal, and
- the splitPoints argument is not passed to the post-aggregator, and
- the numBins argument is greater than 2 (or not specified, which
leads to the default of 10 being used)
In that case, the query fails and returns this error:
{
"error": "Unknown exception",
"errorClass": "org.apache.datasketches.common.SketchesArgumentException",
"host": null,
"errorCode": "legacyQueryException",
"persona": "OPERATOR",
"category": "RUNTIME_FAILURE",
"errorMessage": "Values must be unique, monotonically increasing and not NaN.",
"context": {
"host": null,
"errorClass": "org.apache.datasketches.common.SketchesArgumentException",
"legacyErrorCode": "Unknown exception"
}
}
This behaviour is undesirable, since the caller doesn't necessarily
know in advance whether the sketch has values that are diverse
enough. With this change, the post-aggregators return [N, 0, 0...]
instead of crashing, where N is the number of values in the sketch,
and the length of the list is equal to numBins. That is what they
already returned for numBins = 2.
Here is an example of a query that would fail:
{"queryType":"timeseries",
"dataSource": {
"type": "inline",
"columnNames": ["foo", "bar"],
"rows": [
["abc", 42.0],
["def", 42.0]
]
},
"intervals":["0000/3000"],
"granularity":"all",
"aggregations":[
{"name":"the_sketch", "fieldName":"bar", "type":"quantilesDoublesSketch"}],
"postAggregations":[
{"name":"the_histogram",
"type":"quantilesDoublesSketchToHistogram",
"field":{"type":"fieldAccess","fieldName":"the_sketch"},
"numBins": 3}]}
I believe this also fixes issue #10585.
* Fix capacity response in mm-less ingestion (#14888)
Changes:
- Fix capacity response in mm-less ingestion.
- Add field usedClusterCapacity to the GET /totalWorkerCapacity response.
This API should be used to get the total ingestion capacity on the overlord.
- Remove method `isK8sTaskRunner` from interface `TaskRunner`
* Using Map to perform comparison
* Minor Change
---------
Co-authored-by: George Shiqi Wu <george.wu@imply.io>
Saw bug where MSQ controller task would continue to hold the task slot even after cancel was issued.
This was due to a deadlock created on work launch. The main thread was waiting for tasks to spawn and the cancel thread was waiting for tasks to finish.
The fix was to instruct the MSQWorkerTaskLauncher thread to stop creating new tasks which would enable the main thread to unblock and release the slot.
Also short circuited the taskRetriable condition. Now the check is run in the MSQWorkerTaskLauncher thread as opposed to the main event thread loop. This will result in faster task failure in case the task is deemed to be non retriable.
* MSQ generates tombstones honoring the query's granularity.
This change tweaks to only account for the infinite-interval tombstones.
For finite-interval tombstones, the MSQ query granualrity will be used
which is consistent with how MSQ works.
* more tests and some cleanup.
* checkstyle
* comment edits
* Throw TooManyBuckets fault based on review; add more tests.
* Add javadocs for both methods on reconciling the methods.
* review: Move testReplaceTombstonesWithTooManyBucketsThrowsException to MsqFaultsTest
* remove unused imports.
* Move TooManyBucketsException to indexing package for shared exception handling.
* lower max bucket for tests and fixup count
* Advance and count the iterator.
* checkstyle
* + Fix for Flaky Test
* + Replacing TreeMap with LinkedHashMap
* + Changing data structure from LinkedHashMap to HashMap
* Fixed flaky test in S3DataSegmentPusherConfigTest.testSerializationValidatingMaxListingLength
* Minor Changes
In the current design, brokers query both data nodes and tasks to fetch the schema of the segments they serve. The table schema is then constructed by combining the schemas of all segments within a datasource. However, this approach leads to a high number of segment metadata queries during broker startup, resulting in slow startup times and various issues outlined in the design proposal.
To address these challenges, we propose centralizing the table schema management process within the coordinator. This change is the first step in that direction. In the new arrangement, the coordinator will take on the responsibility of querying both data nodes and tasks to fetch segment schema and subsequently building the table schema. Brokers will now simply query the Coordinator to fetch table schema. Importantly, brokers will still retain the capability to build table schemas if the need arises, ensuring both flexibility and resilience.
* Use filters for pruning properly for hash-joins.
Native used them too aggressively: it might use filters for the RHS
to prune the LHS. MSQ used them not at all. Now, both use them properly,
pruning based on base (LHS) columns only.
* Fix tests.
* Fix style.
* Clear filterFields too.
* Update.
* Add system fields to input sources.
Main changes:
1) The SystemField enum defines system fields "__file_uri", "__file_path",
and "__file_bucket". They are associated with each input entity.
2) The SystemFieldInputSource interface can be added to any InputSource
to make it system-field-capable. It sets up serialization of a list
of configured "systemFields" in the JSON form of the input source, and
provides a method getSystemFieldValue for computing the value of each
system field. Cloud object, HDFS, HTTP, and Local now have this.
* Fix various LocalInputSource calls.
* Fix style stuff.
* Fixups.
* Fix tests and coverage.
* better documentation for the differences between arrays and mvds
* add outputType to ExpressionPostAggregator to make docs true
* add output coercion if outputType is defined on ExpressionPostAgg
* updated post-aggregations.md to be consistent with aggregations.md and filters.md and use tables
While running queries on real time tasks using MSQ, there is an issue with queries with certain order by columns.
If the query specifies a non time column, the query is planned as it is supported by MSQ. However, this throws an exception when passed to real time tasks once as the native query stack does not support it. This PR resolves this by removing the ordering from the query before contacting real time tasks.
Fixes a bug with MSQ while reading data from real time tasks with non time ordering
ServiceClientImpl logs the cause of every retry, even though we are retrying the connection attempt. This leads to slight pollution in the logs because a lot of the time, the reason for retrying is the same. This is seen primarily in MSQ, when the worker task hasn't launched yet however controller attempts to connect to the worker task, which can lead to scary-looking messages (with INFO log level), even though they are normal.
This PR changes the logging logic to log every 10 (arbitrary number) retries instead of every retry, to reduce the pollution of the logs.
Note: If there are no retries left, the client returns an exception, which would get thrown up by the caller, and therefore this change doesn't hide any important information.
Functions that accept literals also allow casted literals. This shouldn't have an impact on the queries that the user writes. It enables the SQL functions to accept explicit cast, which is required with JDBC.
* Update S3 retry logic based on the underlying cause in case of IOException.
4xx and other errors wrapped in IOException for instance aren't retriable.
* Fix CI
This PR:
adds a flag to JsonToParquet to do the fix during conversion
updates the json files to more correct conents
some resultset mismatches were fixed by this
updates parquet to 1.13.1
Patch adds an undocumented parameter taskLockType to MSQ so that we can start enabling this feature for users who are interested in testing the new lock types.
This PR addresses a bug with waiting for segments to be loaded. In the case of append, segments would be created with the same version. This caused the number of segments returned to be incorrect.
This PR changes this to keep track of the range of partition numbers as well for each version, which lets the task wait for the correct set of segments. The partition numbers are expected to be continuous since the task obtains the lock for the segment while running.
Adding the ability to limit the pages sizes of select queries.
We piggyback on the same machinery that is used to control the numRowsPerSegment.
This patch introduces a new context parameter rowsPerPage for which the default value is set to 100000 rows.
This patch also optimizes adding the last selectResults stage only when the previous stages have sorted outputs. Currently for each select query with selectDestination=durableStorage, we used to add this extra selectResults stage.
* sql compatible tri-state native logical filters when druid.expressions.useStrictBooleans=true and druid.generic.useDefaultValueForNull=false, and new druid.generic.useThreeValueLogicForNativeFilters=true
* log.warn if non-default configurations are used to guide operators towards SQL complaint behavior
* fixes
* check for latest rewrite place
* Revert "check for latest rewrite place"
This reverts commit 5cf1e2c1ca.
* some stuff
(cherry picked from commit ab346d4373ea888eb8ef6115e018e7fb0d27407f)
* update test output
* updates to test ouptuts
* some stuff
* move validator
* cleanup
* fix
* change test slightly
* add apidoc cleanup warnings
* cleanup/etc
* instead of telling the story; add a fail with some reason whats the issue
* lead-lag fix
* add test
* remove unnecessary throw
* druidexception-trial
* Revert "druidexception-trial"
This reverts commit 8fa06644bc.
* undo changes to no_grouping; add no_grouping2
* add missing assert on resultcount
* rename method; update
* introduce enum/etc
* make resultmatchmode accessible from TestBuilder#expectedResults
* fix dump results to use log
* fix
* handle null correctly
* disable feature type based things for MSQ
* fix varianssqlaggtest
* use eps in other test
* fix intellij error
* add final
* addrss review
* update test/string/etc
* write concat in 3 lines :D
MSQ uses the string dimension schema for ARRAY<STRING> typed columns, which creates MVDs instead of string arrays as required. Therefore someone trying to ingest columns of type ARRAY<STRING> from an external data source or another data source would get STRING columns in the newly generated segments.
This patch changes the following:
- Use auto dimension schema to ingest the ARRAY<STRING> columns, which will create columns with the desired type.
- Add an undocumented flag ingestStringArraysAsMVDs to preserve the legacy behavior. Legacy behaviour is turned on by default.
- Create MSQArraysInsertTest and refactor some of the tests in MSQInsertTest.
* add a bunch of tests with array typed columns to CalciteArraysQueryTest
* fix a bug with unnest filter pushdown when filtering on unnested array columns
This PR aims to add the capabilities to:
1. Fetch the realtime segment metadata from the coordinator server view,
2. Adds the ability for workers to query indexers, similar to how brokers do the same for native queries.
Instead of passing the constants around in a new parameter; InputAccessor was introduced to take care of transparently handling the constants - this new class started picking up some copy-paste debris around field accesses; and made them a little bit more readble.
The sql standard is not very restrictive regarding this:
If AVG is specified and DT is exact numeric, then the declared type of the result is an implemen-
tation-defined exact numeric type with precision not less than the precision of DT and scale not
less than the scale of DT.
so; using the same type is also ok (without patch);
however the avg of 0 and 1 is 0 right now because of the retention of the integer typ
Postgres,MySql and Oracle and Drill seem to increase precision ; mssql returns 0
http://sqlfiddle.com/#!9/6f7248/1
I think we should also increase precision as its already calculated more precisely
Add segmentLoadWait as a query context parameter. If this is true, the controller queries the broker and waits till the segments created (if any) have been loaded by the load rules. The controller also provides this information in the live reports and task reports. If this is false, the controller exits immediately after finishing the query.
Row-based frames, and by extension, MSQ now supports numeric array types. This means that all queries consuming or producing arrays would also work with MSQ. Numeric arrays can also be ingested via MSQ. Post this patch, queries like, SELECT [1, 2] would work with MSQ since they consume a numeric array, instead of failing with an unsupported column type exception.
This patch introduces "processor managers" to processor factories, as a replacement for the sequence of processors. Processor managers can use the results of earlier processors to influence the creation of later processors, which provides us with the building block we need to ensure that broadcast join data is only read once.
In particular, when broadcast join is happening, the BaseFrameProcessorFactory now uses a ChainedProcessorManager to first run BroadcastJoinSegmentMapFnProcessor (in a single thread), and then run all of the regular processors (possibly multithreaded).
This change updates dependencies as needed and fixes tests to remove code incompatible with Java 21
As a result all unit tests now pass with Java 21.
* update maven-shade-plugin to 3.5.0 and follow-up to #15042
* explain why we need to override configuration when specifying outputFile
* remove configuration from dependency management in favor of explicit overrides in each module.
* update to mockito to 5.5.0 for Java 21 support when running with Java 11+
* continue using latest mockito 4.x (4.11.0) when running with Java 8
* remove need to mock private fields
* exclude incorrectly declared mockito dependency from pac4j-oidc
* remove mocking of ByteBuffer, since sealed classes can no longer be mocked in Java 21
* add JVM options workaround for system-rules junit plugin not supporting Java 18+
* exclude older versions of byte-buddy from assertj-core
* fix for Java 19 changes in floating point string representation
* fix missing InitializedNullHandlingTest
* update easymock to 5.2.0 for Java 21 compatibility
* update animal-sniffer-plugin to 1.23
* update nl.jqno.equalsverifier to 3.15.1
* update exec-maven-plugin to 3.1.0
This change is meant to fix a issue where passing too large of a task payload to the mm-less task runner will cause the peon to fail to startup because the payload is passed (compressed) as a environment variable (TASK_JSON). In linux systems the limit for a environment variable is commonly 128KB, for windows systems less than this. Setting a env variable longer than this results in a bunch of "Argument list too long" errors.
Upgrade maven shade plugin to try to fix build failures
Sometimes we get maven shade errors in our integ tests becasue we don't run clean in between runs to clear the cache in order to speed them up. This can lead to the below error.
Error: Failed to execute goal org.apache.maven.plugins:maven-shade-plugin:3.2.4:shade (opentelemetry-extension) on project opentelemetry-emitter: Error creating shaded jar: duplicate entry: META-INF/services/org.apache.druid.opentelemetry.shaded.io.grpc.NameResolverProvider
See: https://issues.apache.org/jira/projects/MSHADE/issues/MSHADE-425?filter=allissues
An example run that failed: https://github.com/apache/druid/actions/runs/6301662092/job/17117142375?pr=14887
According to the ticket this is fixed by updating shade to 3.4.1.
When I updated to 3.4.1 I kept running into a different issue during static checks. (Caused by: java.lang.NoClassDefFoundError: com/github/rvesse/airline/parser/errors/ParseException)
I had to add the createDependencyReducedPom: false to get the build to pass.
The dependency reduced pom feature was added in 3.3.0 which we were not using before so setting it explicitly to false should not be a issue. https://issues.apache.org/jira/browse/MSHADE-36)
The aggregators had incorrect types for getResultType when shouldFinalze
is false. They had the finalized type, but they should have had the
intermediate type.
Also includes a refactor of how ExprMacroTable is handled in tests, to make
it easier to add tests for this to the MSQ module. The bug was originally
noticed because the incorrect result types caused MSQ queries with DS_HLL
to behave erratically.
* Remove stale comment since we're on avro version 1.11.1
* Update exception blocks. With 1.11.1, read() only throws IOException.
* Unit tests
* Cleanup and add more tests.
This entails:
Removing the enableUnnest flag and additional machinery
Updating the datasource plan and frame processors to support unnest
Adding support in MSQ for UnnestDataSource and FilteredDataSource
CalciteArrayTest now has a MSQ test component
Additional tests for Unnest on MSQ
With PR #14322 , MSQ insert/Replace q's will wait for segment to be loaded on the historical's before finishing.
The patch introduces a bug where in the main thread had a thread.sleep() which could not be interrupted via the cancel calls from the overlord.
This new patch addressed that problem by moving the thread.sleep inside a thread of its own. Thus the main thread is now waiting on the future object of this execution.
The cancel call can now shutdown the executor service via another method thus unblocking the main thread to proceed.
This commit pulls out some changes from #14407 to simplify that PR.
Changes:
- Rename `IndexerMetadataStorageCoordinator.announceHistoricalSegments` to `commitSegments`
- Rename the overloaded method to `commitSegmentsAndMetadata`
- Fix some typos
Currently, only the user who has submitted the async query has permission to interact with the status APIs for that async query. However, often we want an administrator to interact with these resources as well.
Druid handles these with the STATE resource traditionally, and if the requesting user has necessary permissions on it as well, alternatively, they should be allowed to interact with the status APIs, irrespective of whether they are the submitter of the query.
* Add IS [NOT] DISTINCT FROM to SQL and join matchers.
Changes:
1) Add "isdistinctfrom" and "notdistinctfrom" native expressions.
2) Add "IS [NOT] DISTINCT FROM" to SQL. It uses the new native expressions
when generating expressions, and is treated the same as equals and
not-equals when generating native filters on literals.
3) Update join matchers to have an "includeNull" parameter that determines
whether we are operating in "equals" mode or "is not distinct from"
mode.
* Main changes:
- Add ARRAY handling to "notdistinctfrom" and "isdistinctfrom".
- Include null in pushed-down filters when using "notdistinctfrom" in a join.
Other changes:
- Adjust join filter analyzer to more explicitly use InDimFilter's ValuesSets,
relying less on remembering to get it right to avoid copies.
* Remove unused "wrap" method.
* Fixes.
* Remove methods we do not need.
* Fix bug with INPUT_REF.
* SQL: Plan non-equijoin conditions as cross join followed by filter.
Druid has previously refused to execute joins with non-equality-based
conditions. This was well-intentioned: the idea was to push people to
write their queries in a different, hopefully more performant way.
But as we're moving towards fuller SQL support, it makes more sense to
allow these conditions to go through with the best plan we can come up
with: a cross join followed by a filter. In some cases this will allow
the query to run, and people will be happy with that. In other cases,
it will run into resource limits during execution. But we should at
least give the query a chance.
This patch also updates the documentation to explain how people can
tell whether their queries are being planned this way.
* cartesian is a word.
* Adjust tests.
* Update docs/querying/datasource.md
Co-authored-by: Benedict Jin <asdf2014@apache.org>
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Currently, after an MSQ query, the web console is responsible for waiting for the segments to load. It does so by checking if there are any segments loading into the datasource ingested into, which can cause some issues, like in cases where the segments would never be loaded, or would end up waiting for other ingests as well.
This PR shifts this responsibility to the controller, which would have the list of segments created.
Changes:
- Make ServiceMetricEvent.Builder extend ServiceEventBuilder<ServiceMetricEvent>
and thus convert it to a plain builder rather than a builder of builder.
- Add methods setCreatedTime , setMetricAndValue to the builder
There is a current issue due to inconsistent metadata between worker and controller in MSQ. A controller can receive one set of segments, which are then marked as unused by, say, a compaction job. The worker would be unable to get the segment information as MetadataResource.
Currently, Druid is using Guava 16.0.1 version. This upgrade to 31.1-jre fixes the following issues.
CVE-2018-10237 (Unbounded memory allocation in Google Guava 11.0 through 24.x before 24.1.1 allows remote attackers to conduct denial of service attacks against servers that depend on this library and deserialize attacker-provided data because the AtomicDoubleArray class (when serialized with Java serialization) and the CompoundOrdering class (when serialized with GWT serialization) perform eager allocation without appropriate checks on what a client has sent and whether the data size is reasonable). We don't use Java or GWT serializations. Despite being false positive they're causing red security scans on Druid distribution.
Latest version of google-client-api is incompatible with the existing Guava version. This PR unblocks Update google client apis to latest version #14414
This PR adds a way to store the topic name in a column. Such a column can be used to distinguish messages coming from different topics in multi-topic ingestion.
Motivation:
- There is no usage of the `SegmentTransactionInsertAction` which passes a
non-null non-empty value of `segmentsToBeDropped`.
- This is not really needed either as overshadowed segments are marked as unused
by the Coordinator and need not be done in the same transaction as committing segments.
- It will also help simplify the changes being made in #14407
Changes:
- Remove `segmentsToBeDropped` from the task action and all intermediate methods
- Remove related tests which are not needed anymore
* Add supervisor /resetOffsets API.
- Add a new endpoint /druid/indexer/v1/supervisor/<supervisorId>/resetOffsets
which accepts DataSourceMetadata as a body parameter.
- Update logs, unit tests and docs.
* Add a new interface method for backwards compatibility.
* Rename
* Adjust tests and javadocs.
* Use CoreInjectorBuilder instead of deprecated makeInjectorWithModules
* UT fix
* Doc updates.
* remove extraneous debugging logs.
* Remove the boolean setting; only ResetHandle() and resetInternal()
* Relax constraints and add a new ResetOffsetsNotice; cleanup old logic.
* A separate ResetOffsetsNotice and some cleanup.
* Minor cleanup
* Add a check & test to verify that sequence numbers are only of type SeekableStreamEndSequenceNumbers
* Add unit tests for the no op implementations for test coverage
* CodeQL fix
* checkstyle from merge conflict
* Doc changes
* DOCUSAURUS code tabs fix. Thanks, Brian!
In this PR, I have gotten rid of multiTopic parameter and instead added a topicPattern parameter. Kafka supervisor will pass topicPattern or topic as the stream name to the core ingestion engine. There is validation to ensure that only one of topic or topicPattern will be set. This new setting is easier to understand than overloading the topic field that earlier could be interpreted differently depending on the value of some other field.
This PR adds support to read from multiple Kafka topics in the same supervisor. A multi-topic ingestion can be useful in scenarios where a cluster admin has no control over input streams. Different teams in an org may create different input topics that they can write the data to. However, the cluster admin wants all this data to be queryable in one data source.
* Update to Calcite 1.35.0
* Update from.ftl for Calcite 1.35.0.
* Fixed tests in Calcite upgrade by doing the following:
1. Added a new rule, CoreRules.PROJECT_FILTER_TRANSPOSE_WHOLE_PROJECT_EXPRESSIONS, to Base rules
2. Refactored the CorrelateUnnestRule
3. Updated CorrelateUnnestRel accordingly
4. Fixed a case with selector filters on the left where Calcite was eliding the virtual column
5. Additional test cases for fixes in 2,3,4
6. Update to StringListAggregator to fail a query if separators are not propagated appropriately
* Refactored for testcases to pass after the upgrade, introduced 2 new data sources for handling filters and select projects
* Added a literalSqlAggregator as the upgraded Calcite involved changes to subquery remove rule. This corrected plans for 2 queries with joins and subqueries by replacing an useless literal dimension with a post agg. Additionally a test with COUNT DISTINCT and FILTER which was failing with Calcite 1.21 is added here which passes with 1.35
* Updated to latest avatica and updated code as SqlUnknownTimeStamp is now used in Calcite which needs to be resolved to a timestamp literal
* Added a wrapper segment ref to use for unnest and filter segment reference
The Azure connector is introduced and MSQ's fault tolerance and durable storage can now be used with Microsoft Azure's blob storage. Also, the results of newly introduced queries from deep storage can now store and fetch the results from Azure's blob storage.
* Rolling supervior task publishing
* add an option for number of task groups to roll over
* better
* remove docs
* oops
* checkstyle
* wip test
* undo partial test change
* remove incomplete test
* 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.
* MSQ WorkerImpl: Ignore ServiceClosedException on postCounters.
A race can happen where postCounters is in flight while the controller
goes offline. When this happens, we should ignore the ServiceClosedException
and continue without posting counters.
* Fix style and logic.
* Remove chatAsync parameter, so chat is always async.
chatAsync has been made default in Druid 26. I have seen good
battle-testing of it in production, and am comfortable removing the
older sync client.
This was the last remaining usage of IndexTaskClient, so this patch
deletes all that stuff too.
* Remove unthrown exception.
* Remove unthrown exception.
* No more TimeoutException.
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.
* 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.
* 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
* Merge core CoordinatorClient with MSQ CoordinatorServiceClient.
Continuing the work from #12696, this patch merges the MSQ
CoordinatorServiceClient into the core CoordinatorClient, yielding a single
interface that serves both needs and is based on the ServiceClient RPC
system rather than DruidLeaderClient.
Also removes the backwards-compatibility code for the handoff API in
CoordinatorBasedSegmentHandoffNotifier, because the new API was added
in 0.14.0. That's long enough ago that we don't need backwards
compatibility for rolling updates.
* Fixups.
* Trigger GHA.
* Remove unnecessary retrying in DruidInputSource. Add "about an hour"
retry policy and h
* EasyMock
* 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>
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.
* Change default handoffConditionTimeout to 15 minutes.
Most of the time, when handoff is taking this long, it's because something
is preventing Historicals from loading new data. In this case, we have
two choices:
1) Stop making progress on ingestion, wait for Historicals to load stuff,
and keep the waiting-for-handoff segments available on realtime tasks.
(handoffConditionTimeout = 0, the current default)
2) Continue making progress on ingestion, by exiting the realtime tasks
that were waiting for handoff. Once the Historicals get their act
together, the segments will be loaded, as they are still there on
deep storage. They will just not be continuously available.
(handoffConditionTimeout > 0)
I believe most users would prefer [2], because [1] risks ingestion falling
behind the stream, which causes many other problems. It can cause data loss
if the stream ages-out data before we have a chance to ingest it.
Due to the way tuningConfigs are serialized -- defaults are baked into the
serialized form that is written to the database -- this default change will
not change anyone's existing supervisors. It will take effect for newly
created supervisors.
* Fix tests.
* Update docs/development/extensions-core/kafka-supervisor-reference.md
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* Update docs/development/extensions-core/kinesis-ingestion.md
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
---------
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
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.
The errorCode of this fault when serialized over the wire was being
set to the name of the class `InsertTimeOutOfBoundsFault` instead of
the CODE `InsertTimeOutOfBounds`. All other faults' errorCodes are
serialized as the respective Fault's code, so making consistent here
as well.
* Fixing an issue in sequential merge where workers without any partial key statistics would get stuck because controller did not change the worker state.
* Removing empty check
* Adding IT for MSQ sequential bug fix.
* 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.
sqlJoinAlgorithm is now a hint to the planner to execute the join in the specified manner. The planner can decide to ignore the hint if it deduces that the specified algorithm can be detrimental to the performance of the join beforehand.
* Refactor HllSketchBuildAggregatorFactory
The usage of ColumnProcessors and HllSketchBuildColumnProcessorFactory
made it very difficult to figure out what was going on from just looking
at the AggregatorFactory or Aggregator code. It also didn't properly
double check that you could use UTF8 ahead of time, even though it's
entirely possible to validate it before trying to use it. This refactor
makes keeps the general indirection that had been implemented by
the Consumer<Supplier<HllSketch>> but centralizes the decision logic and
makes it easier to understand the code.
* Test fixes
* Add test that validates the types are maintained
* Add back indirection to avoid buffer calls
* Cover floats and doubles are the same thing
* Static checks
* 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.
Apache Druid brings multiple direct and transitive dependencies that are affected by plethora of CVEs.
This PR attempts to update all the dependencies that did not require code refactoring.
This PR modifies pom files, license file and OWASP Dependency Check suppression file.
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.
* Changes the get results API in SqlStatementResource to take a page number instead of row/offset.
* Adds "pages" containing information on each page to the results status.
* Update the "numRows" and "sizeInByes" to "numTotalRows" and "totalSizeInBytes" respectively, which are totalled across all pages.
* 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
1) Fix a problem where the fault wasn't reported when the left-hand side
had too many buffered frames. (Instead, frames continued to be buffered,
eventually running the server out of memory.)
2) Always update the mark when rewinding isn't necessary. It fixes a problem where
frames would be needlessly buffered when there isn't a key match across
the two sides.
3) Memory reserved for building the trackers now change based on the heap sized
* 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.
Mocks generally have state and should not be static. In particular, the
"Yielder" included in one of the mocks can only be iterated once, which
made the test suite order-dependent.
* Support complex variance object inputs for variance SQL agg function
* Add test
* Include complexTypeChecker, address PR comments
* Checkstyle, javadoc link
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.
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.
Changes:
- Throw an `InsertCannotAllocateSegmentFault` if the allocated segment is not aligned with
the requested granularity.
- Tests to verify new behaviour
* MSQ: Change default clusterStatisticsMergeMode to SEQUENTIAL.
This is an undocumented parameter that controls how cluster-by statistics
are merged. In PARALLEL mode, statistics are gathered from workers all
at once. In SEQUENTIAL mode, statistics are gathered time chunk by time
chunk. This improves accuracy for jobs with many time chunks, and reduces
memory usage.
The main downside of SEQUENTIAL is that it can take longer, but in most
situations I've seen, PARALLEL is only really usable in cases where the
sketches are small enough that SEQUENTIAL would also run relatively
quickly. So it seems like SEQUENTIAL is a better default.
* Switch off-test from SEQUENTIAL to PARALLEL.
* Fix sequential merge for situations where there are no time chunks at all.
* Add a couple more tests.
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.
* SQL OperatorConversions: Introduce.aggregatorBuilder, allow CAST-as-literal.
Four main changes:
1) Provide aggregatorBuilder, a more consistent way of defining the
SqlAggFunction we need for all of our SQL aggregators. The mechanism
is analogous to the one we already use for SQL functions
(OperatorConversions.operatorBuilder).
2) Allow CASTs of constants to be considered as "literalOperands". This
fixes an issue where various of our operators are defined with
OperandTypes.LITERAL as part of their checkers, which doesn't allow
casts. However, in these cases we generally _do_ want to allow casts.
The important piece is that the value must be reducible to a constant,
not that the SQL text is literally a literal.
3) Update DataSketches SQL aggregators to use the new aggregatorBuilder
functionality. The main user-visible effect here is [2]: the aggregators
would now accept, for example, "CAST(0.99 AS DOUBLE)" as a literal
argument. Other aggregators could be updated in a future patch.
4) Rename "requiredOperands" to "requiredOperandCount", because the
old name was confusing. (It rhymes with "literalOperands" but the
arguments mean different things.)
* Adjust method calls.
* S3: Attach SSE key to doesObjectExist calls.
We did not previously attach the SSE key to the doesObjectExist request,
leading to an inconsistency that may cause problems on "S3-compatible"
implementations. This patch implements doesObjectExist using similar
logic to the S3 client itself, but calls our implementation of
getObjectMetadata rather than the S3 client's, ensuring the request
is decorated with the SSE key.
* Fix tests.
* Fix compatibility issue with SqlTaskResource
The DruidException changes broke the response format
for errors coming back from the SqlTaskResource, so fix those
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.
* Throw ValidationException if CLUSTERED BY column descending order is specified.
- Fails query planning
* Some more tests.
* fixup existing comment
* Update comment
* checkstyle fix: remove unused imports
* Remove InsertCannotOrderByDescendingFault and deprecate the fault in readme.
* move deprecated field to the bottom
They were not previously loaded because supportsQueries was false.
This patch sets supportsQueries to true, and clarifies in Task
javadocs that supportsQueries can be true for tasks that aren't
directly queryable over HTTP.
* Limit select results in MSQ
* reduce number of files in test
* add truncated flag
* avoid materializing select results to list, use iterable instead
* javadocs
* fix kafka input format reader schema discovery and partial schema discovery to actually work right, by re-using dimension filtering logic of MapInputRowParser
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
* Fix EarliestLatestBySqlAggregator signature; Include function name for all signatures.
* Single quote function signatures, space between args and remove \n.
* fixup UT assertion
The same aggregator can have two output names for a SQL like:
INSERT INTO foo
SELECT x, COUNT(*) AS y, COUNT(*) AS z
FROM t
GROUP BY 1
PARTITIONED BY ALL
In this case, the SQL planner will create a query with a single "count"
aggregator mapped to output names "y" and "z". The prior MSQ code did
not properly handle this case, instead throwing an error like:
Expected single output for query column[a0] but got [[1, 2]]
It was found that several supported tasks / input sources did not have implementations for the methods used by the input source security feature, causing these tasks and input sources to fail when used with this feature. This pr adds the needed missing implementations. Also securing the sampling endpoint with input source security, when enabled.
### 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
* 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
In StreamChunkParser#parseWithInputFormat, we call byteEntityReader.read() without handling a potential ParseException, which is thrown during this function call by the delegate AvroStreamReader#intermediateRowIterator.
A ParseException can be thrown if an Avro stream has corrupt data or data that doesn't conform to the schema specified or for other decoding reasons. This exception if uncaught, can cause ingestion to fail.
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
* If a worker dies after it has finished generating results, MSQ decides to not retry it as it has no active work orders. However, since we don't keep track of it further, if it is required for a future stage, the controller hangs waiting for the worker to be ready. This PR keeps tracks of any workers the controller decides to not restart immediately and while starting workers for the next stage, queues these workers for retry.
* fix typo in s3 docs. add readme to s3 module.
* Update extensions-core/s3-extensions/README.md
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
* cleanup readme for s3 extension and link to repo markdown doc instead of web docs
---------
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
Hadoop 2 often causes red security scans on Druid distribution because of the dependencies it brings. We want to move away from Hadoop 2 and provide Hadoop 3 distribution available. Switch druid to building with Hadoop 3 by default. Druid will still be compatible with Hadoop 2 and users can build hadoop-2 compatible distribution using hadoop2 profile.
* 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.
* 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.
When ingesting from an external source which already contains a column "__time", currently, the value is dropped. Changing the time column name in the external input slice reader resolves this.
* 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
### Description
This pr fixes a few bugs found with the inputSource security feature.
1. `KillUnusedSegmentsTask` previously had no definition for the `getInputSourceResources`, which caused an unsupportedOperationException to be thrown when this task type was submitted with the inputSource security feature enabled. This task type should not require any input source specific resources, so returning an empty set for this task type now.
2. Fixed a bug where when the input source type security feature is enabled, all of the input source type specific resources used where authenticated against:
`{"resource": {"name": "EXTERNAL", "type": "{INPUT_SOURCE_TYPE}"}, "action": "READ"}`
When they should be instead authenticated against:
`{"resource": {"name": "{INPUT_SOURCE_TYPE}", "type": "EXTERNAL"}, "action": "READ"}`
3. fixed bug where supervisor tasks were not authenticated against the specific input source types used, if input source security feature was enabled.
*
Adds new run time parameter druid.indexer.task.tmpStorageBytesPerTask. This sets a limit for the amount of temporary storage disk space used by tasks. This limit is currently only respected by MSQ tasks.
* Removes query context parameters intermediateSuperSorterStorageMaxLocalBytes and composedIntermediateSuperSorterStorageEnabled. Composed intermediate super sorter (which was enabled by composedIntermediateSuperSorterStorageEnabled) is now enabled automatically if durableShuffleStorage is set to true. intermediateSuperSorterStorageMaxLocalBytes is calculated from the limit set by the run time parameter druid.indexer.task.tmpStorageBytesPerTask.
* "maxResultsSize" has been removed from the S3OutputConfig and a default "chunkSize" of 100MiB is now present. This change primarily affects users who wish to use durable storage for MSQ jobs.
* MSQ: Use the same result coercion routines as the regular SQL endpoint.
The main changes are to move NativeQueryMaker.coerce to SqlResults, and
to formally make the list of sqlTypeNames from the MSQ results reports
use SqlTypeNames.
- Change the default to MSQ-compatible rather than MSQ-incompatible.
The explicit marker function is now "notMsqCompatible()".
* 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.
The GCP initialization pulls credentials for
talking to GCP. We want that to only happen
when fully required and thus want the GCP-related
objects lazily instantiated.
* 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.
### Description
Previously msq controller and worker tasks did not have implementations for the `getInputSourceResources()` method. This causes the submission of these tasks to fail if the following auth config is enabled:
`druid.auth.enableInputSourceSecurity=true`
Added implementations of this method for these tasks that return an empty set of input sources. This means that for these task types, if `druid.auth.enableInputSourceSecurity=true` config is used, the input source types will be properly computed and authorized in the SQL layer, but not if the equivalent controller / worker tasks are submitted to the task endpoint.
### Description
This change allows for input sources used during MSQ ingestion to be authorized for multiple input source types, instead of just 1. Such an input source that allows for multiple types is the CombiningInputSource.
Also fixed bug that caused some input source specific functions to be authorized against the permissions
`
[
new ResourceAction(new Resource(ResourceType.EXTERNAL, ResourceType.EXTERNAL), Action.READ),
new ResourceAction(new Resource(ResourceType.EXTERNAL, {input_source_type}), Action.READ)
]
`
when the inputSource based authorization feature is enabled, when it should instead be authorized against
`
[
new ResourceAction(new Resource(ResourceType.EXTERNAL, {input_source_type}), Action.READ)
]
`
* 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.
* 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.
* Add a new fault "QueryRuntimeError" to MSQ engine to capture native query errors.
* Fixed bug in MSQ fault tolerance where worker were being retried if `UnexpectedMultiValueDimensionException` was thrown.
* An exception from the query runtime with `org.apache.druid.query` as the package name is thrown as a QueryRuntimeError
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!)
This change introduces the concept of input source type security model, proposed in #13837.. With this change, this feature is only available at the SQL layer, but we will expand to native layer in a follow up PR.
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 having the usage all external sources being authorized against the hardcoded resource action
new ResourceAction(new Resource(ResourceType.EXTERNAL, ResourceType.EXTERNAL), Action.READ
When this config is enabled, the users will be required to be authorized for the following resource action
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..
Documentation has not been added for the feature as it is not complete at the moment, as we still need to enable this for the native layer in a follow up pr.
This PR is a follow-up to #13819 so that the Tuple sketch functionality can be used in SQL for both ingestion using Multi-Stage Queries (MSQ) and also for analytic queries against Tuple sketch columns.
* 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
Expands the OIDC based auth in Druid by adding a JWT Authenticator that validates ID Tokens associated with a request. The existing pac4j authenticator works for authenticating web users while accessing the console, whereas this authenticator is for validating Druid API requests made by Direct clients. Services already supporting OIDC can attach their ID tokens to the Druid requests
under the Authorization request header.
* Add segment generator counters to reports
* Remove unneeded annotation
* Fix checkstyle and coverage
* Add persist and merged as new metrics
* Address review comments
* Fix checkstyle
* Create metrics class to handle updating counters
* Address review comments
* Add rowsPushed as a new metrics
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
* Lower default maxRowsInMemory for realtime ingestion.
The thinking here is that for best ingestion throughput, we want
intermediate persists to be as big as possible without using up all
available memory. So, we rely mainly on maxBytesInMemory. The default
maxRowsInMemory (1 million) is really just a safety: in case we have
a large number of very small rows, we don't want to get overwhelmed
by per-row overheads.
However, maximum ingestion throughput isn't necessarily the primary
goal for realtime ingestion. Query performance is also important. And
because query performance is not as good on the in-memory dataset, it's
helpful to keep it from growing too large. 150k seems like a reasonable
balance here. It means that for a typical 5 million row segment, we
won't trigger more than 33 persists due to this limit, which is a
reasonable number of persists.
* Update tests.
* Update server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Fix test.
* Fix link.
---------
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Removing intermediateSuperSorterStorageMaxLocalBytes, maxInputBytesPerWorker, composedIntermediateSuperSorterStorageEnabled, clusterStatisticsMergeMode from docs
* Adding documentation in the context class.
* 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.
* 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.