* enable quidem uri support for `druidtest:///?ComponentSupplier=Nested` and similar
* changes the way `SqlTestFrameworkConfig` is being applied; all options will have their own annotation (its kinda impossible to detect that an annotation has a set value or its the default)
* enables hierarchical processing of config annotation (was needed to enable class level supplier annotation)
* moves uri processing related string2config stuff into `SqlTestFrameworkConfig`
With this PR changes, MSQ tasks (MSQControllerTask and MSQWorkerTask) only load the required lookups during querying and ingestion, based on the value of CTX_LOOKUPS_TO_LOAD key in the query context.
Add validation for reindex with realtime sources.
With the addition of concurrent compaction, it is possible to ingest data while querying from realtime sources with MSQ into the same datasource. This could potentially lead to issues if the interval that is ingested into is replaced by an MSQ job, which has queried only some of the data from the realtime task.
This PR adds validation to check that the datasource being ingested into is not being queried from, if the query includes realtime sources.
* * add another catalog clustering columns unit test
* * dissallow clusterKeys with descending order
* * make more clear that clustering is re-written into ingest node
whether a catalog table or not
* * when partitionedBy is stored in catalog, user shouldnt need to specify
it in order to specify clustering
* * fix intellij inspection failure
update dependencies to address new batch of CVEs:
- Azure POM from 1.2.19 to 1.2.23 to update transitive dependency nimbus-jose-jwt to address: CVE-2023-52428
- commons-configuration2 from 2.8.0 to 2.10.1 to address: CVE-2024-29131 CVE-2024-29133
- bcpkix-jdk18on from 1.76 to 1.78.1 to address: CVE-2024-30172 CVE-2024-30171 CVE-2024-29857
* MSQ controller: Support in-memory shuffles; towards JVM reuse.
This patch contains two controller changes that make progress towards a
lower-latency MSQ.
First, support for in-memory shuffles. The main feature of in-memory shuffles,
as far as the controller is concerned, is that they are not fully buffered. That
means that whenever a producer stage uses in-memory output, its consumer must run
concurrently. The controller determines which stages run concurrently, and when
they start and stop.
"Leapfrogging" allows any chain of sort-based stages to use in-memory shuffles
even if we can only run two stages at once. For example, in a linear chain of
stages 0 -> 1 -> 2 where all do sort-based shuffles, we can use in-memory shuffling
for each one while only running two at once. (When stage 1 is done reading input
and about to start writing its output, we can stop 0 and start 2.)
1) New OutputChannelMode enum attached to WorkOrders that tells workers
whether stage output should be in memory (MEMORY), or use local or durable
storage.
2) New logic in the ControllerQueryKernel to determine which stages can use
in-memory shuffling (ControllerUtils#computeStageGroups) and to launch them
at the appropriate time (ControllerQueryKernel#createNewKernels).
3) New "doneReadingInput" method on Controller (passed down to the stage kernels)
which allows stages to transition to POST_READING even if they are not
gathering statistics. This is important because it enables "leapfrogging"
for HASH_LOCAL_SORT shuffles, and for GLOBAL_SORT shuffles with 1 partition.
4) Moved result-reading from ControllerContext#writeReports to new QueryListener
interface, which ControllerImpl feeds results to row-by-row while the query
is still running. Important so we can read query results from the final
stage using an in-memory channel.
5) New class ControllerQueryKernelConfig holds configs that control kernel
behavior (such as whether to pipeline, maximum number of concurrent stages,
etc). Generated by the ControllerContext.
Second, a refactor towards running workers in persistent JVMs that are able to
cache data across queries. This is helpful because I believe we'll want to reuse
JVMs and cached data for latency reasons.
1) Move creation of WorkerManager and TableInputSpecSlicer to the
ControllerContext, rather than ControllerImpl. This allows managing workers and
work assignment differently when JVMs are reusable.
2) Lift the Controller Jersey resource out from ControllerChatHandler to a
reusable resource.
3) Move memory introspection to a MemoryIntrospector interface, and introduce
ControllerMemoryParameters that uses it. This makes it easier to run MSQ in
process types other than Indexer and Peon.
Both of these areas will have follow-ups that make similar changes on the
worker side.
* Address static checks.
* Address static checks.
* Fixes.
* Report writer tests.
* Adjustments.
* Fix reports.
* Review updates.
* Adjust name.
* Small changes.
Changes:
- Add new config `lagAggregate` to `LagBasedAutoScalerConfig`
- Add field `aggregateForScaling` to `LagStats`
- Use the new field/config to determine which aggregate to use to compute lag
- Remove method `Supervisor.computeLagForAutoScaler()`
Changes:
1) Check for handoff of upgraded realtime segments.
2) Drop sink only when all associated realtime segments have been abandoned.
3) Delete pending segments upon commit to prevent unnecessary upgrades and
partition space exhaustion when a concurrent replace happens. This also prevents
potential data duplication.
4) Register pending segment upgrade only on those tasks to which the segment is associated.
I'm adding OIDC context to the AuthenticationResult returned by pac4j extension. I wanted to use this context as input in OpenPolicyAgent authorization. Since AuthenticationResult already accepts context as a parameter it felt okay to pass the profile attributes there.
Fixes a bug introduced in #16296, where the sketch might not be
initialized if get() is called without calling aggregate(). Also adds
a test for this case.
Issue: #14989
The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Thereafter, we addressed the problem of publishing schema for realtime segments (#15475). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information.
This is the final change which involves publishing segment schema for finalized segments from task and periodically polling them in the Coordinator.
Buffer aggregators can contain some cached objects within them, such as
Memory references or HLL Unions. Prior to this patch, various Grouper
implementations were not releasing this state when resetting their own
internal state, which could lead to excessive memory use.
This patch renames AggregatorAdapater#close to "reset", and updates
Grouper implementations to call this reset method whenever they reset
their internal state.
The base method on BufferAggregator and VectorAggregator remains named
"close", for compatibility with existing extensions, but the contract
is adjusted to say that the aggregator may be reused after the method
is called. All existing implementations in core already adhere to this
new contract, except for the ArrayOfDoubles build flavors, which are
updated in this patch to adhere.
Additionally, this patch harmonizes buffer sketch helpers to call their
clear method "clear" rather than a mix of "clear" and "close". (Others
were already using "clear".)
Tries to address the comments made on #16284 after merged.
Changes:
- Remove method `Supervisor.getLagMetric()`
- Add method `Supervisor.computeLagForAutoScaler()`
- Remove classes `LagMetric` and `LagMetricTest`
Changes:
- Add column `task_allocator_id` to `pendingSegments` metadata table.
- Add column `upgraded_from_segment_id` to `pendingSegments` metadata table.
- Add interface `PendingSegmentAllocatingTask` and implement it by all tasks which
can allocate pending segments.
- Use `taskAllocatorId` to identify the task (and its sub-tasks or replicas) to which
a pending segment has been allocated.
- Perform active cleanup of pending segments in `TaskLockbox` once there are no
active tasks for the corresponding task allocator id.
- When committing APPEND segments, also commit all upgraded pending segments
corresponding to that task allocator id.
- When committing REPLACE segments, upgrade all overlapping pending segments in
the same transaction.
Bug:
#15724 introduced a bug where a rolling upgrade would cause all task locations
returned by the Overlord on an older version to be unknown.
Fix:
If the new API fails, fall back to single task status API which always returns a valid task location.
Currently, export creates the files at the provided destination. The addition of the manifest file will provide a list of files created as part of the manifest. This will allow easier consumption of the data exported from Druid, especially for automated data pipelines
Follow up to #16217
Changes:
- Update `OverlordClient.getReportAsMap()` to return `TaskReport.ReportMap`
- Move the following classes to `org.apache.druid.indexer.report` in the `druid-processing` module
- `TaskReport`
- `KillTaskReport`
- `IngestionStatsAndErrorsTaskReport`
- `TaskContextReport`
- `TaskReportFileWriter`
- `SingleFileTaskReportFileWriter`
- `TaskReportSerdeTest`
- Remove `MsqOverlordResourceTestClient` as it had only one method
which is already present in `OverlordResourceTestClient` itself
Changes:
- Add `TaskContextEnricher` interface to improve task management and monitoring
- Invoke `enrichContext` in `TaskQueue.add()` whenever a new task is submitted to the Overlord
- Add `TaskContextReport` to write out task context information in reports
* SQL tests: avoid mixing skip and cannot vectorize.
skipVectorize switches off vectorization tests completely, and
cannotVectorize turns vectorization tests into negative tests. It doesn't
make sense to use them together, so this patch makes it an error to do so,
and cleans up cases where both are mentioned.
This patch also has the effect of changing various tests from skipVectorize
to cannotVectorize, because in the past when both were mentioned,
skipVectorize would take priority.
* Fix bug with StringAnyAggregatorFactory attempting to vectorize when it cannt.
* Fix tests.
Compaction in the native engine by default records the state of compaction for each segment in the lastCompactionState segment field. This PR adds support for doing the same in the MSQ engine, targeted for future cases such as REPLACE and compaction done via MSQ.
Note that this PR doesn't implicitly store the compaction state for MSQ replace tasks; it is stored with flag "storeCompactionState": true in the query context.
Current Runtime Exceptions generated while writing frames only include the exception itself without including the name of the column they were encountered in. This patch introduces the further information in the error and makes it non-retryable.
This PR logs the segment type and reason chosen. It also adds it to the query report, to be displayed in the UI.
This PR adds a new section to the reports, segmentReport. This contains the segment type created, if the query is an ingestion, and null otherwise.
* Reduce upload buffer size in GoogleTaskLogs.
Use a 1MB upload buffer, rather than the default of 15 MB in the API client. This is
mainly because MMs may upload logs in parallel, and typically have small heaps. The
default-sized 15 MB buffers add up quickly and can cause a MM to run out of memory.
* Make bufferSize a nullable Integer. Add tests.
Support for exporting msq results to gcs bucket. This is essentially copying the logic of s3 export for gs, originally done by @adarshsanjeev in this PR - #15689
Changes
- No functional changes
- Add method `AbstractBatchIndexTask.buildIngestionStatsReport()` used in several batch tasks
- Add utility method `AbstractBatchIndexTask.addBuildSegmentStatsToReport()`
- Use boolean argument to represent a full report instead of the String `full`
in internal methods. (REST API remains unchanged.)
- Rename `IngestionStatsAndErrorsTaskReportData` to `IngestionStatsAndErrors`
- Clean up some of the methods
This PR aims to introduce Window functions on MSQ by doing the following:
Introduce a Window querykit for handling window queries along with its factory and a processor for window queries
If a window operator is present with a partition by clause, pushes the partition as a shuffle spec of the previous stage
In presence of empty OVER() clause lets all operators loose on a single rac
In presence of no empty OVER() clause, breaks down each window into individual stages
Associated machinery to handle window functions in MSQ
Introduced a separate hidden engine feature WINDOW_LEAF_OPERATOR which is set only for MSQ engine. In presence of this feature, the planner plans without the leaf operators by creating a window query over an inner scan query. In case of native this is set to false and the planner generates the leafOperators
Guardrails around materialization
Comprehensive UTs
WorkerAssignmentStrategy.AUTO was missing a check for maxWorkerCount
in the case where the inputs to a stage are not dynamically sliceable.
A common case here is when the inputs to a stage are other stages.