Commit Graph

2126 Commits

Author SHA1 Message Date
AmatyaAvadhanula 65b69cded4
Filter pending segments upgraded with transactional replace (#15169)
* Filter pending segments upgraded with transactional replace

* Push sequence name filter to metadata query
2023-10-23 21:18:47 +05:30
AmatyaAvadhanula 33fdd770f7
Consider only supervisors with append lock for concurrent transactional replace (#15220)
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.
2023-10-22 14:12:36 +05:30
AmatyaAvadhanula a8febd457c
A Replacing task must read segments created before it acquired its lock (#15085)
* Replacing tasks must read segments created before they acquired their locks
2023-10-19 11:13:07 +05:30
Karan Kumar 953ce79439
Add undocumented taskLockType to MSQ. (#15168)
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.
2023-10-17 21:44:04 +05:30
George Shiqi Wu dc0b163e19
Separate task lifecycle from kubernetes/location lifecycle (#15133)
* 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
2023-10-17 08:17:43 -07:00
AmatyaAvadhanula d25caaefa4
Add support for streaming ingestion with concurrent replace (#15039)
Add support for streaming ingestion with concurrent replace

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2023-10-13 09:09:03 +05:30
Laksh Singla 5f86072456
Prepare master for Druid 29 (#15121)
Prepare master for Druid 29
2023-10-11 10:33:45 +05:30
AmatyaAvadhanula 40a6dc4631
Optimize used segment fetching in Kill tasks (#15107)
* Optimize used segment fetching in Kill tasks
2023-10-09 17:54:13 +05:30
Xavier Léauté adef2069b1
Make unit tests pass with Java 21 (#15014)
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
2023-10-03 22:41:21 -07:00
George Shiqi Wu 64754b6799
Allow users to pass task payload via deep storage instead of environment variable (#14887)
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.
2023-10-03 14:08:59 +05:30
YongGang 86087cee0a
Fix Peon not fail gracefully (#14880)
* fix Peon not fail gracefully

* move methods to Task interface

* fix checkstyle

* extract to interface

* check runThread nullability

* fix merge conflict

* minor refine

* minor refine

* fix unit test

* increase latch waiting time
2023-09-29 12:39:59 -07:00
AmatyaAvadhanula f7a549123b
Commit segments only when they are covered by active locks (#15027)
* Commit segments only when they are covered by active locks
2023-09-25 13:45:42 +05:30
AmatyaAvadhanula c62193c4d7
Add support for concurrent batch Append and Replace (#14407)
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
2023-09-25 07:06:37 +05:30
Kashif Faraz d7c152c82c
Add a TaskReport for "kill" tasks (#15023)
- 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.
2023-09-23 07:44:27 +05:30
Kashif Faraz 409bffe7f2
Rename IMSC.announceHistoricalSegments to commitSegments (#15021)
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
2023-09-21 16:19:03 +05:30
AmatyaAvadhanula 0e3df2d2e9
Clean up stale locks if segment allocation fails (#14966)
* Clean up stale locks if segment allocation fails due to an exception
2023-09-14 14:58:02 +05:30
Clint Wylie 891f0a3fe9
longer compatibility window for nested column format v4 (#14955)
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'
2023-09-12 14:07:53 -07:00
George Shiqi Wu f773d83914
Mixed task runner for migration to mm-less ingestion (#14918)
* 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>
2023-09-11 18:09:46 -07:00
Kashif Faraz 289ee1e011
Refactor: Cleanup NoopTask (#14938)
Changes:
- Simplify static `create` methods for `NoopTask`
- Remove `FirehoseFactory`, `IsReadyResult`, `readyTime` from `NoopTask`
as these fields were not being used anywhere
- Update tests
2023-09-05 09:15:41 +05:30
panhongan d4e972e1e4
Add checking for new checkpoint (#14353)
Check that a checkpoint is non-empty before adding it to the checkpoint sequence 
in a SeekableStreamSupervisor
2023-09-04 13:18:55 +05:30
Kashif Faraz 7f26b80e21
Simplify ServiceMetricEvent.Builder (#14933)
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
2023-09-01 11:30:45 +05:30
George Shiqi Wu ad32f84586
Fix capacity response in mm-less ingestion (#14888)
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`
2023-08-25 08:17:38 +05:30
Adarsh Sanjeev dfb5a98888
Add coordinator API for unused segments (#14846)
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.
2023-08-23 14:51:25 +05:30
Clint Wylie fb053c399c
consolidate json and auto indexers, remove v4 nested column serializer (#14456) 2023-08-22 18:50:11 -07:00
AmatyaAvadhanula bd505062de
Improve streaming ingestion completion timeout error message (#14636)
* Improve streaming ingestion completion timeout error message

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
Co-authored-by: Benedict Jin <asdf2014@apache.org>
2023-08-22 14:33:28 +05:30
Clint Wylie 194a9c9abc
set druid.expressions.useStrictBooleans to true by default (#14734) 2023-08-22 00:19:56 -07:00
Tejaswini Bandlamudi d87056e708
Upgrade guava version to 31.1-jre (#14767)
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
2023-08-22 12:09:53 +05:30
Kashif Faraz 92906059d2
Remove segmentsToBeDropped from SegmentTransactionInsertAction (#14883)
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
2023-08-21 20:08:56 +05:30
Kashif Faraz c211dcc4b3
Clean up compaction logs on coordinator (#14875)
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
2023-08-21 17:30:41 +05:30
Jonathan Wei a8eaa1e4ed
Skip streaming auto-scaling action if supervisor is idle (#14773)
* Skip streaming auto-scaling action if supervisor is idle

* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>

---------

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>
2023-08-17 19:43:25 -05:00
Abhishek Radhakrishnan 37db5d9b81
Reset offsets supervisor API (#14772)
* 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!
2023-08-17 14:13:10 -07:00
Clint Wylie 6b14dde50e
deprecate config-magic in favor of json configuration stuff (#14695)
* json config based processing and broker merge configs to deprecate config-magic
2023-08-16 18:23:57 -07:00
AmatyaAvadhanula e16096735b
Fix 404 when segment is used but not in the Coordinator snapshot (#14762)
* Fix 404 when used segment has not been updated in the Coordinator snapshot

* Add unit test
2023-08-14 13:20:43 +05:30
Rishabh Singh 0dc305f9e4
Upgrade hibernate validator version to fix CVE-2019-10219 (#14757) 2023-08-14 11:50:51 +05:30
Tejaswini Bandlamudi a45b25fa1d
Removes support for Hadoop 2 (#14763)
Removing Hadoop 2 support as discussed in https://lists.apache.org/list?dev@druid.apache.org:lte=1M:hadoop
2023-08-09 17:47:52 +05:30
Suneet Saldanha 2af0ab2425
Metric to report time spent fetching and analyzing segments (#14752)
* Metric to report time spent fetching and analyzing segments

* fix test

* spell check

* fix tests

* checkstyle

* remove unused variable

* Update docs/operations/metrics.md

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* Update docs/operations/metrics.md

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* Update docs/operations/metrics.md

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

---------

Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
2023-08-07 18:32:48 -07:00
Suneet Saldanha b624a4ec4a
Rolling Supervisor restarts at taskDuration (#14396)
* 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
2023-08-07 16:24:32 -07:00
George Shiqi Wu 14940dc3ed
Add pod name to TaskLocation for easier observability and debugging. (#14758)
* Add pod name to location

* Add log

* fix style

* Update extensions-contrib/kubernetes-overlord-extensions/src/main/java/org/apache/druid/k8s/overlord/KubernetesPeonLifecycle.java

Co-authored-by: Suneet Saldanha <suneet@apache.org>

* Fix unit tests

---------

Co-authored-by: Suneet Saldanha <suneet@apache.org>
2023-08-07 12:33:35 -07:00
zachjsh ba957a9b97
Add ability to limit the number of segments killed in kill task (#14662)
### 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.
2023-08-03 22:17:04 -04:00
Gian Merlino 5387f1bac0
Remove chatAsync parameter, so chat is always async. (#14692)
* 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.
2023-07-31 19:42:51 -07:00
Jason Koch 44d5c1a15f
split KillUnusedSegmentsTask to processing in smaller chunks (#14642)
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
2023-07-31 12:56:27 -07:00
AmatyaAvadhanula c648b1cb36
Add task toolbox to DruidInputSource (#14507)
Add task toolbox to DruidInputSource
2023-07-31 13:12:01 +05:30
TSFenwick 9a9038c7ae
Speed up kill tasks by deleting segments in batch (#14131)
* 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
2023-07-27 15:34:44 -07:00
Gian Merlino 986a271a7d
Merge core CoordinatorClient with MSQ CoordinatorServiceClient. (#14652)
* 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
2023-07-27 13:23:37 -07:00
Gian Merlino 2f9619a96f
Use OverlordClient for all Overlord RPCs. (#14581)
* 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.
2023-07-24 21:14:27 -07:00
Gian Merlino bac5ef347c
Add ingest/input/bytes metric and Kafka consumer metrics. (#14582)
* 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>
2023-07-20 10:56:22 +08:00
Clint Wylie 913416c669
add equality, null, and range filter (#14542)
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
2023-07-18 12:15:22 -07:00
Maytas Monsereenusorn aef221f71b
Allow multiple consoleAppender to be used in peon logging (#14521)
* Allow multiple consoleAppender to be used in peon logging

* Fix Attempted to append to non-started appender error
2023-07-17 21:29:45 -07:00
AmatyaAvadhanula 0412f40d36
Prepare master branch for next release, 28.0.0 (#14595)
* Prepare master branch for next release, 28.0.0
2023-07-18 09:22:30 +05:30
Gian Merlino 450ecd6370
More efficient generation of ImmutableWorkerHolder from WorkerHolder. (#14546)
* 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.
2023-07-13 07:57:16 -07:00