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.
Changes:
- Add task context parameter `taskLockType`. This determines the type of lock used by a batch task.
- Add new task actions for transactional replace and append of segments
- Add methods StorageCoordinator.commitAppendSegments and commitReplaceSegments
- Upgrade segments to appropriate versions when performing replace and append
- Add new metadata table `upgradeSegments` to track segments that need to be upgraded
- Add tests
- Add `KillTaskReport` that contains stats for `numSegmentsKilled`,
`numBatchesProcessed`, `numSegmentsMarkedAsUnused`
- Fix bug where exception message had no formatter but was was still being passed some args.
- Add some comments regarding deprecation of `markAsUnused` flag.
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
When materializing the results as frames, we defer the creation of the frames in ScanQueryQueryToolChest, which passes through the catch-all block reserved for catching cases when we don't have the complete row signature in the query (and falls back to the old code).
This PR aims to resolve it by adding the frame generation code to the try-catch block we have at the outer level.
Changes:
- Move following configs from `CliCoordinator` to `DruidCoordinatorConfig`:
- `druid.coordinator.kill.on`
- `druid.coordinator.kill.pendingSegments.on`
- `druid.coordinator.kill.supervisors.on`
- `druid.coordinator.kill.rules.on`
- `druid.coordinator.kill.audit.on`
- `druid.coordinator.kill.datasource.on`
- `druid.coordinator.kill.compaction.on`
- In the Coordinator style used by historical management duties, always instantiate all
the metadata cleanup duties but execute only if enabled. In the existing code, they are
instantiated only when enabled by using optional binding with Guice.
- Add a wrapper `MetadataManager` which contains handles to all the different
metadata managers for rules, supervisors, segments, etc.
- Add a `CoordinatorConfigManager` to simplify read and update of coordinator configs
- Remove persistence related methods from `CoordinatorCompactionConfig` and
`CoordinatorDynamicConfig` as these are config classes.
- Remove annotations `@CoordinatorIndexingServiceDuty`,
`@CoordinatorMetadataStoreManagementDuty`
changes:
* add back nested column v4 serializers
* 'json' schema by default still uses the newer 'nested common format' used by 'auto', but now has an optional 'formatVersion' property which can be specified to override format versions on native ingest jobs
* add system config to specify default column format stuff, 'druid.indexing.formats', and property 'druid.indexing.formats.nestedColumnFormatVersion' to specify system level preferred nested column format for friendly rolling upgrades from versions which do not support the newer 'nested common format' used by 'auto'
Changes:
- Add new metric `kill/pendingSegments/count` with dimension `dataSource`
- Add tests for `KillStalePendingSegments`
- Reduce no-op logs that spit out for each datasource even when no pending
segments have been deleted. This can get particularly noisy at low values of `indexingPeriod`.
- Refactor the code in `KillStalePendingSegments` for readability and add javadocs
A new monitor SubqueryCountStatsMonitor which emits the metrics corresponding to the subqueries and their execution is now introduced. Moreover, the user can now also use the auto mode to automatically set the number of bytes available per query for the inlining of its subquery's results.
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:
[A] Remove config `decommissioningMaxPercentOfMaxSegmentsToMove`
- It is a complicated config 😅 ,
- It is always desirable to prioritize move from decommissioning servers so that
they can be terminated quickly, so this should always be 100%
- It is already handled by `smartSegmentLoading` (enabled by default)
[B] Remove config `maxNonPrimaryReplicantsToLoad`
This was added in #11135 to address two requirements:
- Prevent coordinator runs from getting stuck assigning too many segments to historicals
- Prevent load of replicas from competing with load of unavailable segments
Both of these requirements are now already met thanks to:
- Round-robin segment assignment
- Prioritization in the new coordinator
- Modifications to `replicationThrottleLimit`
- `smartSegmentLoading` (enabled by default)
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
Changes:
- Reduce log level of some coordinator stats, which only denote normal coordinator operation.
These stats are still emitted and can be logged by setting debugDimensions in the coordinator
dynamic config.
- Initialize SegmentLoadingConfig only for historical management duties. This config is not
needed in other duties and initializing it creates logs which are misleading.
Changes
- Increase value of `replicationThrottleLimit` computed by `smartSegmentLoading` from
2% to 5% of total number of used segments.
- Assign replicas to a tier even when some replicas are already being loaded in that tier
- Limit the total number of replicas in load queue at start of run + replica assignments in
the run to the `replicationThrottleLimit`.
i.e. for every tier,
num loading replicas at start of run + num replicas assigned in run <= replicationThrottleLimit
Changes:
- Determine the default value of balancerComputeThreads based on number of
coordinator cpus rather than number of segments. Even if the number of segments
is low and we create more balancer threads, it doesn't hurt the system as threads
would mostly be idle.
- Remove unused field from SegmentLoadQueueManager
Expected values:
- Clusters with ~1M segments typically work with Coordinators having 16 cores or more.
This would give us 8 balancer threads, which is the same as the current maximum.
- On small clusters, even a single thread is enough to do the required balancing work.
### Description
This change enables the `KillUnusedSegments` coordinator duty to be scheduled continuously. Things that prevented this, or made this difficult before were the following:
1. If scheduled at fast enough rate, the duty would find the same intervals to kill for the same datasources, while kill tasks submitted for those same datasources and intervals were already underway, thus wasting task slots on duplicated work.
2. The task resources used by auto kill were previously unbounded. Each duty run period, if unused
segments were found for any datasource, a kill task would be submitted to kill them.
This pr solves for both of these issues:
1. The duty keeps track of the end time of the last interval found when killing unused segments for each datasource, in a in memory map. The end time for each datasource, if found, is used as the start time lower bound, when searching for unused intervals for that same datasource. Each duty run, we remove any datasource keys from this map that are no longer found to match datasources in the system, or in whitelist, and also remove a datasource entry, if there is found to be no unused segments for the datasource, which happens when we fail to find an interval which includes unused segments. Removing the datasource entry from the map, allows for searching for unusedSegments in the datasource from the beginning of time once again
2. The unbounded task resource usage can be mitigated with coordinator dynamic config added as part of ba957a9b97
Operators can configure continous auto kill by providing coordinator runtime properties similar to the following:
```
druid.coordinator.period.indexingPeriod=PT60S
druid.coordinator.kill.period=PT60S
```
And providing sensible limits to the killTask usage via coordinator dynamic properties.
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.
Motivation:
- Clean up `DruidCoordinator` and move methods to classes where they are most relevant
Changes:
- No functional change
- Add duty `PrepareBalancerAndLoadQueues` to replace `UpdateCoordinatorState`
- Move map of `LoadQueuePeon` from `DruidCoordinator` to `LoadQueueTaskMaster`
- Make `BalancerStrategyFactory` an abstract class and keep the balancer executor here
- Move reporting of used segment stats and historical capacity stats from
`CollectSegmentAndServerStats` to `PrepareBalancerAndLoadQueues`
- Move reporting of unavailable and under-replicated segment stats from
`CollectSegmentAndServerStats` to `UpdateReplicationStatus` duty
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
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
Changes:
- Move logic of `NewestSegmentFirstIterator.needsCompaction` to `CompactionStatus`
to improve testability and readability
- Capture the list of checks performed to determine if compaction is needed in a readable
manner in `CompactionStatus.CHECKS`
- Make `CompactionSegmentIterator` iterate over instances of `SegmentsToCompact`
instead of `List<DataSegment>`. This allows use of the `umbrellaInterval` later.
- Replace usages of `QueueEntry` with `SegmentsToCompact`
- Move `SegmentsToCompact` out of `NewestSegmentFirstIterator`
- Simplify `CompactionStatistics`
- Reduce level of less important logs to debug
- No change made to tests to ensure correctness
Changes:
- Use separate executor for every duty group
- This change is thread-safe as every duty group uses its own copy of
`DruidCoordinatorRuntimeParams` and does not share any other mutable instances
with other duty groups.
- With the exception of `HistoricalManagementDuties`, duty groups are typically not
very compute intensive and mostly perform database or HTTP I/O. So, coordinator
resources would still mostly be available for `HistoricalManagementDuties`.
Follow up changes to #12599
Changes:
- Rename column `used_flag_last_updated` to `used_status_last_updated`
- Remove new CLI tool `UpdateTables`.
- We already have a `CreateTables` with similar functionality, which should be able to
handle update cases too.
- Any user running the cluster for the first time should either just have `connector.createTables`
enabled or run `CreateTables` which should create tables at the latest version.
- For instance, the `UpdateTables` tool would be inadequate when a new metadata table has
been added to Druid, and users would have to run `CreateTables` anyway.
- Remove `upgrade-prep.md` and include that info in `metadata-init.md`.
- Fix log messages to adhere to Druid style
- Use lambdas
* 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.