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
* Moving all unnest cursor code atop refactored code for unnest
* Updating unnest cursor
* Removing dedup and fixing up some null checks
* AllowList changes
* Fixing some NPEs
* Using bitset for allowlist
* Updating the initialization only when cursor is in non-done state
* Updating code to skip rows not in allow list
* Adding a flag for cases when first element is not in allowed list
* Updating for a null in allowList
* Splitting unnest cursor into 2 subclasses
* Intercepting some apis with columnName for new unnested column
* Adding test cases and renaming some stuff
* checkstyle fixes
* Moving to an interface for Unnest
* handling null rows in a dimension
* Updating cursors after comments part-1
* Addressing comments and adding some more tests
* Reverting a change to ScanQueryRunner and improving a comment
* removing an unused function
* Updating cursors after comments part 2
* One last fix for review comments
* Making some functions private, deleting some comments, adding a test for unnest of unnest with allowList
* Adding an exception for a case
* Closure for unnest data source
* Adding some javadocs
* One minor change in makeDimSelector of columnarCursor
* Updating an error message
* Update processing/src/main/java/org/apache/druid/segment/DimensionUnnestCursor.java
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Unnesting on virtual columns was missing an object array, adding that to support virtual columns unnesting
* Updating exceptions to use UOE
* Renamed files, added column capability test on adapter, return statement and made unnest datasource not cacheable for the time being
* Handling for null values in dim selector
* Fixing a NPE for null row
* Updating capabilities
* Updating capabilities
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
SQL test framework extensions
* Capture planner artifacts: logical plan, etc.
* Planner test builder validates the logical plan
* Validation for the SQL resut schema (we already have
validation for the Druid row signature)
* Better Guice integration: properties, reuse Guice modules
* Avoid need for hand-coded expr, macro tables
* Retire some of the test-specific query component creation
* Fix query log hook race condition
* update static-checks GHA to run sequentially
remove static-checks from travis.yml
move docs, web-console, packaging checks from travis to GHA
* nit
* nit
* groups all checks, runs on 8, 11, 17 jdks
* nit
* adds license info
* update permissions on scripts folder
* nit
* nit
* fix packaging check
* changes naming, cleans repo before license checks
* simulate failure
* bump up license checks
* test license checks failure
* test license checks failure
* test license checks failure
* verify gha script run exit code
* fail fast in case of shell script
* verified fail fast in case of shell script
* initial commit for jdbc tutorial
(cherry picked from commit 04c4adad71e5436b76c3425fe369df03aaaf0acb)
* add commentary
* address comments from charles
* add query context to example
* fix typo
* add links
* Apply suggestions from code review
Co-authored-by: Frank Chen <frankchen@apache.org>
* fix datatype
* address feedback
* add parameterize to spelling file. the past tense version was already there
Co-authored-by: Frank Chen <frankchen@apache.org>
* add faults tests for the multi stage query
* add too many parttiions fault
* add toomanyinputfilesfault
* programmatically generate the file
* refactor
* Trigger Build
https://github.com/apache/druid/pull/13027 PR replaces `filter` parameter with
`objectGlob` in ingestion input source. However, this will cause existing ingestion
jobs to fail if they are using a filter already. This PR adds old filter functionality
alongside objectGlob to preserve backward compatibility.
Detects self-redirects, redirect loops, long redirect chains, and redirects to unknown servers.
Treat all of these cases as an unavailable service, retrying if the retry policy allows it.
Previously, some of these cases would lead to a prompt, unretryable error. This caused
clients contacting an Overlord during a leader change to fail with error messages like:
org.apache.druid.rpc.RpcException: Service [overlord] redirected too many times
Additionally, a slight refactor of callbacks in ServiceClientImpl improves readability of
the flow through onSuccess.
* fixes BlockLayoutColumnarLongs close method to nullify internal buffer.
* fixes other BlockLayoutColumnar supplier close methods to nullify internal buffers.
* fix spotbugs
* we can read where we want to
we can leave your bounds behind
'cause if the memory is not there
we really don't care
and we'll crash this process of mine
* Attach IO error to parse error when we can't contact Avro schema registry.
The change in #12080 lost the original exception context. This patch
adds it back.
* Add hamcrest-core.
* Fix format string.
The batch segment sampling performs significantly better than the older method
of sampling if there are a large number of used segments. It also avoids duplicates.
Changes:
- Make batch segment sampling the default
- Deprecate the property `useBatchedSegmentSampler`
- Remove unused coordinator config `druid.coordinator.loadqueuepeon.repeatDelay`
- Cleanup `KillUnusedSegments`
- Simplify `KillUnusedSegmentsTest`, add better tests, remove redundant tests
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.
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.
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]".