A SegmentTransactionReplaceAction must only update the mapping of tasks with append locks that are running concurrently. To ensure this, we return the supervisor id only if it has the taskLockType as APPEND in its context.
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.
* Separate k8s and druid task lifecycles
* Remove extra log lines
* Fix unit tests
* fix unit tests
* Fix unit tests
* notify listeners on task completion
* Fix unit test
* unused var
* PR changes
* Fix unit tests
* Fix checkstyle
* PR changes
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.
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
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'
* save work
* Working
* Fix runner constructor
* Working runner
* extra log lines
* try using lifecycle for everything
* clean up configs
* cleanup /workers call
* Use a single config
* Allow selecting runner
* debug changes
* Work on composite task runner
* Unit tests running
* Add documentation
* Add some javadocs
* Fix spelling
* Use standard libraries
* code review
* fix
* fix
* use taskRunner as string
* checkstyl
---------
Co-authored-by: Suneet Saldanha <suneet@apache.org>
Changes:
- Simplify static `create` methods for `NoopTask`
- Remove `FirehoseFactory`, `IsReadyResult`, `readyTime` from `NoopTask`
as these fields were not being used anywhere
- Update tests
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:
- 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`
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
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
* 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!
* 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
### 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.
* 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.
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
* 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
* More efficient generation of ImmutableWorkerHolder from WorkerHolder.
Taking the work done in #12096 a little further:
1) Applying a similar optimization to WorkerHolder (HttpRemoteTaskRunner).
The original patch only helped with the ZkWorker (RemoteTaskRunner).
2) Improve the ZkWorker version somewhat by avoiding multiple iterations
through the task announcements map.
* Pick better names and use better logic.
* Only runnable tasks.
* Fix test.
* Fix testBlacklistZKWorkers50Percent.