Currently, Druid is using Guava 16.0.1 version. This upgrade to 31.1-jre fixes the following issues.
CVE-2018-10237 (Unbounded memory allocation in Google Guava 11.0 through 24.x before 24.1.1 allows remote attackers to conduct denial of service attacks against servers that depend on this library and deserialize attacker-provided data because the AtomicDoubleArray class (when serialized with Java serialization) and the CompoundOrdering class (when serialized with GWT serialization) perform eager allocation without appropriate checks on what a client has sent and whether the data size is reasonable). We don't use Java or GWT serializations. Despite being false positive they're causing red security scans on Druid distribution.
Latest version of google-client-api is incompatible with the existing Guava version. This PR unblocks Update google client apis to latest version #14414
Motivation:
- There is no usage of the `SegmentTransactionInsertAction` which passes a
non-null non-empty value of `segmentsToBeDropped`.
- This is not really needed either as overshadowed segments are marked as unused
by the Coordinator and need not be done in the same transaction as committing segments.
- It will also help simplify the changes being made in #14407
Changes:
- Remove `segmentsToBeDropped` from the task action and all intermediate methods
- Remove related tests which are not needed anymore
Changes:
- Move logic of `NewestSegmentFirstIterator.needsCompaction` to `CompactionStatus`
to improve testability and readability
- Capture the list of checks performed to determine if compaction is needed in a readable
manner in `CompactionStatus.CHECKS`
- Make `CompactionSegmentIterator` iterate over instances of `SegmentsToCompact`
instead of `List<DataSegment>`. This allows use of the `umbrellaInterval` later.
- Replace usages of `QueueEntry` with `SegmentsToCompact`
- Move `SegmentsToCompact` out of `NewestSegmentFirstIterator`
- Simplify `CompactionStatistics`
- Reduce level of less important logs to debug
- No change made to tests to ensure correctness
* Add supervisor /resetOffsets API.
- Add a new endpoint /druid/indexer/v1/supervisor/<supervisorId>/resetOffsets
which accepts DataSourceMetadata as a body parameter.
- Update logs, unit tests and docs.
* Add a new interface method for backwards compatibility.
* Rename
* Adjust tests and javadocs.
* Use CoreInjectorBuilder instead of deprecated makeInjectorWithModules
* UT fix
* Doc updates.
* remove extraneous debugging logs.
* Remove the boolean setting; only ResetHandle() and resetInternal()
* Relax constraints and add a new ResetOffsetsNotice; cleanup old logic.
* A separate ResetOffsetsNotice and some cleanup.
* Minor cleanup
* Add a check & test to verify that sequence numbers are only of type SeekableStreamEndSequenceNumbers
* Add unit tests for the no op implementations for test coverage
* CodeQL fix
* checkstyle from merge conflict
* Doc changes
* DOCUSAURUS code tabs fix. Thanks, Brian!
* Rolling supervior task publishing
* add an option for number of task groups to roll over
* better
* remove docs
* oops
* checkstyle
* wip test
* undo partial test change
* remove incomplete test
### Description
Previously, the `maxSegments` configured for auto kill could be ignored if an interval of data for a given datasource had more than this number of unused segments, causing the kill task spawned with the task of deleting unused segments in that given interval of data to delete more than the `maxSegments` configured. Now each kill task spawned by the auto kill coordinator duty, will kill at most `limit` segments. This is done by adding a new config property to the `KillUnusedSegmentTask` which allows users to specify this limit.
* Remove chatAsync parameter, so chat is always async.
chatAsync has been made default in Druid 26. I have seen good
battle-testing of it in production, and am comfortable removing the
older sync client.
This was the last remaining usage of IndexTaskClient, so this patch
deletes all that stuff too.
* Remove unthrown exception.
* Remove unthrown exception.
* No more TimeoutException.
split KillUnusedSegmentsTask to smaller batches
Processing in smaller chunks allows the task execution to yield the TaskLockbox lock,
which allows the overlord to continue being responsive to other tasks and users while
this particular kill task is executing.
* introduce KillUnusedSegmentsTask batchSize parameter to control size of batching
* provide an explanation for kill task batchSize parameter
* add logging details for kill batch progress
* allow for batched delete of segments instead of deleting segment data one by one
create new batchdelete method in datasegment killer that has default functionality
of iterating through all segments and calling delete on them. This will enable
a slow rollout of other deepstorage implementations to move to a batched delete
on their own time
* cleanup batchdelete segments
* batch delete with the omni data deleter
cleaned up code
just need to add tests and docs for this functionality
* update java doc to explain how it will try to use batch if function is overwritten
* rename killBatch to kill
add unit tests
* add omniDataSegmentKillerTest for deleting multiple segments at a time. fix checkstyle
* explain test peculiarity better
* clean up batch kill in s3.
* remove unused return value. cleanup comments and fix checkstyle
* default to batch delete. more specific java docs. list segments that couldn't be deleted
if there was a client error or server error
* simplify error handling
* add tests where an exception is thrown when killing multiple s3 segments
* add test for failing to delete two calls with the s3 client
* fix javadoc for kill(List<DataSegment> segments) clean up tests remove feature flag
* fix typo in javadocs
* fix test failure
* fix checkstyle and improve tests
* fix intellij inspections issues
* address comments, make delete multiple segments not assume same bucket
* fix test errors
* better grammar and punctuation. fix test. and better logging for exception
* remove unused code
* avoid extra arraylist instantiation
* fix broken test
* fix broken test
* fix tests to use assert.throws
* Merge core CoordinatorClient with MSQ CoordinatorServiceClient.
Continuing the work from #12696, this patch merges the MSQ
CoordinatorServiceClient into the core CoordinatorClient, yielding a single
interface that serves both needs and is based on the ServiceClient RPC
system rather than DruidLeaderClient.
Also removes the backwards-compatibility code for the handoff API in
CoordinatorBasedSegmentHandoffNotifier, because the new API was added
in 0.14.0. That's long enough ago that we don't need backwards
compatibility for rolling updates.
* Fixups.
* Trigger GHA.
* Remove unnecessary retrying in DruidInputSource. Add "about an hour"
retry policy and h
* EasyMock
* Use OverlordClient for all Overlord RPCs.
Continuing the work from #12696, this patch removes HttpIndexingServiceClient
and the IndexingService flavor of DruidLeaderClient completely. All remaining
usages are migrated to OverlordClient.
Supporting changes include:
1) Add a variety of methods to OverlordClient.
2) Update MetadataTaskStorage to skip the complete-task lookup when
the caller requests zero completed tasks. This helps performance of
the "get active tasks" APIs, which don't want to see complete ones.
* Use less forbidden APIs.
* Fixes from CI.
* Add test coverage.
* Two more tests.
* Fix test.
* Updates from CR.
* Remove unthrown exceptions.
* Refactor to improve testability and test coverage.
* Add isNil tests.
* Remove unnecessary "deserialize" methods.
* Add ingest/input/bytes metric and Kafka consumer metrics.
New metrics:
1) ingest/input/bytes. Equivalent to processedBytes in the task reports.
2) kafka/consumer/bytesConsumed: Equivalent to the Kafka consumer
metric "bytes-consumed-total". Only emitted for Kafka tasks.
3) kafka/consumer/recordsConsumed: Equivalent to the Kafka consumer
metric "records-consumed-total". Only emitted for Kafka tasks.
* Fix anchor.
* Fix KafkaConsumerMonitor.
* Interface updates.
* Doc changes.
* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java
Co-authored-by: Benedict Jin <asdf2014@apache.org>
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
changes:
* new filters that preserve match value typing to better handle filtering different column types
* sql planner uses new filters by default in sql compatible null handling mode
* remove isFilterable from column capabilities
* proper handling of array filtering, add array processor to column processors
* javadoc for sql test filter functions
* range filter support for arrays, tons more tests, fixes
* add dimension selector tests for mixed type roots
* support json equality
* rename semantic index maker thingys to mostly have plural names since they typically make many indexes, e.g. StringValueSetIndex -> StringValueSetIndexes
* add cooler equality index maker, ValueIndexes
* fix missing string utf8 index supplier
* expression array comparator stuff
* More efficient generation of ImmutableWorkerHolder from WorkerHolder.
Taking the work done in #12096 a little further:
1) Applying a similar optimization to WorkerHolder (HttpRemoteTaskRunner).
The original patch only helped with the ZkWorker (RemoteTaskRunner).
2) Improve the ZkWorker version somewhat by avoiding multiple iterations
through the task announcements map.
* Pick better names and use better logic.
* Only runnable tasks.
* Fix test.
* Fix testBlacklistZKWorkers50Percent.
* 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.
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.
Description:
`TaskQueue.notifyStatus` is often a heavy call as it performs the following operations:
- Update task status in metadata DB
- Update task locks in metadata DB
- Request (synchronously) the task runner to shutdown the completed task
- Clean up in-memory data structures
This method can often be slow and can cause worker sync / task runners to slow down.
Main changes:
- Run task completion callbacks in a separate executor to handle task completion updates
- Add new config `druid.indexer.queue.taskCompleteHandlerNumThreads`
- Add metrics to monitor number of processed and queued items
- There are still other paths that can invoke `notifyStatus`, but those need not be moved to
the new executor as they are synchronous on purpose.
Other changes:
- Add new metrics `task/status/queue/count`, `task/status/handled/count`
- Add `TaskCountStatsProvider.getStats()` which deprecates the other `getXXXTaskCount` methods.
- Use `CoordinatorRunStats` to collect and report metrics. This class has been used as is
for now but will later be renamed and repurposed to use across all Druid services.
The wait doesn't seem to serve a purpose, other than causing delays
when checking isInitialized() for a large number of things that have
not yet been initialized.
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
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.
* 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
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.
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.
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.
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
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
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.
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.
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.