* Add new configurable buffer period to create gap between mark unused and kill of segment
* Changes after testing
* fixes and improvements
* changes after initial self review
* self review changes
* update sql statement that was lacking last_used
* shore up some code in SqlMetadataConnector after self review
* fix derby compatibility and improve testing/docs
* fix checkstyle violations
* Fixes post merge with master
* add some unit tests to improve coverage
* ignore test coverage on new UpdateTools cli tool
* another attempt to ignore UpdateTables in coverage check
* change column name to used_flag_last_updated
* fix a method signature after column name switch
* update docs spelling
* Update spelling dictionary
* Fixing up docs/spelling and integrating altering tasks table with my alteration code
* Update NULL values for used_flag_last_updated in the background
* Remove logic to allow segs with null used_flag_last_updated to be killed regardless of bufferPeriod
* remove unneeded things now that the new column is automatically updated
* Test new background row updater method
* fix broken tests
* fix create table statement
* cleanup DDL formatting
* Revert adding columns to entry table by default
* fix compilation issues after merge with master
* discovered and fixed metastore inserts that were breaking integration tests
* fixup forgotten insert by using pattern of sharing now timestamp across columns
* fix issue introduced by merge
* fixup after merge with master
* add some directions to docs in the case of segment table validation issues
* 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!
Changes:
- No change in behaviour if `smartSegmentLoading` is disabled
- If `smartSegmentLoading` is enabled
- Compute `balancerComputeThreads` based on `numUsedSegments`
- Compute `maxSegmentsToMove` based on `balancerComputeThreads`
- Compute `segmentsToMoveToFixSkew` based on usage skew
- Compute `segmentsToMove = Math.min(maxSegmentsToMove, segmentsToMoveToFixSkew)`
Limits:
- 1 <= `balancerComputeThreads` <= 8
- `maxSegmentsToMove` <= 20% of total segments
- `minSegmentsToMove` = 0.15% of total segments
`cachingCost` has been deprecated in #14484 and is not advised to be used in
production clusters as it may cause usage skew across historicals which the
coordinator is unable to rectify. This PR completely disables `cachingCost` strategy
as it has now been rendered redundant due to recent performance improvements
made to `cost` strategy.
Changes
- Disable `cachingCost` strategy
- Add `DisabledCachingCostBalancerStrategyFactory` for the time being so that we
can give a proper error message before falling back to `CostBalancerStrategy`. This
will be removed in subsequent releases.
- Retain `CachingCostBalancerStrategy` for testing/benchmarking purposes.
- Add javadocs to `DiskNormalizedCostBalancerStrategy`
### Description
Added the following metrics, which are calculated from the `KillUnusedSegments` coordinatorDuty
`"killTask/availableSlot/count"`: calculates the number remaining task slots available for auto kill
`"killTask/maxSlot/count"`: calculates the maximum number of tasks available for auto kill
`"killTask/task/count"`: calculates the number of tasks submitted by auto kill.
#### Release note
NEW: metrics added for auto kill
`"killTask/availableSlot/count"`: calculates the number remaining task slots available for auto kill
`"killTask/maxSlot/count"`: calculates the maximum number of tasks available for auto kill
`"killTask/task/count"`: calculates the number of tasks submitted by auto kill.
The current version of jackson-databind is flagged for vulnerabilities CVE-2020-28491 (Although cbor format is not used in druid), CVE-2020-36518 (Seems genuine as deeply nested json in can cause resource exhaustion). Updating the dependency to the latest version 2.12.7 to fix these vulnerabilities.
### Description
Previously, the `KillUnusedSegments` coordinator duty, in charge of periodically deleting unused segments, could spawn an unlimited number of kill tasks for unused segments. This change adds 2 new coordinator dynamic configs that can be used to control the limit of tasks spawned by this coordinator duty
`killTaskSlotRatio`: Ratio of total available task slots, including autoscaling if applicable that will be allowed for kill tasks. This limit only applies for kill tasks that are spawned automatically by the coordinator's auto kill duty. Default is 1, which allows all available tasks to be used, which is the existing behavior
`maxKillTaskSlots`: Maximum number of tasks that will be allowed for kill tasks. This limit only applies for kill tasks that are spawned automatically by the coordinator's auto kill duty. Default is INT.MAX, which essentially allows for unbounded number of tasks, which is the existing behavior.
Realize that we can effectively get away with just the one `killTaskSlotRatio`, but following similarly to the compaction config, which has similar properties; I thought it was good to have some control of the upper limit regardless of ratio provided.
#### Release note
NEW: `killTaskSlotRatio` and `maxKillTaskSlots` coordinator dynamic config properties added that allow control of task resource usage spawned by `KillUnusedSegments` coordinator task (auto kill)
### Description
Previously, the `maxSegments` configured for auto kill could be ignored if an interval of data for a given datasource had more than this number of unused segments, causing the kill task spawned with the task of deleting unused segments in that given interval of data to delete more than the `maxSegments` configured. Now each kill task spawned by the auto kill coordinator duty, will kill at most `limit` segments. This is done by adding a new config property to the `KillUnusedSegmentTask` which allows users to specify this limit.
* 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.
Changes:
- Add interface `SegmentDeleteHandler` for marking segments as unused
- In `StrategicSegmentAssigner`, collect all segments on which a drop rule applies in a list
- Process the list above as a batch delete rather than individual deletes
- Improve alert messages when an invalid tier is specified in a load rule
- Improve alert message when no rule applies on a segment
split KillUnusedSegmentsTask to smaller batches
Processing in smaller chunks allows the task execution to yield the TaskLockbox lock,
which allows the overlord to continue being responsive to other tasks and users while
this particular kill task is executing.
* introduce KillUnusedSegmentsTask batchSize parameter to control size of batching
* provide an explanation for kill task batchSize parameter
* add logging details for kill batch progress
Changes
- Rename `LoadQueuePeonTester` to `TestLoadQueuePeon`
- Simplify `TestLoadQueuePeon` by removing dependency on `CuratorLoadQueuePeon`
- Remove usages of mock peons in `LoadRuleTest` and use `TestLoadQueuePeon` instead
* 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.
Related to #14634
Changes:
- Update `IndexerSQLMetadataStorageCoordinator.deleteSegments` to use
JDBI PreparedBatch instead of issuing single DELETE statements
This PR uses the QoSFilter available in Jetty to park the query requests that exceed a configured limit. This is done so that other HTTP requests such as health check calls do not get blocked if the query server is busy serving long-running queries. The same mechanism can also be used in the future to isolate interactive queries from long-running select queries from interactive queries within the same broker.
Right now, you can still get that isolation by setting druid.query.scheduler.numThreads to a value lowe than druid.server.http.numThreads. That enables total laning but the side effect is that excess requests are not queued and rejected outright that leads to a bad user experience.
Parked requests are timed out after 30 seconds by default. I overrode that to the maxQueryTimeout in this PR.
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
Tests to verify the following behaviour have been added:
- Segments from more populous servers are more likely to be picked irrespective of
sample size.
- Segments from all servers are equally likely to be picked if all servers have equivalent
number of segments.
* 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>
* Add aggregatorMergeStrategy property to SegmentMetadaQuery.
- Adds a new property aggregatorMergeStrategy to segmentMetadata query.
aggregatorMergeStrategy currently supports three types of merge strategies -
the legacy strict and lenient strategies, and the new latest strategy.
- The latest strategy considers the latest aggregator from the latest segment
by time order when there's a conflict when merging aggregators from different
segments.
- Deprecate lenientAggregatorMerge property; The API validates that both the new
and old properties are not set, and returns an exception.
- When merging segments as part of segmentMetadata query, the segments have a more
elaborate id -- <datasource>_<interval>_merged_<partition_number> format, similar to
the name format that segments usually contain. Previously it was simply "merged".
- Adjust unit tests to test the latest strategy, to assert the returned complete
SegmentAnalysis object instead of just the aggregators for completeness.
* Don't explicitly set strict strategy in tests
* Apply suggestions from code review
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* Update docs/querying/segmentmetadataquery.md
* Apply suggestions from code review
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
---------
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* 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.
Cache is disabled for GroupByStrategyV2 on broker since the pr #3820 [groupBy v2: Results not fully merged when caching is enabled on the broker]. But we can enable the result-level cache on broker for GroupByStrategyV2 and keep the segment-level cache disabled.
Description:
`TaskQueue.notifyStatus` is often a heavy call as it performs the following operations:
- Update task status in metadata DB
- Update task locks in metadata DB
- Request (synchronously) the task runner to shutdown the completed task
- Clean up in-memory data structures
This method can often be slow and can cause worker sync / task runners to slow down.
Main changes:
- Run task completion callbacks in a separate executor to handle task completion updates
- Add new config `druid.indexer.queue.taskCompleteHandlerNumThreads`
- Add metrics to monitor number of processed and queued items
- There are still other paths that can invoke `notifyStatus`, but those need not be moved to
the new executor as they are synchronous on purpose.
Other changes:
- Add new metrics `task/status/queue/count`, `task/status/handled/count`
- Add `TaskCountStatsProvider.getStats()` which deprecates the other `getXXXTaskCount` methods.
- Use `CoordinatorRunStats` to collect and report metrics. This class has been used as is
for now but will later be renamed and repurposed to use across all Druid services.
The wait doesn't seem to serve a purpose, other than causing delays
when checking isInitialized() for a large number of things that have
not yet been initialized.
UniformGranularityTest's test to test a large number of intervals
runs through 10 years of 1 second intervals. This pushes a lot of
stuff through IntervalIterator and shows up in terms of test
runtime as one of the hottest tests. Most of the time is going to
constructing jodatime objects because it is doing things with
DateTime objects instead of millis. Change the calls to use
millis instead and things go faster.
If a server is removed during `HttpServerInventoryView.serverInventoryInitialized`,
the initialization gets stuck as this server is never synced. The method eventually times
out (default 250s).
Fix: Mark a server as stopped if it is removed. `serverInventoryInitialized` only waits for
non-stopped servers to sync.
Other changes:
- Add new metrics for better debugging of slow broker/coordinator startup
- `segment/serverview/sync/healthy`: whether the server view is syncing properly with a server
- `segment/serverview/sync/unstableTime`: time for which sync with a server has been unstable
- Clean up logging in `HttpServerInventoryView` and `ChangeRequestHttpSyncer`
- Minor refactor for readability
- Add utility class `Stopwatch`
- Add tests and stubs
After #13197 , several coordinator configs are now redundant as they are not being
used anymore, neither with `smartSegmentLoading` nor otherwise.
Changes:
- Remove dynamic configs `emitBalancingStats`: balancer error stats are always
emitted, debug stats can be logged by using `debugDimensions`
- `useBatchedSegmentSampler`, `percentOfSegmentsToConsiderPerMove`:
batched segment sampling is always used
- Add test to verify deserialization with unknown properties
- Update `CoordinatorRunStats` to always track stats, this can be optimized later.
* 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