* discover nested columns when using nested column indexer for schemaless
* move useNestedColumnIndexerForSchemaDiscovery from AppendableIndexSpec to DimensionsSpec
* 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.
* 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
* 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
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`
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.
* 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.)
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`
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
* fixes BlockLayoutColumnarLongs close method to nullify internal buffer.
* fixes other BlockLayoutColumnar supplier close methods to nullify internal buffers.
* fix spotbugs
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]".
* 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>
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
* 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.
* 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>
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.
* 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.
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.
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
* Fixing RACE in HTTP remote task Runner
* Changes in the interface
* Updating documentation
* Adding test cases to SwitchingTaskLogStreamer
* Adding more tests
* 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.
* Fix race in TaskQueue.notifyStatus.
It was possible for manageInternal to relaunch a task while it was
being cleaned up, due to a race that happens when notifyStatus is
called to clean up a successful task:
1) In a critical section, notifyStatus removes the task from "tasks".
2) Outside a critical section, notifyStatus calls taskRunner.shutdown
to let the task runner know it can clear out its data structures.
3) In a critical section, syncFromStorage adds the task back to "tasks",
because it is still present in metadata storage.
4) In a critical section, manageInternalCritical notices that the task
is in "tasks" and is not running in the taskRunner, so it launches
it again.
5) In a (different) critical section, notifyStatus updates the metadata
store to set the task status to SUCCESS.
6) The task continues running even though it should not be.
The possibility for this race was introduced in #12099, which shrunk
the critical section in notifyStatus. Prior to that patch, a single
critical section encompassed (1), (2), and (5), so the ordering above
was not possible.
This patch does the following:
1) Fixes the race by adding a recentlyCompletedTasks set that prevents
the main management loop from doing anything with tasks that are
currently being cleaned up.
2) Switches the order of the critical sections in notifyStatus, so
metadata store updates happen first. This is useful in case of
server failures: it ensures that if the Overlord fails in the midst
of notifyStatus, then completed-task statuses are still available in
ZK or on MMs for the next Overlord. (Those are cleaned up by
taskRunner.shutdown, which formerly ran first.) This isn't related
to the race described above, but is fixed opportunistically as part
of the same patch.
3) Changes the "tasks" list to a map. Many operations require retrieval
or removal of individual tasks; those are now O(1) instead of O(N)
in the number of running tasks.
4) Changes various log messages to use task ID instead of full task
payload, to make the logs more readable.
* Fix format string.
* Update comment.
Kinesis ingestion requires all shards to have at least 1 record at the required position in druid.
Even if this is satisified initially, resharding the stream can lead to empty intermediate shards. A significant delay in writing to newly created shards was also problematic.
Kinesis shard sequence numbers are big integers. Introduce two more custom sequence tokens UNREAD_TRIM_HORIZON and UNREAD_LATEST to indicate that a shard has not been read from and that it needs to be read from the start or the end respectively.
These values can be used to avoid the need to read at least one record to obtain a sequence number for ingesting a newly discovered shard.
If a record cannot be obtained immediately, use a marker to obtain the relevant shardIterator and use this shardIterator to obtain a valid sequence number. As long as a valid sequence number is not obtained, continue storing the token as the offset.
These tokens (UNREAD_TRIM_HORIZON and UNREAD_LATEST) are logically ordered to be earlier than any valid sequence number.
However, the ordering requires a few subtle changes to the existing mechanism for record sequence validation:
The sequence availability check ensures that the current offset is before the earliest available sequence in the shard. However, current token being an UNREAD token indicates that any sequence number in the shard is valid (despite the ordering)
Kinesis sequence numbers are inclusive i.e if current sequence == end sequence, there are more records left to read.
However, the equality check is exclusive when dealing with UNREAD tokens.
* Refactor Guice initialization
Builders for various module collections
Revise the extensions loader
Injector builders for server startup
Move Hadoop init to indexer
Clean up server node role filtering
Calcite test injector builder
* Revisions from review comments
* Build fixes
* Revisions from review comments
Few indexing tasks register RealtimeMetricsMonitor or TaskRealtimeMetricsMonitor with the process’s MonitorScheduler when they start. These monitors never unregister themselves (they always return true, they'd need to return false to unregister). Each of these monitors emits a set of metrics once every druid.monitoring.emissionPeriod.
As a result, after executing several tasks for a while, Indexer emits metrics of these tasks even after they're long gone.
Proposed Solution
Since one should be able to obtain the last round of ingestion metrics after the task unregisters the monitor, introducing lastRoundMetricsToBePushed variable to keep track of the same and overriding the AbstractMonitor.monitor method in RealtimeMetricsMonitor, TaskRealtimeMetricsMonitor to implement the new logic.
* Add EIGHT_HOUR into possible list of Granularities.
* Add the missing definition.
* fix test.
* Fix another test.
* Stylecheck finally passed.
Co-authored-by: Didip Kerabat <didip@apple.com>
* Mid-level service client and updated high-level clients.
Our servers talk to each other over HTTP. We have a low-level HTTP
client (HttpClient) that is super-asynchronous and super-customizable
through its handlers. It's also proven to be quite robust: we use it
for Broker -> Historical communication over the wide variety of query
types and workloads we support.
But the low-level client has no facilities for service location or
retries, which means we have a variety of high-level clients that
implement these in their own ways. Some high-level clients do a better
job than others. This patch adds a mid-level ServiceClient that makes
it easier for high-level clients to be built correctly and harmoniously,
and migrates some of the high-level logic to use ServiceClients.
Main changes:
1) Add ServiceClient org.apache.druid.rpc package. That package also
contains supporting stuff like ServiceLocator and RetryPolicy
interfaces, and a DiscoveryServiceLocator based on
DruidNodeDiscoveryProvider.
2) Add high-level OverlordClient in org.apache.druid.rpc.indexing.
3) Indexing task client creator in TaskServiceClients. It uses
SpecificTaskServiceLocator to find the tasks. This improves on
ClientInfoTaskProvider by caching task locations for up to 30 seconds
across calls, reducing load on the Overlord.
4) Rework ParallelIndexSupervisorTaskClient to use a ServiceClient
instead of extending IndexTaskClient.
5) Rework RemoteTaskActionClient to use a ServiceClient instead of
DruidLeaderClient.
6) Rework LocalIntermediaryDataManager, TaskMonitor, and
ParallelIndexSupervisorTask. As a result, MiddleManager, Peon, and
Overlord no longer need IndexingServiceClient (which internally used
DruidLeaderClient).
There are some concrete benefits over the prior logic, namely:
- DruidLeaderClient does retries in its "go" method, but only retries
exactly 5 times, does not sleep between retries, and does not retry
retryable HTTP codes like 502, 503, 504. (It only retries IOExceptions.)
ServiceClient handles retries in a more reasonable way.
- DruidLeaderClient's methods are all synchronous, whereas ServiceClient
methods are asynchronous. This is used in one place so far: the
SpecificTaskServiceLocator, so we don't need to block a thread trying
to locate a task. It can be used in other places in the future.
- HttpIndexingServiceClient does not properly handle all server errors.
In some cases, it tries to parse a server error as a successful
response (for example: in getTaskStatus).
- IndexTaskClient currently makes an Overlord call on every task-to-task
HTTP request, as a way to find where the target task is. ServiceClient,
through SpecificTaskServiceLocator, caches these target locations
for a period of time.
* Style adjustments.
* For the coverage.
* Adjustments.
* Better behaviors.
* Fixes.
* Poison StupidPool and fix resource leaks
There are various resource leaks from test setup as well as some
corners in query processing. We poison the StupidPool to start failing
tests when the leaks come and fix any issues uncovered from that so
that we can start from a clean baseline.
Unfortunately, because of how poisoning works,
we can only fail future checkouts from the same pool,
which means that there is a natural race between a
leak happening -> GC occurs -> leak detected -> pool poisoned.
This race means that, depending on interleaving of tests,
if the very last time that an object is checked out
from the pool leaks, then it won't get caught.
At some point in the future, something will catch it,
however and from that point on it will be deterministic.
* Remove various things left over from iterations
* Clean up FilterAnalysis and add javadoc on StupidPool
* Revert changes to .idea/misc.xml that accidentally got pushed
* Style and test branches
* Stylistic woes
The expiry timeout is compared against the current time but the condition is reversed.
This means that as soon as a supervisor task finishes, its partitions are cleaned up,
irrespective of the specified `intermediaryPartitionTimeout` period.
After these changes, the `intermediaryPartitionTimeout` will start getting honored.
Changes
* Fix the condition
* Add tests to verify the new correct behaviour
* Reduce the default expiry timeout from P1D to PT5M
to retain current behaviour in case of default configs.
* Fix flaky KafkaIndexTaskTest.
The testRunTransactionModeRollback case had many race conditions. Most notably,
it would commit a transaction and then immediately check to see that the results
were *not* indexed. This is racey because it relied on the indexing thread being
slower than the test thread.
Now, the case waits for the transaction to be processed by the indexing thread
before checking the results.
* Changes from review.
* Remove null and empty fields from native queries
* Test fixes
* Attempted IT fix.
* Revisions from review comments
* Build fixes resulting from changes suggested by reviews
* IT fix for changed segment size
The web-console (indirectly) calls the Overlord’s GET tasks API to fetch the tasks' summary which in turn queries the metadata tasks table. This query tries to fetch several columns, including payload, of all the rows at once. This introduces a significant memory overhead and can cause unresponsiveness or overlord failure when the ingestion tab is opened multiple times (due to several parallel calls to this API)
Another thing to note is that the task table (the payload column in particular) can be very large. Extracting large payloads from such tables can be very slow, leading to slow UI. While we are fixing the memory pressure in the overlord, we can also fix the slowness in UI caused by fetching large payloads from the table. Fetching large payloads also puts pressure on the metadata store as reported in the community (Metadata store query performance degrades as the tasks in druid_tasks table grows · Issue #12318 · apache/druid )
The task summaries returned as a response for the API are several times smaller and can fit comfortably in memory. So, there is an opportunity here to fix the memory usage, slow ingestion, and under-pressure metadata store by removing the need to handle large payloads in every layer we can. Of course, the solution becomes complex as we try to fix more layers. With that in mind, this page captures two approaches. They vary in complexity and also in the degree to which they fix the aforementioned problems.
* ForkingTaskRunner: Set ActiveProcessorCount for tasks.
This prevents various automatically-sized thread pools from being unreasonably
large (we don't want each task to size its pools as if it is the only thing on
the entire machine).
* Fix tests.
* Add missing LifecycleStart annotation.
* ForkingTaskRunner needs ManageLifecycle.
Often users are submitting queries, and ingestion specs that work only if the relevant extension is not loaded. However, the error is too technical for the users and doesn't suggest them to check for missing extensions. This PR modifies the error message so users can at least check their settings before assuming that the error is because of a bug.
* Emit state of replace and append for native batch tasks
* Emit count of one depending on batch ingestion mode (APPEND, OVERWRITE, REPLACE)
* Add metric to compaction job
* Avoid null ptr exc when null emitter
* Coverage
* Emit tombstone & segment counts
* Tasks need a type
* Spelling
* Integrate BatchIngestionMode in batch ingestion tasks functionality
* Typos
* Remove batch ingestion type from metric since it is already in a dimension. Move IngestionMode to AbstractTask to facilitate having mode as a dimension. Add metrics to streaming. Add missing coverage.
* Avoid inner class referenced by sub-class inspection. Refactor computation of IngestionMode to make it more robust to null IOConfig and fix test.
* Spelling
* Avoid polluting the Task interface
* Rename computeCompaction methods to avoid ambiguous java compiler error if they are passed null. Other minor cleanup.
* Deal with potential cardinality estimate being negative and add logging
* Fix typo in name
* Refine and minimize logging
* Make it info based on code review
* Create a named constant for the magic number
Issue:
Even though `CompactionTuningConfig` allows a `maxColumnsToMerge` config
(to optimize memory usage, particulary for datasources with many dimensions),
the corresponding client object `ClientCompactionTaskQueryTuningConfig`
(used by the coordinator duty `CompactSegments` to trigger auto-compaction)
does not contain this field. Thus, the value of `maxColumnsToMerge` specified
in any datasource compaction config is ignored.
Changes:
- Add field `maxColumnsToMerge` in `ClientCompactionTaskQueryTuningConfig`
and `UserCompactionTaskQueryTuningConfig`
- Fix tests
* RemoteTaskRunner: Fix NPE in streamTaskReports.
It is possible for a work item to drop out of runningTasks after the
ZkWorker is retrieved. In this case, the current code would throw
an NPE.
* Additional tests and additional fixes.
* Fix import.
1) Align "Assigning task" log messages between RTR and HRTR.
2) Remove confusing reference to "Coordinator".
3) Move "Not assigning task" message from INFO to DEBUG. It's not super
important to see this message: we mainly want to see what _does_ get
assigned.
4) Reword "Task switched from pending to running" message to better
match the structure of the "Assigning task" message from the same
method.
* Add builder for TaskToolbox.
The main purpose of this change is to make it easier to create
TaskToolboxes in tests. However, the builder is used in production
too, by TaskToolboxFactory.
* Fix imports, adjust formatting.
* Fix import.
Currently all Druid processes share the same log4j2 configuration file located in _common directory. Since peon processes are spawned by middle manager process, they derivate the environment variables from the middle manager. These variables include those in the log4j2.xml controlling to which file the logger writes the log.
But current task logging mechanism requires the peon processes to output the log to console so that the middle manager can redirect the console output to a file and upload this file to task log storage.
So, this PR imposes this requirement to peon processes, whatever the configuration is in the shared log4j2.xml, peon processes always write the log to console.
* concurrency: introduce GuardedBy to TaskQueue
* perf: Introduce TaskQueueScaleTest to test performance of TaskQueue with large task counts
This introduces a test case to confirm how long it will take to launch and manage (aka shutdown)
a large number of threads in the TaskQueue.
h/t to @gianm for main implementation.
* perf: improve scalability of TaskQueue with large task counts
* linter fixes, expand test coverage
* pr feedback suggestion; swap to different linter
* swap to use SuppressWarnings
* Fix TaskQueueScaleTest.
Co-authored-by: Gian Merlino <gian@imply.io>
Following up on #12315, which pushed most of the logic of building ImmutableBitmap into BitmapIndex in order to hide the details of how column indexes are implemented from the Filter implementations, this PR totally refashions how Filter consume indexes. The end result, while a rather dramatic reshuffling of the existing code, should be extraordinarily flexible, eventually allowing us to model any type of index we can imagine, and providing the machinery to build the filters that use them, while also allowing for other column implementations to implement the built-in index types to provide adapters to make use indexing in the current set filters that Druid provides.
This PR is to measure how long a task stays in the pending queue and emits the value with the metric task/pending/time. The metric is measured in RemoteTaskRunner and HttpRemoteTaskRunner.
An example of the metric:
```
2022-04-26T21:59:09,488 INFO [rtr-pending-tasks-runner-0] org.apache.druid.java.util.emitter.core.LoggingEmitter - {"feed":"metrics","timestamp":"2022-04-26T21:59:09.487Z","service":"druid/coordinator","host":"localhost:8081","version":"2022.02.0-iap-SNAPSHOT","metric":"task/pending/time","value":8,"dataSource":"wikipedia","taskId":"index_parallel_wikipedia_gecpcglg_2022-04-26T21:59:09.432Z","taskType":"index_parallel"}
```
------------------------------------------
Key changed/added classes in this PR
Emit metric task/pending/time in classes RemoteTaskRunner and HttpRemoteTaskRunner.
Update related factory classes and tests.
* Make tombstones ingestible by having them return an empty result set.
* Spotbug
* Coverage
* Coverage
* Remove unnecessary exception (checkstyle)
* Fix integration test and add one more to test dropExisting set to false over tombstones
* Force dropExisting to true in auto-compaction when the interval contains only tombstones
* Checkstyle, fix unit test
* Changed flag by mistake, fixing it
* Remove method from interface since this method is specific to only DruidSegmentInputentity
* Fix typo
* Adapt to latest code
* Update comments when only tombstones to compact
* Move empty iterator to a new DruidTombstoneSegmentReader
* Code review feedback
* Checkstyle
* Review feedback
* Coverage
* Optionally load segment index files into page cache on bootstrap and new segment download
* Fix unit test failure
* Fix test case
* fix spelling
* fix spelling
* fix test and test coverage issues
Co-authored-by: Jian Wang <wjhypo@gmail.com>
* add impl
* add impl
* fix checkstyle
* add impl
* add unit test
* fix stuff
* fix stuff
* fix stuff
* add unit test
* add more unit tests
* add more unit tests
* add IT
* add IT
* add IT
* add IT
* add ITs
* address comments
* fix test
* fix test
* fix test
* address comments
* address comments
* address comments
* fix conflict
* fix checkstyle
* address comments
* fix test
* fix checkstyle
* fix test
* fix test
* fix IT
The `javaOpts` property is being read from task context but not `javaOptsArray`.
Changes:
- Read `javaOptsArray` from task context in `ForkingTaskRunner`.
- Add test to verify that `javaOptsArray` in task context takes precedence over `javaOpts`
* Store null columns in the segments
* fix test
* remove NullNumericColumn and unused dependency
* fix compile failure
* use guava instead of apache commons
* split new tests
* unused imports
* address comments
Parallel indexing with range partitioning can often cause OOM in the
`ParallelIndexSupervisorTask` during the dimension distribution phase.
This typically happens because of too many `StringSketch` objects
obtained from the different `partial_dimension_distribution` sub-tasks.
We need not keep any of the sketches in memory until we need to compute
the PartitionBoundaries for the respective interval.
Changes
- Extract `StringDistribution` from `DimensionDistributionReport`s when they are received
and write to disk inside the task/temp/distributions
- After all the subtasks have finished, iterate over all the intervals one by one
- For each interval, read the distributions from disk, merge them and create `PartitionBoundaries`.
- Cleanup task/temp/distributions directory when all `PartitionBoundaries` have been determined