Commit Graph

2301 Commits

Author SHA1 Message Date
AmatyaAvadhanula c41e99e10c
Do not allocate week granular segments unless requested (#15589)
* Do not allocate week granular segments unless explicitly requested
2024-01-05 12:14:52 +05:30
Kashif Faraz c937068625
Improve polling in segment allocation queue (#15590)
Description
When batchAllocationWaitTime is set to 0, the segment allocation queue is polled continuously even when it is empty. This would take up cpu cycles unnecessarily.

Some existing race conditions would also become more frequent when the batchAllocationWaitTime is 0. This PR tries to better address those race conditions as well.

Changes
Do not reschedule a poll if queue is empty
When a new batch is added to queue, schedule a poll
Simplify keyToBatch map
Handle race conditions better
As soon as a batch starts getting processed, do not add any more requests to it
2024-01-04 17:42:02 +05:30
Abhishek Radhakrishnan 9c7d7fc777
Allow empty inserts and replaces in MSQ. (#15495)
* Allow empty inserts and replace.

- Introduce a new query context failOnEmptyInsert which defaults to false.
- When this context is false (default), MSQE will now allow empty inserts and replaces.
- When this context is true, MSQE will throw the existing InsertCannotBeEmpty MSQ fault.
- For REPLACE ALL over an ALL grain segment, the query will generate a tombstone spanning eternity
which will be removed eventually be the coordinator.
- Add unit tests in MSQInsertTest, MSQReplaceTest to test the new default behavior (i.e., when failOnEmptyInsert = false)
- Update unit tests in MSQFaultsTest to test the non-default behavior (i.e., when failOnEmptyInsert = true)

* Ignore test to see if it's the culprit for OOM

* Add heap dump config

* Bump up -Xmx from 1500 MB to 2048 MB

* Add steps to tarball and collect hprof dump to GHA action

* put back mx to 1500MB to trigger the failure

* add the step to reusable unit test workflow as well

* Revert the temp heap dump & @Ignore changes since max heap size is increased

* Minor updates

* Review comments

1. Doc suggestions
2. Add tests for empty insert and replace queries with ALL grain and limit in the
   default failOnEmptyInsert mode (=false). Add similar tests to MSQFaultsTest with
   failOnEmptyInsert = true, so the query does fail with an InsertCannotBeEmpty fault.
3. Nullable annotation and javadocs

* Add comment
	replace_limit.patch
2024-01-02 13:05:51 -08:00
kaisun2000 a5e9b14be0
Add delay before the peon drops the segments after publishing them (#15373)
Currently in the realtime ingestion (Kafka/Kinesis) case, after publishing the segments, upon acknowledgement from the coordinator that the segments are already placed in some historicals, the peon would unannounce the segments (basically saying the segments are not in this peon anymore to the whole cluster) and drop the segments from cache and sink timeline in one shot.

The in transit queries from the brokers that still thinks the segments are in the peon can get a NullPointer exception when the peon is unsetting the hydrants in the sinks.

The fix would let the peon to wait for a configurable delay period before dropping segments, remove segments from cache etc after the peon unannounce the segments.

This delayed approach is similar to how the historicals handle segments moving out.
2024-01-02 11:08:28 +05:30
Kashif Faraz 9f568858ef
Add logging implementation for AuditManager and audit more endpoints (#15480)
Changes
- Add `log` implementation for `AuditManager` alongwith `SQLAuditManager`
- `LoggingAuditManager` simply logs the audit event. Thus, it returns empty for
all `fetchAuditHistory` calls.
- Add new config `druid.audit.manager.type` which can take values `log`, `sql` (default)
- Add new config `druid.audit.manager.logLevel` which can take values `DEBUG`, `INFO`, `WARN`.
This gets activated only if `type` is `log`.
- Remove usage of `ConfigSerde` from `AuditManager` as audit is not just limited to configs
- Add `AuditSerdeHelper` for a single implementation of serialization/deserialization of
audit payload and other utility methods.
2023-12-19 13:14:04 +05:30
Kashif Faraz feeb4f0fb0
Allocate pending segments at latest committed version (#15459)
The segment allocation algorithm reuses an already allocated pending segment if the new allocation request is made for the same parameters:

datasource
sequence name
same interval
same value of skipSegmentLineageCheck (false for batch append, true for streaming append)
same previous segment id (used only when skipSegmentLineageCheck = false)
The above parameters can thus uniquely identify a pending segment (enforced by the UNIQUE constraint on the sequence_name_prev_id_sha1 column in druid_pendingSegments metadata table).

This reuse is done in order to

allow replica tasks (in case of streaming ingestion) to use the same set of segment IDs.
allow re-run of a failed batch task to use the same segment ID and prevent unnecessary allocations
2023-12-14 16:18:39 +05:30
TestBoost 85af2c8340
only create used and unused segments once to make the test faster (#15533) 2023-12-12 09:31:04 +05:30
Clint Wylie 64fcb32bcf
add native 'array contains element' filter (#15366)
* add native arrayContainsElement filter to use array column element indexes
2023-11-29 03:33:00 -08:00
Clint Wylie 97623b408c
add optional 'castToType' parameter to 'auto' column schema (#15417)
* auto but.. with an expected type
2023-11-28 17:19:23 -08:00
George Shiqi Wu 3d1d26f824
Fix mmless ingestion and index tasks (#15372)
* Fix mmless ingestion and index tasks

* Move comment

* remove dup test
2023-11-28 10:06:07 -05:00
Karan Kumar a0188192de
Fixing failing compaction/parallel index jobs during upgrade due to new actions being available on the overlord. (#15430)
* Fixing failing compaction/parallel index jobs during upgrade due to new actions not available on the overlord.

* Fixing build

* Removing extra space.

* Fixing json getter.

* Review comments.
2023-11-25 13:50:29 +05:30
Kashif Faraz 67c7b6248c
Fix log typos, clean up some kill messages in SeekableStreamSupervisor (#15424)
Changes:
- Fix log `Got end of partition marker for partition [%s] from task [%s] in discoverTasks`
by fixing order of args
- Simplify in-line classes by using lambda
- Update kill task message from `Task [%s] failed to respond to [set end offsets]
 in a timely manner, killing task` to `Failed to set end offsets, killing task`
- Clean up tests
2023-11-24 16:09:10 +05:30
Vishesh Garg 4ab0b71513
Fix missing task failure error message on Overlord caused by MessageBodyWriter not found error on Middle Manager (#15412)
Fixes missing task failure error message on Overlord.

The error message was missing since TaskManagementResource#assignTask API wasn't annotated with @Produces(MediaType.APPLICATION_JSON) resulting in the response being treated as application/octet-stream, that in turn lead to MessageBodyWriter not found error on the middle manager. The exception is not logged on the middle manager itself since it happens even before entering the assignTask function -- while mapping arg Task -> MSQControllerTask.
2023-11-24 11:35:56 +05:30
Sachidananda Maharana 2f269fe065
Returning correct Response Code HTTP 429 when taskQueue reached maxSize (#15409)
Currently when we submit a task to druid and number of currently active tasks has already reached (druid.indexer.queue.maxSize) then 500 ISE is thrown as per shown in the screenshot in #15380.

This fix will return HTTP 429 Too Many Requests(with proper error message) instead of 500 ISE, when we submit a task and queueSize has reached.
2023-11-22 14:27:20 +05:30
Kashif Faraz 4ba3cf5221
Add test to verify sequence name of Kafka task (#15397)
* Add test to verify sequence name of Kafka and Kinesis tasks
2023-11-21 10:17:32 +05:30
AmatyaAvadhanula 77828bead4
Fetch active task payloads from memory (#15377)
The TaskQueue maintains a map of active task ids to tasks, which can be utilized to get active task payloads, before falling back to the metadata store.
2023-11-17 12:19:20 +05:30
AmatyaAvadhanula cdc192d38d
Prevent multiple attempts to publish segments for the same sequence (#14995)
* Prevent a race that may cause multiple attempts to publish segments for the same sequence
2023-11-16 14:21:26 +05:30
Krishna Anandan bedf246ed2
Fixing 1 flaky test in testAPIs() (#15375) 2023-11-15 18:38:20 +05:30
Abhishek Radhakrishnan 2e79fd56a7
MSQ generates tombstones honoring granularity specified in a `REPLACE` query. (#15243)
* MSQ generates tombstones honoring the query's granularity.

This change tweaks to only account for the infinite-interval tombstones.
For finite-interval tombstones, the MSQ query granualrity will be used
which is consistent with how MSQ works.

* more tests and some cleanup.

* checkstyle

* comment edits

* Throw TooManyBuckets fault based on review; add more tests.

* Add javadocs for both methods on reconciling the methods.

* review: Move testReplaceTombstonesWithTooManyBucketsThrowsException to MsqFaultsTest

* remove unused imports.

* Move TooManyBucketsException to indexing package for shared exception handling.

* lower max bucket for tests and fixup count

* Advance and count the iterator.

* checkstyle
2023-11-14 23:35:36 -08:00
George Shiqi Wu 130bfbfc6d
Revert "Separate task lifecycle from kubernetes/location lifecycle (#15133)" (#15346)
This reverts commit dc0b163e19.
2023-11-08 13:12:30 -05:00
George Shiqi Wu a8906b6ea0
Fix k8s task runner failure reporting (#15311)
* Fix k8s task runner failure reporting

* Fix reference

* add jsonignore

* PR changes
2023-11-03 21:28:46 -04:00
Clint Wylie 5d39b94149
allow compaction to work with spatial dimensions (#15321) 2023-11-03 11:27:50 -07:00
Gian Merlino d87d92bc43
Add system fields to input sources. (#15276)
* Add system fields to input sources.

Main changes:

1) The SystemField enum defines system fields "__file_uri", "__file_path",
   and "__file_bucket". They are associated with each input entity.

2) The SystemFieldInputSource interface can be added to any InputSource
   to make it system-field-capable. It sets up serialization of a list
   of configured "systemFields" in the JSON form of the input source, and
   provides a method getSystemFieldValue for computing the value of each
   system field. Cloud object, HDFS, HTTP, and Local now have this.

* Fix various LocalInputSource calls.

* Fix style stuff.

* Fixups.

* Fix tests and coverage.
2023-11-02 10:31:28 -07:00
AmatyaAvadhanula dc3213b05d
Fix used segment retrieval in Kill tasks (#15306)
Fix used segment retrieval in Kill tasks
2023-11-02 19:07:17 +05:30
Alexander Saydakov f1132d20c5
use datasketches-java 4.2.0 (#15257)
* use datasketches-java 4.2.0

* use exclusive mode

* fixed issues raised by CodeQL

* fixed issue raised by spotbugs

* fixed issues raised by intellij

* added missing import

* Update QuantilesSketchKeyCollector search mode and adjust tests.

* Update sizeOf functions and add unit tests

* Add unit tests

---------

Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
Co-authored-by: Adarsh Sanjeev <adarshsanjeev@gmail.com>
2023-10-26 16:28:33 -07:00
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
Gian Merlino 63ee69b4e8
Claim full support for Java 17. (#14384)
* Claim full support for Java 17.

No production code has changed, except the startup scripts.

Changes:

1) Allow Java 17 without DRUID_SKIP_JAVA_CHECK.

2) Include the full list of opens and exports on both Java 11 and 17.

3) Document that Java 17 is both supported and preferred.

4) Switch some tests from Java 11 to 17 to get better coverage on the
   preferred version.

* Doc update.

* Update errorprone.

* Update docker_build_containers.sh.

* Update errorprone in licenses.yaml.

* Add some more run-javas.

* Additional run-javas.

* Update errorprone.

* Suppress new errorprone error.

* Add exports and opens in ForkingTaskRunner for Java 11+.

Test, doc changes.

* Additional errorprone updates.

* Update for errorprone.

* Restore old fomatting in LdapCredentialsValidator.

* Copy bin/ too.

* Fix Java 15, 17 build line in docker_build_containers.sh.

* Update busybox image.

* One more java command.

* Fix interpolation.

* IT commandline refinements.

* Switch to busybox 1.34.1-glibc.

* POM adjustments, build and test one IT on 17.

* Additional debugging.

* Fix silly thing.

* Adjust command line.

* Add exports and opens one more place.

* Additional harmonization of strong encapsulation parameters.
2023-07-07 12:52:35 -07:00
Gian Merlino 021a01df45
RTR, HRTR: Fix incorrect maxLazyWorkers check in markLazyWorkers. (#14545)
Recently #14532 fixed a problem when maxLazyWorkers == 0 and lazyWorkers
starts out empty. Unfortunately, even after that patch, there remained
a more general version of this problem when maxLazyWorkers == lazyWorkers.size().
This patch fixes it.

I'm not sure if this would actually happen in production, because the
provisioning strategies do try to avoid calling markWorkersLazy until
previously-initiated terminations have finished. Nevertheless, it still
seems like a good thing to fix.
2023-07-07 10:08:12 -07:00
Kashif Faraz 40d0dc9e0e
Use separate executor to handle task updates in TaskQueue (#14533)
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.
2023-07-07 20:43:12 +05:30
Gian Merlino 1fe61bc869
ChangeRequestHttpSyncer: Don't wait 1ms when checking isInitialized(). (#14547)
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.
2023-07-07 05:54:39 -07:00
Kashif Faraz d63eff3b1b
Reduce contention in HttpRemoteTaskRunner.getKnownTasks() (#14541) 2023-07-07 13:43:59 +05:30
Gian Merlino 037f09bef2
HttpRemoteTaskRunner: Fix markLazyWorkers for maxLazyWorkers == 0. (#14532) 2023-07-06 11:51:04 -07:00
Kashif Faraz 87bb1b9709
Fix bug during initialization of HttpServerInventoryView (#14517)
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
2023-07-06 13:04:53 +05:30
AmatyaAvadhanula 609833c97b
Do not emit negative lag because of stale offsets (#14292)
The latest topic offsets are polled frequently and used to determine the lag based on the current offsets. However, when the offsets are stale (which can happen due to connection issues commonly), we may see a negative lag .

This PR prevents emission of metrics when the offsets are stale and at least one of the partitions has a negative lag.
2023-07-05 14:44:23 +05:30
Clint Wylie 277aaa5c57
remove druid.processing.columnCache.sizeBytes and CachingIndexed, combine string column implementations (#14500)
* 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
2023-07-02 19:37:15 -07:00
Karan Kumar cb3a9d2b57
Adding Interactive API's for MSQ engine (#14416)
This PR aims to expose a new API called
"@path("/druid/v2/sql/statements/")" which takes the same payload as the current "/druid/v2/sql" endpoint and allows users to fetch results in an async manner.
2023-06-28 17:51:58 +05:30
Clint Wylie 31b9d5695d
Extend InitializedNullHandlingTest instead of NullHandlingTest (#14467)
NullHandlingTest is an actual test, it shouldn't be used as a base class
2023-06-22 15:01:50 +05:30
imply-cheddar cfd07a95b7
Errors take 3 (#14004)
Introduce DruidException, an exception whose goal in life is to be delivered to a user.

DruidException itself has javadoc on it to describe how it should be used.  This commit both introduces the Exception and adjusts some of the places that are generating exceptions to generate DruidException objects instead, as a way to show how the Exception should be used.

This work was a 3rd iteration on top of work that was started by Paul Rogers.  I don't know if his name will survive the squash-and-merge, so I'm calling it out here and thanking him for starting on this.
2023-06-19 01:11:13 -07:00
George Shiqi Wu 64af9bfe5b
Add groupId to metrics (#14402)
* Add group id as a dimension

* Revert changes

* Add to forking task runner

* Add missing metrics

* Fix indenting

* revert metrics

* Fix indentation
2023-06-16 09:28:16 -07:00
Gian Merlino 85656a467c
MSQ: Load broadcast tables on workers. (#14437)
They were not previously loaded because supportsQueries was false.
This patch sets supportsQueries to true, and clarifies in Task
javadocs that supportsQueries can be true for tasks that aren't
directly queryable over HTTP.
2023-06-16 12:02:20 +05:30
Clint Wylie 8454cc619a
auto columns fixes (#14422)
changes:
* auto columns no longer participate in generic 'null column' handling, this was a mistake to try to support and caused ingestion failures due to mismatched ColumnFormat, and will be replaced in the future with nested common format constant column functionality (not in this PR)
* fix bugs with auto columns which contain empty objects, empty arrays, or primitive types mixed with either of these empty constructs
* fix bug with bound filter when upper is null equivalent but is strict
2023-06-14 08:57:06 -07:00
Kashif Faraz 6e158704cb
Do not retry INSERT task into metadata if max_allowed_packet limit is violated (#14271)
Changes
- Add a `DruidException` which contains a user-facing error message, HTTP response code
- Make `EntryExistsException` extend `DruidException`
- If metadata store max_allowed_packet limit is violated while inserting a new task, throw
`DruidException` with response code 400 (bad request) to prevent retries
- Add `SQLMetadataConnector.isRootCausePacketTooBigException` with impl for MySQL
2023-06-10 12:15:44 +05:30
Harini Rajendran 4ff6026d30
Adding SegmentMetadataEvent and publishing them via KafkaEmitter (#14281)
In this PR, we are enhancing KafkaEmitter, to emit metadata about published segments (SegmentMetadataEvent) into a Kafka topic. This segment metadata information that gets published into Kafka, can be used by any other downstream services to query Druid intelligently based on the segments published. The segment metadata gets published into kafka topic in json string format similar to other events.
2023-06-02 21:28:26 +05:30
Andreas Maechler 45014bd5b4
Handle all types of exceptions when initializing input source in sampler API (#14355)
The sampler API returns a `400 bad request` response if it encounters a `SamplerException`.
Otherwise, it returns a generic `500 Internal server error` response, with the message
"The RuntimeException could not be mapped to a response, re-throwing to the HTTP container".

This commit updates `RecordSupplierInputSource` to handle all types of exceptions instead of just
`InterruptedException`and wrap them in a `SamplerException` so that the actual error is
propagated back to the user.
2023-06-02 19:43:53 +05:30
zachjsh 04a82da63d
Input source security fixes (#14266)
It was found that several supported tasks / input sources did not have implementations for the methods used by the input source security feature, causing these tasks and input sources to fail when used with this feature. This pr adds the needed missing implementations. Also securing the sampling endpoint with input source security, when enabled.
2023-06-01 16:37:19 -07:00
Rishabh Singh 2086ff88bc
Add logging for task stop operations (#14192)
Log more details when task cannot be stopped for various reasons
2023-05-30 18:50:52 +05:30
Alexander Saydakov 4131c0df13
use the latest datasketches-java-4.0.0 (#14334)
* use the latest datasketches-java-4.0.0

* updated versions of datasketches

* adjusted expectation

* fixed the expectations

---------

Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
2023-05-27 22:19:18 -07:00
Kashif Faraz 0cde3a8b52
Fix regression in batch segment allocation (#14337)
* Improve batch segment allocation logs

* Fix batch seg alloc regression

* Fix logs

* Fix logs

* Fix tests and logs
2023-05-25 22:34:54 -07:00
AmatyaAvadhanula e9913abbbf
Add new lock types: APPEND and REPLACE (#14258)
* Add new lock types: APPEND and REPLACE
2023-05-14 22:38:32 -07:00
imply-cheddar f9861808bc
Be able to load segments on Peons (#14239)
* Be able to load segments on Peons

This change introduces a new config on WorkerConfig
that indicates how many bytes of each storage
location to use for storage of a task.  Said config
is divided up amongst the locations and slots
and then used to set TaskConfig.tmpStorageBytesPerTask

The Peons use their local task dir and
tmpStorageBytesPerTask as their StorageLocations for
the SegmentManager such that they can accept broadcast
segments.
2023-05-12 16:51:00 -07:00
Kashif Faraz ba11b3d462
Refactor: Add OverlordDuty to replace OverlordHelper and align with CoordinatorDuty (#14235)
Changes:
- Replace `OverlordHelper` with `OverlordDuty` to align with `CoordinatorDuty`
  - Each duty has a `run()` method and defines a `Schedule` with an initial delay and period.
  - Update existing duties `TaskLogAutoCleaner` and `DurableStorageCleaner`
- Add utility class `Configs`
- Update log, error messages and javadocs
- Other minor style improvements
2023-05-12 22:39:56 +05:30
AmatyaAvadhanula 47e48ee657
Remove incorrect optimization (#14246) 2023-05-11 00:54:41 -07:00
Clint Wylie e833a4700d
suppress hadoop3 cve that seem not applicable to us (#14252) 2023-05-10 23:08:05 -07:00
Abhishek Radhakrishnan 46dabab36d
Fix NPE in test parse exception report. Add more tests with different thresholds. (#14209) 2023-05-05 10:05:41 -07:00
Abhishek Radhakrishnan 68f908e511
Fix uncaught `ParseException` when reading Avro from Kafka (#14183)
In StreamChunkParser#parseWithInputFormat, we call byteEntityReader.read() without handling a potential ParseException, which is thrown during this function call by the delegate AvroStreamReader#intermediateRowIterator.
A ParseException can be thrown if an Avro stream has corrupt data or data that doesn't conform to the schema specified or for other decoding reasons. This exception if uncaught, can cause ingestion to fail.
2023-05-04 12:35:36 +05:30
AmatyaAvadhanula ac7181bbda
Persist supervisor spec only after successful start (#14150)
* Persist spec after successful start

* Fix checkstyle.

* checkstyle after mvn install
2023-05-03 18:27:39 +05:30
Clint Wylie 90ea192d9c
fix bugs with auto encoded long vector deserializers (#14186)
This PR fixes an issue when using 'auto' encoded LONG typed columns and the 'vectorized' query engine. These columns use a delta based bit-packing mechanism, and errors in the vectorized reader would cause it to incorrectly read column values for some bit sizes (1 through 32 bits). This is a regression caused by #11004, which added the optimized readers to improve performance, so impacts Druid versions 0.22.0+.

While writing the test I finally got sad enough about IndexSpec not having a "builder", so I made one, and switched all the things to use it. Apologies for the noise in this bug fix PR, the only real changes are in VSizeLongSerde, and the tests that have been modified to cover the buggy behavior, VSizeLongSerdeTest and ExpressionVectorSelectorsTest. Everything else is just cleanup of IndexSpec usage.
2023-05-01 11:49:27 +05:30
Suneet Saldanha 84c11df980
Make LoggingEmitter more useful by using Markers (#14121)
* Make LoggingEmitter more useful

* Skip code coverage for facade classes

* fix spellcheck

* code review

* fix dependency

* logging.md

* fix checkstyle

* Add back jacoco version to main pom
2023-04-27 15:06:06 -07:00
Tejaswini Bandlamudi 774073b2e7
Update Hadoop3 as default build version (#14005)
Hadoop 2 often causes red security scans on Druid distribution because of the dependencies it brings. We want to move away from Hadoop 2 and provide Hadoop 3 distribution available. Switch druid to building with Hadoop 3 by default. Druid will still be compatible with Hadoop 2 and users can build hadoop-2 compatible distribution using hadoop2 profile.
2023-04-26 12:52:51 +05:30
Gian Merlino a7d4162195
Compaction: Block input specs not aligned with segmentGranularity. (#14127)
* Compaction: Block input specs not aligned with segmentGranularity.

When input intervals are not aligned with segmentGranularity, data may be
overshadowed if it lies in the space between the input intervals and the
output segmentGranularity.

In MSQ REPLACE, this is a validation error. IMO the same behavior makes
sense for compaction tasks. In case anyone was depending on the ability
to compact nonaligned intervals, a configuration parameter
allowNonAlignedInterval is provided. I don't expect it to be used much.

* Remove unused.

* ITCompactionTaskTest uses non-aligned intervals.
2023-04-25 17:06:16 -07:00
Nicholas Lippis 9d4cc501f7
return task status reported by peon (#14040)
* return task status reported by peon

* Write TaskStatus to file in AbstractTask.cleanUp

* Get TaskStatus from task log

* Fix merge conflicts in AbstractTaskTest

* Add unit tests for TaskLogPusher, TaskLogStreamer, NoopTaskLogs to satisfy code coverage

* Add license headerss

* Fix style

* Remove unknown exception declarations
2023-04-24 12:05:39 -07:00
TSFenwick accd5536df
Allow for Log4J to be configured for peons but still ensure console logging is enforced (#14094)
* Allow for Log4J to be configured for peons but still ensure console logging is enforced

This change will allow for log4j to be configured for peons but require console logging is still
configured for them to ensure peon logs are saved to deep storage.

Also fixed the test ConsoleLoggingEnforcementTest to use a valid appender for the non console
Config as the previous config was incorrect and would never return a logger.

* fix checkstyle

* add warning to logger when it overwrites all loggers to be console

* optimize calls for altering logging config for ConsoleLoggingEnforcementConfigurationFactory

add getName to the druid logger class

* update docs, and error message

* edit docs to be more clear

* fix checkstyle issues

* CI fixes - LoggerTest code coverage and fix spelling issue for logging docs
2023-04-24 10:41:56 -07:00
Clint Wylie 887f8db1b5
preserve explicitly specified dimension schema in "logical" schema of sampler response (#14144) 2023-04-23 21:28:05 +05:30
zachjsh 04da0102cb
KillTask should return empty inputSource resources (#14106)
### Description

This pr fixes a few bugs found with the inputSource security feature.

1. `KillUnusedSegmentsTask` previously had no definition for the `getInputSourceResources`, which caused an unsupportedOperationException to be thrown when this task type was submitted with the inputSource security feature enabled. This task type should not require any input source specific resources, so returning an empty set for this task type now.

2. Fixed a bug where when the input source type security feature is enabled, all of the input source type specific resources used where authenticated against:

`{"resource": {"name": "EXTERNAL", "type": "{INPUT_SOURCE_TYPE}"}, "action": "READ"}`

When they should be instead authenticated against:

`{"resource": {"name": "{INPUT_SOURCE_TYPE}", "type": "EXTERNAL"}, "action": "READ"}`

3. fixed bug where supervisor tasks were not authenticated against the specific input source types used, if input source security feature was enabled.
2023-04-18 15:27:16 -04:00
Adarsh Sanjeev a7d5c64aeb
Move MSQ temporary storage to a runtime parameter instead of being configured from query context (#14061)
* 
    Adds new run time parameter druid.indexer.task.tmpStorageBytesPerTask. This sets a limit for the amount of temporary storage disk space used by tasks. This limit is currently only respected by MSQ tasks.
*   Removes query context parameters intermediateSuperSorterStorageMaxLocalBytes and composedIntermediateSuperSorterStorageEnabled. Composed intermediate super sorter (which was enabled by composedIntermediateSuperSorterStorageEnabled) is now enabled automatically if durableShuffleStorage is set to true. intermediateSuperSorterStorageMaxLocalBytes is calculated from the limit set by the run time parameter druid.indexer.task.tmpStorageBytesPerTask.
2023-04-18 16:56:51 +05:30
Rohan Garg 086b2b8efe
Log merge and push timings for PartialGenericSegmentMergeTask (#14089) 2023-04-18 11:51:26 +05:30
imply-cheddar aaa6cc1883
Make the tasks run with only a single directory (#14063)
* Make the tasks run with only a single directory

There was a change that tried to get indexing to run on multiple disks
It made a bunch of changes to how tasks run, effectively hiding the
"safe" directory for tasks to write files into from the task code itself
making it extremely difficult to do anything correctly inside of a task.

This change reverts those changes inside of the tasks and makes it so that
only the task runners are the ones that make decisions about which
mount points should be used for storing task-related files.

It adds the config druid.worker.baseTaskDirs which can be used by the
task runners to know which directories they should schedule tasks inside of.
The TaskConfig remains the authoritative source of configuration for where
and how an individual task should be operating.
2023-04-13 00:45:02 -07:00
Clint Wylie 179e2e8108
adjust useSchemaDiscovery to also include the behavior of includeAllDimensions to support partial schema declaration without having to set two flags (#14076) 2023-04-12 23:12:49 -07:00
Clint Wylie 9ed8beca5e
bug fixes and add support for boolean inputs to classic long dimension indexer (#14069)
changes:
* adds support for boolean inputs to the classic long dimension indexer, which plays nice with LONG being the semi official boolean type in Druid, and even nicer when druid.expressions.useStrictBooleans is set to true, since the sampler when using the new 'auto' schema when 'useSchemaDiscovery' is specified on the dimensions spec will call the type out as LONG
* fix bugs with sampler response and new schema discovery stuff incorrectly using classic 'json' type for the logical schema instead of the new 'auto' type
2023-04-11 20:49:52 -07:00
Clint Wylie 1aef72aa7e
Bump up the version in pom to 27.0.0 in preparation of release (#14051) 2023-04-10 14:56:59 +05:30
Karan Kumar 8712098301
Fixing overlord unable to become a leader when syncing the lock from metadata store. (#14038) 2023-04-10 12:37:31 +05:30
zachjsh 5c0221375c
Allow for Input source security in native task layer (#14003)
Fixes #13837.

### Description

This change allows for input source type security in the native task layer.

To enable this feature, the user must set the following property to true:

`druid.auth.enableInputSourceSecurity=true`

The default value for this property is false, which will continue the existing functionality of needing authorization to write to the respective datasource.

When this config is enabled, the users will be required to be authorized for the following resource action, in addition to write permission on the respective datasource.

`new ResourceAction(new Resource(ResourceType.EXTERNAL, {INPUT_SOURCE_TYPE}, Action.READ`

where `{INPUT_SOURCE_TYPE}` is the type of the input source being used;, http, inline, s3, etc..

Only tasks that provide a non-default implementation of the `getInputSourceResources` method can be submitted when config `druid.auth.enableInputSourceSecurity=true` is set. Otherwise, a 400 error will be thrown.
2023-04-06 13:13:09 -04:00
Clint Wylie 1c8a184677
add null safety checks for DiscoveryDruidNode services for more resilient http server and task views (#13930)
* add null safety checks for DiscoveryDruidNode services for more resilient http server and task vi
2023-04-05 02:45:39 -07:00
Clint Wylie d21babc5b8
remix nested columns (#14014)
changes:
* introduce ColumnFormat to separate physical storage format from logical type. ColumnFormat is now used instead of ColumnCapabilities to get column handlers for segment creation
* introduce new 'auto' type indexer and merger which produces a new common nested format of columns, which is the next logical iteration of the nested column stuff. Essentially this is an automatic type column indexer that produces the most appropriate column for the given inputs, making either STRING, ARRAY<STRING>, LONG, ARRAY<LONG>, DOUBLE, ARRAY<DOUBLE>, or COMPLEX<json>.
* revert NestedDataColumnIndexer, NestedDataColumnMerger, NestedDataColumnSerializer to their version pre #13803 behavior (v4) for backwards compatibility
* fix a bug in RoaringBitmapSerdeFactory if anything actually ever wrote out an empty bitmap using toBytes and then later tried to read it (the nerve!)
2023-04-04 17:51:59 -07:00
Clint Wylie 518698a952
lower segment heap footprint and fix bug with expression type coercion (#14002) 2023-03-31 13:53:22 -07:00
kaijianding 13ffeb50ba
should retry when failed to pause realtime task (#11515) 2023-03-25 19:03:13 +05:30
Kashif Faraz b7752a909c
Enable round-robin segment assignment and batch segment allocation by default (#13942)
Changes:
- Set `useRoundRobinSegmentAssignment` in coordinator dynamic config to `true` by default.
- Set `batchSegmentAllocation` in `TaskLockConfig` (used in Overlord runtime properties) to `true` by default.
2023-03-22 08:20:01 +05:30
Gian Merlino 1c7a03a47b
Lower default maxRowsInMemory for realtime ingestion. (#13939)
* Lower default maxRowsInMemory for realtime ingestion.

The thinking here is that for best ingestion throughput, we want
intermediate persists to be as big as possible without using up all
available memory. So, we rely mainly on maxBytesInMemory. The default
maxRowsInMemory (1 million) is really just a safety: in case we have
a large number of very small rows, we don't want to get overwhelmed
by per-row overheads.

However, maximum ingestion throughput isn't necessarily the primary
goal for realtime ingestion. Query performance is also important. And
because query performance is not as good on the in-memory dataset, it's
helpful to keep it from growing too large. 150k seems like a reasonable
balance here. It means that for a typical 5 million row segment, we
won't trigger more than 33 persists due to this limit, which is a
reasonable number of persists.

* Update tests.

* Update server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

* Fix test.

* Fix link.

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2023-03-21 10:36:36 -07:00
Laksh Singla c16d9da35a
Improve documentation for tombstone generation and minor improvement (#13907)
* As a follow up to #13893, this PR improves the comments added along with examples for the code, as well as adds handling for an edge case where the generated tombstone boundaries were overshooting the bounds of MIN_TIME (or MAX_TIME).
2023-03-10 06:59:51 +05:30
Gian Merlino fe9d0c46d5
Improve memory efficiency of WrappedRoaringBitmap. (#13889)
* Improve memory efficiency of WrappedRoaringBitmap.

Two changes:

1) Use an int[] for sizes 4 or below.
2) Remove the boolean compressRunOnSerialization. Doesn't save much
   space, but it does save a little, and it isn't adding a ton of value
   to have it be configurable. It was originally configurable in case
   anything broke when enabling it, but it's been a while and nothing
   has broken.

* Slight adjustment.

* Adjust for inspection.

* Updates.

* Update snaps.

* Update test.

* Adjust test.

* Fix snaps.
2023-03-09 15:48:02 -08:00
Laksh Singla dc67296e9d
Fix for OOM in the Tombstone generating logic in MSQ (#13893)
fix OOMs using a different logic for generating tombstones

---------

Co-authored-by: Paul Rogers <paul-rogers@users.noreply.github.com>
2023-03-08 21:38:08 -08:00
Clint Wylie 68db39d08a
fix ci (#13901)
This PR is #13899 plus spotbugs fix to fix the failures introduced by #13815
2023-03-08 16:55:47 +05:30
Nicholas Lippis faac43eabe
Use base task dir in kubernetes task runner (#13880)
* Use TaskConfig to get task dir in KubernetesTaskRunner

* Use the first path specified in baseTaskDirPaths instead of deprecated baseTaskDirPath

* Use getBaseTaskDirPaths in generate command
2023-03-07 15:30:42 -07:00
Karan Kumar 65c3954942
Adding forbidden api for Properties#get() and Properties#getOrDefault() (#13882)
Properties#getOrDefault method does not check the default map for values where as Properties#getProperty() does.
2023-03-06 10:42:04 +05:30
Tejaswini Bandlamudi 7103cb4b9d
Removes FiniteFirehoseFactory and its implementations (#12852)
The FiniteFirehoseFactory and InputRowParser classes were deprecated in 0.17.0 (#8823) in favor of InputSource & InputFormat. This PR removes the FiniteFirehoseFactory and all its implementations along with classes solely used by them like Fetcher (Used by PrefetchableTextFilesFirehoseFactory). Refactors classes including tests using FiniteFirehoseFactory to use InputSource instead.
Removing InputRowParser may not be as trivial as many classes that aren't deprecated depends on it (with no alternatives), like EventReceiverFirehoseFactory. Hence FirehoseFactory, EventReceiverFirehoseFactory, and Firehose are marked deprecated.
2023-03-02 18:07:17 +05:30
Laksh Singla ca68fd93a6
Generate tombstones when running MSQ's replace (#13706)
*When running REPLACE queries, the segments which contain no data are dropped (marked as unused). This PR aims to generate tombstones in place of segments which contain no data to mark their deletion, as is the behavior with the native ingestion.

This will cause InsertCannotReplaceExistingSegmentFault to be removed since it was generated if the interval to be marked unused didn't fully overlap one of the existing segments to replace.
2023-03-01 12:01:30 +05:30
Clint Wylie 1d8fff4096
sampler + type detection = bff (#13711)
* sampler + type detection = bff
* split logical and physical dimensions, tidy up
2023-02-28 04:14:30 -08:00
Abhishek Agarwal d2dbb8b2c0
Fix infinite checkpointing between tasks and overlord (#13825)
If the intermediate handoff period is less than the task duration and there is no new data in the input topic, task will continuously checkpoint the same offsets again and again. This PR fixes that bug by resetting the checkpoint time even when the task receives the same end offset request again.
2023-02-22 19:25:59 +05:30
Clint Wylie 08b5951cc5
merge druid-core, extendedset, and druid-hll into druid-processing to simplify everything (#13698)
* merge druid-core, extendedset, and druid-hll into druid-processing to simplify everything
* fix poms and license stuff
* mockito is evil
* allow reset of JvmUtils RuntimeInfo if tests used static injection to override
2023-02-17 14:27:41 -08:00
Paul Rogers 333196d207
Code cleanup & message improvements (#13778)
* Misc cleanup edits

Correct spacing
Add type parameters
Add toString() methods to formats so tests compare correctly
IT doc revisions
Error message edits
Display UT query results when tests fail

* Edit

* Build fix

* Build fixes
2023-02-15 15:22:54 +05:30
Suneet Saldanha 714ac07b52
Allow users to add additional metadata to ingestion metrics (#13760)
* Allow users to add additional metadata to ingestion metrics

When submitting an ingestion spec, users may pass a map of metadata
in the ingestion spec config that will be added to ingestion metrics.

This will make it possible for operators to tag metrics with other
metadata that doesn't necessarily line up with the existing tags
like taskId.

Druid clusters that ingest these metrics can take advantage of the
nested data columns feature to process this additional metadata.

* rename to tags

* docs

* tests

* fix test

* make code cov happy

* checkstyle
2023-02-08 18:07:23 -08:00
AmatyaAvadhanula 34c04daa9f
Fix infinite iteration in http sync monitoring (#13731)
* Fix infinite iteration in http task runner

* Fix infinite iteration in http server view

* Add tests
2023-02-08 15:14:11 +05:30
AmatyaAvadhanula 0cf1fc3d55
Indexing on multiple disks (#13476)
* Initial commit

* Simple UTs

* Parameterize tests

* Parameterized tests for k8s task runner

* Fix restore bug

* Refactor TaskStorageDirTracker

* Change CliPeon args
2023-02-08 11:31:34 +05:30
Churro f022a9f246
When a task fails and doesn't throw an exception, report it correctly… (#13668)
* When a task fails and doesn't throw an exception, report it correctly in mm-less druid

* Removing unthrown exception from test
2023-02-02 09:04:18 -08:00
Kashif Faraz 7c188d80b8
Make batch segment allocation logs less noisy (#13725) 2023-02-02 09:54:53 +05:30
Clint Wylie fb26a1093d
discover nested columns when using nested column indexer for schemaless ingestion (#13672)
* discover nested columns when using nested column indexer for schemaless
* move useNestedColumnIndexerForSchemaDiscovery from AppendableIndexSpec to DimensionsSpec
2023-01-18 12:57:28 -08:00
Gian Merlino 182c4fad29
Kinesis: More robust default fetch settings. (#13539)
* Kinesis: More robust default fetch settings.

1) Default recordsPerFetch and recordBufferSize based on available memory
   rather than using hardcoded numbers. For this, we need an estimate
   of record size. Use 10 KB for regular records and 1 MB for aggregated
   records. With 1 GB heaps, 2 processors per task, and nonaggregated
   records, recordBufferSize comes out to the same as the old
   default (10000), and recordsPerFetch comes out slightly lower (1250
   instead of 4000).

2) Default maxRecordsPerPoll based on whether records are aggregated
   or not (100 if not aggregated, 1 if aggregated). Prior default was 100.

3) Default fetchThreads based on processors divided by task count on
   Indexers, rather than overall processor count.

4) Additionally clean up the serialized JSON a bit by adding various
   JsonInclude annotations.

* Updates for tests.

* Additional important verify.
2023-01-13 11:03:54 +05:30
Clint Wylie b5b740bbbb
allow using nested column indexer for schema discovery (#13653)
* single typed "root" only nested columns now mimic "regular" columns of those types
* incremental index can now use nested column indexer instead of string indexer for discovered columns
2023-01-12 18:31:12 -08:00
Adarsh Sanjeev 0a486c3bcf
Update forbidden apis with fixed executor (#13633)
* Update forbidden apis with fixed executor
2023-01-12 15:34:36 +05:30
Karan Kumar 56076d33fb
Worker retry for MSQ task (#13353)
* Initial commit.

* Fixing error message in retry exceeded exception

* Cleaning up some code

* Adding some test cases.

* Adding java docs.

* Finishing up state test cases.

* Adding some more java docs and fixing spot bugs, intellij inspections

* Fixing intellij inspections and added tests

* Documenting error codes

* Migrate current integration batch tests to equivalent MSQ tests (#13374)

* Migrate current integration batch tests to equivalent MSQ tests using new IT framework

* Fix build issues

* Trigger Build

* Adding more tests and addressing comments

* fixBuildIssues

* fix dependency issues

* Parameterized the test and addressed comments

* Addressing comments

* fixing checkstyle errors

* Adressing comments

* Adding ITTest which kills the worker abruptly

* Review comments phase one

* Adding doc changes

* Adjusting for single threaded execution.

* Adding Sequential Merge PR state handling

* Merge things

* Fixing checkstyle.

* Adding new context param for fault tolerance.
Adding stale task handling in sketchFetcher.
Adding UT's.

* Merge things

* Merge things

* Adding parameterized tests
Created separate module for faultToleranceTests

* Adding missed files

* Review comments and fixing tests.

* Documentation things.

* Fixing IT

* Controller impl fix.

* Fixing racy WorkerSketchFetcherTest.java exception handling.

Co-authored-by: abhagraw <99210446+abhagraw@users.noreply.github.com>
Co-authored-by: Karan Kumar <cryptoe@karans-mbp.lan>
2023-01-11 07:38:29 +05:30
Maytas Monsereenusorn 62a105ee65
Add context to HadoopIngestionSpec (#13624)
* add context to HadoopIngestionSpec

* fix alert
2023-01-09 14:37:02 -10:00
AmatyaAvadhanula af05cfa78c
Fix shutdown in httpRemote task runner (#13558)
* Fix shutdown in httpRemote task runner

* Add UT
2022-12-22 14:50:04 +05:30
imply-cheddar 089d8da561
Support Framing for Window Aggregations (#13514)
* Support Framing for Window Aggregations

This adds support for framing  over ROWS
for window aggregations.

Still not implemented as yet:
1. RANGE frames
2. Multiple different frames in the same query
3. Frames on last/first functions
2022-12-14 18:04:39 -08:00
Kashif Faraz 58a3acc2c4
Add InputStats to track bytes processed by a task (#13520)
This commit adds a new class `InputStats` to track the total bytes processed by a task.
The field `processedBytes` is published in task reports along with other row stats.

Major changes:
- Add class `InputStats` to track processed bytes
- Add method `InputSourceReader.read(InputStats)` to read input rows while counting bytes.
> Since we need to count the bytes, we could not just have a wrapper around `InputSourceReader` or `InputEntityReader` (the way `CountableInputSourceReader` does) because the `InputSourceReader` only deals with `InputRow`s and the byte information is already lost.
- Classic batch: Use the new `InputSourceReader.read(inputStats)` in `AbstractBatchIndexTask`
- Streaming: Increment `processedBytes` in `StreamChunkParser`. This does not use the new `InputSourceReader.read(inputStats)` method.
- Extend `InputStats` with `RowIngestionMeters` so that bytes can be exposed in task reports

Other changes:
- Update tests to verify the value of `processedBytes`
- Rename `MutableRowIngestionMeters` to `SimpleRowIngestionMeters` and remove duplicate class
- Replace `CacheTestSegmentCacheManager` with `NoopSegmentCacheManager`
- Refactor `KafkaIndexTaskTest` and `KinesisIndexTaskTest`
2022-12-13 18:54:42 +05:30
somu-imply 7682b0b6b1
Analysis refactor (#13501)
Refactor DataSource to have a getAnalysis method()

This removes various parts of the code where while loops and instanceof
checks were being used to walk through the structure of DataSource objects
in order to build a DataSourceAnalysis.  Instead we just ask the DataSource
for its analysis and allow the stack to rebuild whatever structure existed.
2022-12-12 17:35:44 -08:00
Gian Merlino de5a4bafcb
Zero-copy local deep storage. (#13394)
* Zero-copy local deep storage.

This is useful for local deep storage, since it reduces disk usage and
makes Historicals able to load segments instantaneously.

Two changes:

1) Introduce "druid.storage.zip" parameter for local storage, which defaults
   to false. This changes default behavior from writing an index.zip to writing
   a regular directory. This is safe to do even during a rolling update, because
   the older code actually already handled unzipped directories being present
   on local deep storage.

2) In LocalDataSegmentPuller and LocalDataSegmentPusher, use hard links
   instead of copies when possible. (Generally this is possible when the
   source and destination directory are on the same filesystem.)
2022-12-12 17:28:24 -08:00
Kashif Faraz 69951273b8
Fix typo in metric name (#13521) 2022-12-08 06:41:23 +05:30
Kashif Faraz c7229fc787
Limit max batch size for segment allocation, add docs (#13503)
Changes:
- Limit max batch size in `SegmentAllocationQueue` to 500
- Rename `batchAllocationMaxWaitTime` to `batchAllocationWaitTime` since the actual
wait time may exceed this configured value.
- Replace usage of `SegmentInsertAction` in `TaskToolbox` with `SegmentTransactionalInsertAction`
2022-12-07 10:07:14 +05:30
Gian Merlino fda0a1aadd
Set chatAsync default to true. (#13491)
This functionality was originally added in #13354.
2022-12-05 20:53:59 -08:00
Kashif Faraz 45a8fa280c
Add SegmentAllocationQueue to batch SegmentAllocateActions (#13369)
In a cluster with a large number of streaming tasks (~1000), SegmentAllocateActions 
on the overlord can often take very long intervals of time to finish thus causing spikes 
in the `task/action/run/time`. This may result in lag building up while a task waits for a
segment to get allocated.

The root causes are:
- large number of metadata calls made to the segments and pending segments tables
- `giant` lock held in `TaskLockbox.tryLock()` to acquire task locks and allocate segments

Since the contention typically arises when several tasks of the same datasource try
to allocate segments for the same interval/granularity, the allocation run times can be
improved by batching the requests together.

Changes
- Add flags
   - `druid.indexer.tasklock.batchSegmentAllocation` (default `false`)
   - `druid.indexer.tasklock.batchAllocationMaxWaitTime` (in millis) (default `1000`)
- Add methods `canPerformAsync` and `performAsync` to `TaskAction`
- Submit each allocate action to a `SegmentAllocationQueue`, and add to correct batch
- Process batch after `batchAllocationMaxWaitTime`
- Acquire `giant` lock just once per batch in `TaskLockbox`
- Reduce metadata calls by batching statements together and updating query filters
- Except for batching, retain the whole behaviour (order of steps, retries, etc.)
- Respond to leadership changes and fail items in queue when not leader
- Emit batch and request level metrics
2022-12-05 14:00:07 +05:30
AmatyaAvadhanula cc307e4c29
Fix needless task shutdown on leader switch (#13411)
* Fix needless task shutdown on leader switch

* Add unit test

* Fix style

* Fix UTs
2022-12-01 18:31:08 +05:30
Tejaswini Bandlamudi b091b32f21
Fixes reindexing bug with filter on long column (#13386)
* fixes BlockLayoutColumnarLongs close method to nullify internal buffer.

* fixes other BlockLayoutColumnar supplier close methods to nullify internal buffers.

* fix spotbugs
2022-11-25 19:22:48 +05:30
Kashif Faraz 7cf761cee4
Prepare master branch for next release, 26.0.0 (#13401)
* Prepare master branch for next release, 26.0.0

* Use docker image for druid 24.0.1

* Fix version in druid-it-cases pom.xml
2022-11-22 15:31:01 +05:30
Gian Merlino bfffbabb56
Async task client for SeekableStreamSupervisors. (#13354)
Main changes:
1) Convert SeekableStreamIndexTaskClient to an interface, move old code
   to SeekableStreamIndexTaskClientSyncImpl, and add new implementation
   SeekableStreamIndexTaskClientAsyncImpl that uses ServiceClient.
2) Add "chatAsync" parameter to seekable stream supervisors that causes
   the supervisor to use an async task client.
3) In SeekableStreamSupervisor.discoverTasks, adjust logic to avoid making
   blocking RPC calls in workerExec threads.
4) In SeekableStreamSupervisor generally, switch from Futures.successfulAsList
   to FutureUtils.coalesce, so we can better capture the errors that occurred
   with contacting individual tasks.

Other, related changes:
1) Add ServiceRetryPolicy.retryNotAvailable, which controls whether
   ServiceClient retries unavailable services. Useful since we do not
   want to retry calls unavailable tasks within the service client. (The
   supervisor does its own higher-level retries.)
2) Add FutureUtils.transformAsync, a more lambda friendly version of
   Futures.transform(f, AsyncFunction).
3) Add FutureUtils.coalesce. Similar to Futures.successfulAsList, but
   returns Either instead of using null on error.
4) Add JacksonUtils.readValue overloads for JavaType and TypeReference.
2022-11-21 19:20:26 +05:30
Gian Merlino b8ca03d283
SeekableStreamSupervisor: Unique type name for GracefulShutdownNotice. (#13399)
Allows GracefulShutdownNotice to be differentiated from ShutdownNotice.
2022-11-21 19:10:14 +05:30
AmatyaAvadhanula de566eb0db
Fix shared lock acquisition criteria (#13390)
Currently, a shared lock is acquired only when all other locks are also shared locks.

This commit updates the behaviour and acquires a shared lock only if all locks
of equal or higher priority are either shared locks or are already revoked.
The lock type of locks with lower priority does not matter as they can be revoked.
2022-11-21 15:31:38 +05:30
Gian Merlino c61313f4c4
Quieter streaming supervisors. (#13392)
Eliminates two common sources of noise with Kafka supervisors that have
large numbers of tasks and partitions:

1) Log the report at DEBUG rather than INFO level at each run cycle.
   It can get quite large, and can be retrieved via API when needed.

2) Use log4j2.xml to quiet down the org.apache.kafka.clients.consumer.internals
   package. Avoids a log message per-partition per-minute as part of seeking
   to the latest offset in the reporting thread. In the tasks, where this
   sort of logging might be more useful, we have another log message with
   the same information: "Seeking partition[%s] to[%s]".
2022-11-20 23:53:17 -08:00
Rohan Garg 6ccf31490e
Allow injection of node-role set to all non base modules (#13371) 2022-11-18 12:12:03 +05:30
Gian Merlino e78f648023
SeekableStreamSupervisor: Don't enqueue duplicate notices. (#13334)
* SeekableStreamSupervisor: Don't enqueue duplicate notices.

Similar goal to #12018, but more aggressive. Don't enqueue a notice at
all if it is equal to one currently in the queue.

* Adjustments from review.

* Update indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/NoticesQueueTest.java

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2022-11-11 01:54:01 -08:00
Gian Merlino 77478f25fb
Add taskActionType dimension to task/action/run/time. (#13333)
* Add taskActionType dimension to task/action/run/time.

* Spelling.
2022-11-11 12:00:08 +05:30
AmatyaAvadhanula fb23e38aa7
Fix messageGap emission (#13346)
* Fix messageGap emission

* Do not emit messageGap after stopping reading events

* Refactoring

* Fix tests
2022-11-10 17:50:19 +05:30
Clint Wylie 44f29030dd
fix flaky RemoteTaskRunnerTest.testRunPendingTaskFailToAssignTask with ugly Thread.sleep (#13344) 2022-11-10 14:28:53 +05:30
AmatyaAvadhanula 0512ae4922
Optimize metadata calls in SeekableStreamSupervisor (#13328)
* Optimize metadata calls

* Modify isTaskCurrent

* Fix tests

* Refactoring
2022-11-10 07:22:51 +05:30
AmatyaAvadhanula a2013e6566
Enhance streaming ingestion metrics (#13331)
Changes:
- Add a metric for partition-wise kafka/kinesis lag for streaming ingestion.
- Emit lag metrics for streaming ingestion when supervisor is not suspended and state is in {RUNNING, IDLE, UNHEALTHY_TASKS, UNHEALTHY_SUPERVISOR}
- Document metrics
2022-11-09 23:44:15 +05:30
Tejaswini Bandlamudi 594545da55
Adds cluster level idleConfig setting for supervisor (#13311)
* adds cluster level idleConfig

* updates docs

* refactoring

* spelling nit

* nit

* nit

* refactoring
2022-11-08 14:54:14 +05:30
AmatyaAvadhanula a738ac9ad7
Improve task pause logging and metrics for streaming ingestion (#13313)
* Improve task pause logging and metrics for streaming ingestion

* Add metrics doc

* Fix spelling
2022-11-07 21:33:54 +05:30
AmatyaAvadhanula 650840ddaf
Add segment handoff time metric (#13238)
* Add segment handoff time metric

* Remove monitors on scheduler stop

* Add warning log for slow handoff

* Remove monitor when scheduler stops
2022-11-07 17:49:10 +05:30
Gian Merlino 227b57dd8e
Compaction: Fetch segments one at a time on main task; skip when possible. (#13280)
* Compaction: Fetch segments one at a time on main task; skip when possible.

Compact tasks include the ability to fetch existing segments and determine
reasonable defaults for granularitySpec, dimensionsSpec, and metricsSpec.
This is a useful feature that makes compact tasks work well even when the
user running the compaction does not have a clear idea of what they want
the compacted segments to be like.

However, this comes at a cost: it takes time, and disk space, to do all
of these fetches. This patch improves the situation in two ways:

1) When segments do need to be fetched, download them one at a time and
   delete them when we're done. This still takes time, but minimizes the
   required disk space.

2) Don't fetch segments on the main compact task when they aren't needed.
   If the user provides a full granularitySpec, dimensionsSpec, and
   metricsSpec, we can skip it.

* Adjustments.

* Changes from code review.

* Fix logic for determining rollup.
2022-11-07 14:50:14 +05:30
Jonathan Wei 2fdaa2fcab
Make RecordSupplierInputSource respect sampler timeout when stream is empty (#13296)
* Make RecordSupplierInputSource respect sampler timeout when stream is empty

* Rename timeout param, make it nullable, add timeout test
2022-11-03 17:45:35 -05:00
Dr. Sizzles e5ad24ff9f
Support for middle manager less druid, tasks launch as k8s jobs (#13156)
* Support for middle manager less druid, tasks launch as k8s jobs

* Fixing forking task runner test

* Test cleanup, dependency cleanup, intellij inspections cleanup

* Changes per PR review

Add configuration option to disable http/https proxy for the k8s client
Update the docs to provide more detail about sidecar support

* Removing un-needed log lines

* Small changes per PR review

* Upon task completion we callback to the overlord to update the status / locaiton, for slower k8s clusters, this reduces locking time significantly

* Merge conflict fix

* Fixing tests and docs

* update tiny-cluster.yaml 

changed `enableTaskLevelLogPush` to `encapsulatedTask`

* Apply suggestions from code review

Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>

* Minor changes per PR request

* Cleanup, adding test to AbstractTask

* Add comment in peon.sh

* Bumping code coverage

* More tests to make code coverage happy

* Doh a duplicate dependnecy

* Integration test setup is weird for k8s, will do this in a different PR

* Reverting back all integration test changes, will do in anotbher PR

* use StringUtils.base64 instead of Base64

* Jdk is nasty, if i compress in jdk 11 in jdk 17 the decompressed result is different

Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
2022-11-02 19:44:47 -07:00
Kashif Faraz fd7864ae33
Improve run time of coordinator duty MarkAsUnusedOvershadowedSegments (#13287)
In clusters with a large number of segments, the duty `MarkAsUnusedOvershadowedSegments`
can take a long very long time to finish. This is because of the costly invocation of 
`timeline.isOvershadowed` which is done for every used segment in every coordinator run.

Changes
- Use `DataSourceSnapshot.getOvershadowedSegments` to get all overshadowed segments
- Iterate over this set instead of all used segments to identify segments that can be marked as unused
- Mark segments as unused in the DB in batches rather than one at a time
- Refactor: Add class `SegmentTimeline` for ease of use and readability while using a
`VersionedIntervalTimeline` of segments.
2022-11-01 20:19:52 +05:30
AmatyaAvadhanula e1ff3ca289
Resume streaming tasks on Overlord switch (#13223)
* Resume streaming tasks on Overlord switch

* Refactoring and better messages

* Better docs

* Add unit test

* Fix tests' setup

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

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

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

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

* Better logs

* Fix test again

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2022-10-29 09:38:49 +05:30
Gian Merlino 5429b9d764
RTR: Dedupe items in getKnownTasks. (#13273)
Fixes a problem where the tasks API in OverlordResource would complain
about duplicate keys in the map it's building.
2022-10-28 08:31:26 -07:00
AmatyaAvadhanula 9cbda66d96
Remove skip ignorable shards (#13221)
* Revert "Improve kinesis task assignment after resharding (#12235)"

This reverts commit 1ec57cb935.
2022-10-28 16:19:01 +05:30
Gian Merlino d98c808d3f
Remove basePersistDirectory from tuning configs. (#13040)
* Remove basePersistDirectory from tuning configs.

Since the removal of CliRealtime, it serves no purpose, since it is
always overridden in production using withBasePersistDirectory given
some subdirectory of the task work directory.

Removing this from the tuning config has a benefit beyond removing
no-longer-needed logic: it also avoids the side effect of empty
"druid-realtime-persist" directories getting created in the systemwide
temp directory.

* Test adjustments to appropriately set basePersistDirectory.

* Remove unused import.

* Fix RATC constructor.
2022-10-21 17:25:36 -07:00
AmatyaAvadhanula b88e1c21ea
Fix Overlord leader election when task lock re-acquisition fails (#13172)
Overlord leader election can sometimes fail due to task lock re-acquisition issues.
This commit solves the issue by failing such tasks and clearing all their locks.
2022-10-17 15:23:16 +05:30
Tejaswini Bandlamudi 3e13584e0e
Adds Idle feature to `SeekableStreamSupervisor` for inactive stream (#13144)
* Idle Seekable stream supervisor changes.

* nit

* nit

* nit

* Adds unit tests

* Supervisor decides it's idle state instead of AutoScaler

* docs update

* nit

* nit

* docs update

* Adds Kafka unit test

* Adds Kafka Integration test.

* Updates travis config.

* Updates kafka-indexing-service dependencies.

* updates previous offsets snapshot & doc

* Doesn't act if supervisor is suspended.

* Fixes highest current offsets fetch bug, adds new Kafka UT tests, doc changes.

* Reverts Kinesis Supervisor idle behaviour changes.

* nit

* nit

* Corrects SeekableStreamSupervisorSpec check on idle behaviour config, adds tests.

* Fixes getHighestCurrentOffsets to fetch offsets of publishing tasks too

* Adds Kafka Supervisor UT

* Improves test coverage in druid-server

* Corrects IT override config

* Doc updates and Syntactic changes

* nit

* supervisorSpec.ioConfig.idleConfig changes
2022-10-12 18:31:08 +05:30
Kashif Faraz b07f01d645
Set useMaxMemoryEstimates=false by default (#13178)
A value of `false` denotes that the new flow with improved estimates will be used.
2022-10-04 15:04:23 +05:30
Kashif Faraz ce5f55e5ce
Fix over-replication caused by balancing when inventory is not updated yet (#13114)
* Add coordinator test framework

* Remove outdated changes

* Add more tests

* Add option to auto-sync inventory

* Minor cleanup

* Fix inspections

* Add README for simulations, add SegmentLoadingNegativeTest

* Fix over-replication from balancing

* Fix README

* Cleanup unnecessary fields from DruidCoordinator

* Add a test

* Fix DruidCoordinatorTest

* Remove unused import

* Fix CuratorDruidCoordinatorTest

* Remove test log4j2.xml
2022-09-29 12:06:23 +05:30
Jonathan Wei 1f1fced6d4
Add JsonInputFormat option to assume newline delimited JSON, improve parse exception handling for multiline JSON (#13089)
* Add JsonInputFormat option to assume newline delimited JSON, improve handling for non-NDJSON

* Fix serde and docs

* Add PR comment check
2022-09-26 19:51:04 -05:00
Laksh Singla 0bfa81b7df
Fix the Injector creation in HadoopTask (#13138)
* Injector fix in HadoopTask

* Log the ExtensionsConfig while instantiating the HadoopTask

* Log the config in the run() method instead of the ctor
2022-09-24 10:38:25 +05:30
Jonathan Wei 331e6d707b
Add KafkaConfigOverrides extension point (#13122)
* Add KafkaConfigOverrides extension point

* X
2022-09-21 11:47:19 +05:30
Gian Merlino 2e729170cc
Kill task: Don't include markAsUnused unless set. (#13104)
Cleans up the serialized JSON.
2022-09-17 14:03:34 -07:00
AmatyaAvadhanula 1311e85f65
Faster fix for dangling tasks upon supervisor termination (#13072)
This commit fixes issues with delayed supervisor termination during certain transient states.
Tasks can be created during supervisor termination and left behind since the cleanup may
not consider these newly added tasks.

#12178 added a lock for the entire process of task creation to prevent such dangling tasks.
But it also introduced a deadlock scenario as follows:
- An invocation of `runInternal` is in progress.
- A `stop` request comes, acquires `stateChangeLock` and submit a `ShutdownNotice`
- `runInternal` keeps waiting to acquire the `stateChangeLock`
- `ShutdownNotice` remains stuck in the notice queue because `runInternal` is still running
- After some timeout, the supervisor goes through a forced termination

Fix:
 * `SeekableStreamSupervisor.runInternal` - do not try to acquire lock if supervisor is already stopping
 * `SupervisorStateManager.maybeSetState` - do not allow transitions from STOPPING state
2022-09-15 15:31:14 +05:30
Abhishek Agarwal 618757352b
Bump up the version to 25.0.0 (#12975)
* Bump up the version to 25.0.0

* Fix the version in console
2022-08-29 11:27:38 +05:30
Karan Kumar 275f834b2a
Race in Task report/log streamer (#12931)
* Fixing RACE in HTTP remote task Runner

* Changes in the interface

* Updating documentation

* Adding test cases to SwitchingTaskLogStreamer

* Adding more tests
2022-08-25 17:56:01 -07:00
Clint Wylie 8ee8786d3c
add maxBytesInMemory and maxClientResponseBytes to SamplerConfig (#12947)
* add maxBytesInMemory and maxClientResponseBytes to SamplerConfig
2022-08-25 00:50:41 -07:00
Gian Merlino 35aaaa9573
Fix serialization in TaskReportFileWriters. (#12938)
* Fix serialization in TaskReportFileWriters.

For some reason, serializing a Map<String, TaskReport> would omit the
"type" field. Explicitly sending each value through the ObjectMapper
fixes this, because the type information does not get lost.

* Fixes for static analysis.
2022-08-24 08:11:01 -07:00
Adarsh Sanjeev 3b58a01c7c
Correct spelling in messages and variable names. (#12932) 2022-08-24 11:06:31 +05:30
Gian Merlino d7d15ba51f
Add druid-multi-stage-query extension. (#12918)
* Add druid-multi-stage-query extension.

* Adjustments from CI.

* Task ID validation.

* Various changes from code review.

* Remove unnecessary code.

* LGTM-related.
2022-08-23 18:44:01 -07:00
Karan Kumar a3a9c5f409
Fixing overlord issued too many redirects (#12908)
* Fixing race in overlord redirects where the node was redirecting to itself

* Fixing test cases
2022-08-17 18:27:39 +05:30