* Handle null values of centralized schema config in PartialMergeTask
* Fix checkstyle
* Do not pass centralized schema config from supervisor task to sub-tasks
* Do not pass ObjectMapper in constructor of task
* Fix logs
* Fix tests
* Router: Authorize permissionless internal requests.
Router-internal requests like /proxy/enabled and errors for invalid
requests should not require permissions, but they still need to be
authorized in order to satisfy the PreResponseAuthorizationCheckFilter.
This patch adds authorization checks that do not require any particular
permissions.
* Fix tests.
* Add interface method for returning canonical lookup name
* Address review comment
* Add test in LookupReferencesManagerTest for coverage check
* Add test in LookupSerdeModuleTest for coverage check
* Fix task bootstrap locations.
* Remove dependency of SegmentCacheManager from SegmentLoadDropHandler.
- The load drop handler code talks to the local cache manager via
SegmentManager.
* Clean up unused imports and stuff.
* Test fixes.
* Intellij inspections and test bind.
* Clean up dependencies some more
* Extract test load spec and factory to its own class.
* Cleanup test util
* Pull SegmentForTesting out to TestSegmentUtils.
* Fix up.
* Minor changes to infoDir
* Replace server announcer mock and verify that.
* Add tests.
* Update javadocs.
* Address review comments.
* Separate methods for download and bootstrap load
* Clean up return types and exception handling.
* No callback for loadSegment().
* Minor cleanup
* Pull out the test helpers into its own static class so it can have better state control.
* LocalCacheManager stuff
* Fix build.
* Fix build.
* Address some CI warnings.
* Minor updates to javadocs and test code.
* Address some CodeQL test warnings and checkstyle fix.
* Pass a Consumer<DataSegment> instead of boolean & rename variables.
* Small updates
* Remove one test constructor.
* Remove the other constructor that wasn't initializing fully and update usages.
* Cleanup withInfoDir() builder and unnecessary test hooks.
* Remove mocks and elaborate on comments.
* Commentary
* Fix a few Intellij inspection warnings.
* Suppress corePoolSize intellij-inspect warning.
The intellij-inspect tool doesn't seem to correctly inspect
lambda usages. See ScheduledExecutors.
* Update docs and add more tests.
* Use hamcrest for asserting order on expectation.
* Shutdown bootstrap exec.
* Fix checkstyle
* fix issue with auto column grouping
changes:
* fixes bug where AutoTypeColumnIndexer reports incorrect cardinality, allowing it to incorrectly use array grouper algorithm for realtime queries producing incorrect results for strings
* fixes bug where auto LONG and DOUBLE type columns incorrectly report not having null values, resulting in incorrect null handling when grouping
* fix test
This PR updates CompactionTask to not load any lookups by default, unless transformSpec is present.
If transformSpec is present, we will make the decision based on context values, loading all lookups by default. This is done to ensure backward compatibility since transformSpec can reference lookups.
If transform spec is not present and no context value is passed, we donot load any lookup.
This behavior can be overridden by supplying lookupLoadingMode and lookupsToLoad in the task context.
Changes:
- Remove deprecated `markAsUnused` parameter from `KillUnusedSegmentsTask`
- Allow `kill` task to use `REPLACE` lock when `useConcurrentLocks` is true
- Use `EXCLUSIVE` lock by default
This parameter has been removed for awhile now as of Druid 0.23.0
https://github.com/apache/druid/pull/12187.
The code was only used in tests to verify that serialization works.
Now remove all references to avoid any confusion.
* * 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
Changes:
- Rename `DataSegmentChangeRequestAndStatus` to `DataSegmentChangeResponse`
- Rename `SegmentLoadDropHandler.Status` to `SegmentChangeStatus`
- Remove method `CoordinatorRunStats.getSnapshotAndReset()` as it was used only in
load queue peon implementations. Using an atomic reference is much simpler.
- Remove `ServerTestHelper.MAPPER`. Use existing `TestHelper.makeJsonMapper()` instead.
* 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.
This PR fixes the first and last vector aggregators and improves their readability. Following changes are introduced
The folding is broken in the vectorized versions. We consider time before checking the folded object.
If the numerical aggregator gets passed any other object type for some other reason (like String), then the aggregator considers it to be folded, even though it shouldn’t be. We should convert these objects to the desired type, and aggregate them properly.
The aggregators must properly use generics. This would minimize the ClassCastException issues that can happen with mixed segment types. We are unifying the string first/last aggregators with numeric versions as well.
The aggregators must aggregate null values (https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java#L55-L56 ). The aggregator should only ignore pairs with time == null, and not value == null
Time nullity is ignored when trying to vectorize the data.
String versions initialized with DateTimes.MIN that is equal to Long.MIN / 2. This can cause incorrect results in case the user enters a custom time column. NOTE: This is still present because it would require a larger refactor in all of the versions.
There is a difference in what users might expect from the results because the code flow is changed (for example, the direction of the for loops, etc), however, this will only change the results, and not the contract set by first/last aggregators, which is that if multiple values have the same timestamp, then any of them can get picked.
If the column is non-existent, the users might expect a change in the timestamp from DateTime.MAX to Long.MAX, because the code incorrectly used DateTime.MAX to initialize the aggregator, however, in case of a custom timestamp column, this might not be the case. The SQL query might be prohibited from using any Long since it requires a cast to the timestamp function that can fail, but AFAICT native queries don't have such limitations.
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:
- Add `LookupLoadingSpec` to support 3 modes of lookup loading: ALL, NONE, ONLY_REQUIRED
- Add method `Task.getLookupLoadingSpec()`
- Do not load any lookups for `KillUnusedSegmentsTask`
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.
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.
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.
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
The default value for druid.coordinator.kill.period (if unspecified) has changed from P1D to the value of druid.coordinator.period.indexingPeriod. Operators can choose to override druid.coordinator.kill.period and that will take precedence over the default behavior.
The default value for the coordinator dynamic config killTaskSlotRatio is updated from 1.0 to 0.1. This ensures that that kill tasks take up only 1 task slot right out-of-the-box instead of taking up all the task slots.
* Remove stale comment and inline canDutyRun()
* druid.coordinator.kill.period defaults to druid.coordinator.period.indexingPeriod if not set.
- Remove the default P1D value for druid.coordinator.kill.period. Instead default
druid.coordinator.kill.period to whatever value druid.coordinator.period.indexingPeriod is set
to if the former config isn't specified.
- If druid.coordinator.kill.period is set, the value will take precedence over
druid.coordinator.period.indexingPeriod
* Update server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java
* Fix checkstyle error
* Clarify comment
* Update server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorConfig.java
* Put back canDutyRun()
* Default killTaskSlotsRatio to 0.1 instead of 1.0 (all slots)
* Fix typo DEFAULT_MAX_COMPACTION_TASK_SLOTS
* Remove unused test method.
* Update default value of killTaskSlotsRatio in docs and web-console default mock
* Move initDuty() after params and config setup.
* Update error message when topic messages.
Suggest resetting the supervisor when the topic changes instead of changing
the supervisor name which is actually making a new supervisor.
* Update server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Cleanup
* Remove log and include oldCommitMetadataFromDb
* Fix test
---------
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
Changes:
- Handle exceptions in the API and map them to a `Response` object with the appropriate error code.
- Replace `AuthorizationUtils.filterAuthorizedResources()` with `DatasourceResourceFilter`.
The endpoint is annotated consistent with other usages.
- Update `DatasourceResourceFilter` to remove the lambda and update javadocs.
The usages information is self-evident with an IDE.
- Adjust the invalid interval exception message.
- Break up the large unit test `testGetUnusedSegmentsInDataSource()` into smaller unit tests
for each test case. Also, validate the error codes.
* Differentiate null and empty lists of segment IDs and versions.
Treat them differently so the. Segment IDs and versions can be An empty list,
in which case, the queries should just not return anything. Versions are optional, so
they can be null, which just indicates nothing, so the queries should return segments with
all possible versions. Segment IDs cannot be null as indicated by the absence of @Nullable
annotation.
* Update javadocs and add empty versions test to kill task.
* Add test for RetrieveSegmentsActions as well.
* Add parameterized segment IDs.
* Refactor into one common method.
* Refactor getConditionForIntervalsAndMatchMode - pass in only what's needed.
* Minor cleanup.