* 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.