Skip to content

Cache segment metadata on the Overlord to speed up segment allocation and other task actions #17653

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 32 commits into from
Feb 11, 2025

Conversation

kfaraz
Copy link
Contributor

@kfaraz kfaraz commented Jan 22, 2025

NOTE: For anyone trying out this patch, please make sure to include #17772 as well.

Description

The Overlord performs several metadata operations on the tables druid_segments and druid_pendingSegments,
such as segment commit, allocation, upgrade and mark used / unused.

Segment allocation, in particular, involves several reads/writes to the metadata store and can often become a bottleneck,
causing ingestion to slow down. This effect is particularly pronounced when streaming ingestion is enabled for
multiple datasources or if there is a lot of late arriving data.

This patch adds an in-memory segment cache to the Overlord to speed up all segment metadata operations.

Assumptions

Design

Summary

  • Add Overlord runtime property druid.manager.segments.useCache to enable cache
  • Keep cache disabled by default
  • When cache is enabled, read metadata from cache and write metadata to both metadata store and cache
  • Poll metadata store periodically in case any metadata update did not make it to the cache (should not happen under stable operational conditions)
  • Non-leader Overlords also poll the metadata store and keep the cache ready in case of failover
  • Upon becoming leader, Overlord needs to finish one poll successfully before cache can be used in transactions

Segment metadata transaction with cache enabled

  • If cache is not enabled, fall back to old flow
  • If not leader, do not proceed
  • If cache is waiting for sync to finish after becoming leader, block until cache is ready
  • Acquire a lock on cache to ensure that another thread does not update it while we are reading from it.
  • Start transaction
  • Get leader term
  • Perform computations
  • For every read, just read from the cache
  • For every write
    • check if leadership has been lost or term has changed
    • this helps safeguard against cases where we lose leadership during the transaction
    • if yes, rollback transaction
    • if not, remember write action to commit to cache later
  • If transaction has succeeded, commit pending writes to cache
  • Close transaction
  • Release lock

Lifecycle of cache

  • Cache is enabled if druid.manager.segments.useCache=true on the Overlord
  • When Overlord starts, start() cache is called putting it in STANDBY mode.
  • In STANDBY mode, the cache polls the metadata store at a period of druid.manager.segments.pollDuration to do the following:
    • Retrieve all segment IDs and their last updated timestamps
    • If the cache has stale or no information for any unused segment, update it
    • If the cache has stale or no information for any used segment, fetch entire segment payload
    • Retrieve all pending segments and update cache if it has stale information
  • Cache cannot be used for transactions while it is in STANDBY mode.
  • When Overlord becomes leader, cache moves to SYNC_PENDING mode.
  • When the next poll starts, it moves the cache from SYNC_PENDING to SYNC_STARTED
  • When this poll completes, cache is marked as READY
  • In READY mode, cache can be used for read and write transactions
  • When Overlord loses leadership, cache is moved back to STANDBY mode
  • When Overlord stops, stop() cache is called stopping the poll

Contents of cache

The cache maintains the following fields for every datasource.

Field Needed In
Map<String, DataSegmentPlus> idToUsedSegment Reads/writes for used segments
Set<String> unusedSegmentIds Checking set of existing segment IDs to avoid duplicate insert
Map<Interval, Map<String, Integer>> intervalVersionToHighestUnusedPartitionNumber Segment allocation to avoid duplicate IDs
Map<Interval, Map<String, PendingSegmentRecord>> intervalToPendingSegments Segment allocation

Code changes

Class / Description
SegmentsMetadataManagerConfig.useCache
  • Enable/disable cache on the Overlord
DatasourceSegmentMetadataReader
  • Interface to perform segment metadata read operations
DatasourceSegmentMetadataWriter
  • Interface to perform segment metadata write operations
HeapMemorySegmentMetadataCache
  • Poll committed and pending segments from the metadata store
DatasourceSegmentCache
  • Cache committed and pending segments of a single datasource
SegmentMetadataTransaction
  • Encapsulate all read/write operations performed within a transaction.
  • This abstraction allows the code to redirect all read/write operations within a transaction to either the cache or to the metadata store itself
SqlSegmentMetadataTransaction
  • Perform read/writes directly on metadata store if cache is disabled or not ready
CachedSegmentMetadataTransaction
  • Perform read only from cache and writes to both metadata store and cache
SqlSegmentMetadataTransactionFactory
  • Create transaction based on state of cache
IndexerSQLMetadataStorageCoordinator
  • Perform all metadata transactions using transaction factory
  • Move metadata reads methods to SqlSegmentsMetadataQuery
  • Move metadata write methods to SqlSegmentMetadataTransaction

Testing

  • Run all ITs successfully with cache enabled in this commit
  • Update existing tests to run both with and without cache
    • IndexerSQLMetadataStorageCoordinatorTest
    • SegmentAllocateActionTest
    • SegmentAllocationQueueTest
  • Add DatasourceSegmentCacheTest

Local cluster

  • Ran a cluster with 600K segments
  • Full sync takes about 1 min
2025-02-10T08:36:43,664 INFO [SegmentMetadataCache-0]
  org.apache.druid.java.util.emitter.core.LoggingEmitter - [metrics]
  {"feed":"metrics",
  "metric":"segment/metadataCache/sync/time",
  "service":"druid/coordinator",
  "host":"localhost:8081",
  "version":"33.0.0-SNAPSHOT",
  "value":68942,
  "timestamp":"2025-02-10T08:36:43.664Z"
  }
  • Delta sync (retrieve all segment IDs but payloads of only updated segments) takes about 3 seconds
2025-02-10T08:36:46,521 INFO [SegmentMetadataCache-0]
  org.apache.druid.java.util.emitter.core.LoggingEmitter - [metrics]
  {"feed":"metrics",
  "metric":"segment/metadataCache/sync/time",
  "service":"druid/coordinator",
  "host":"localhost:8081",
  "version":"33.0.0-SNAPSHOT",
  "value":2855,
  "timestamp":"2025-02-10T08:36:46.521Z"
  }

Pending items

  • Cluster testing

Release note

Add Overlord runtime property druid.manager.segments.useCache (default value false).
Set this to true to turn on segment metadata caching on the Overlord. This allows segment metadata operations
such as reads and segment allocation to be sped up significantly.

The following metrics have been added:

  • segment/used/count
  • segment/unused/count
  • segment/pending/count
  • segment/metadataCache/sync/time
  • segment/metadataCache/deleted
  • segment/metadataCache/skipped
  • segment/metadataCache/used/stale
  • segment/metadataCache/used/updated
  • segment/metadataCache/unused/updated
  • segment/metadataCache/pending/deleted
  • segment/metadataCache/pending/updated
  • segment/metadataCache/pending/skipped

Upgrade notes

The flag druid.manager.segments.useCache to enable the segment cache should be turned on only when
Druid has been upgraded to a version containing both this patch #17653 and #17545 .
When Druid is being downgraded to an older, the feature flag must first be turned off.


This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • a release note entry in the PR description.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

@kfaraz kfaraz marked this pull request as ready for review January 28, 2025 12:34
// Assume that the metadata write operation succeeded
// Do not update the cache just yet, add to the list of pending writes
pendingCacheWrites.add(writer -> {
T ignored = action.apply(writer);

Check notice

Code scanning / CodeQL

Unread local variable Note

Variable 'T ignored' is never read.
@AmatyaAvadhanula
Copy link
Contributor

@kfaraz thank you for the changes.

Could you please call out the dependency on #17545 in the description?
I also think that it is important to turn off this feature flag prior to downgrades. (because it is potentially dangerous when the Coordinator is downgraded to Druid version <= 31.0.1 and the Overlord is still on version >= 33.x.y with this config turned on for a brief period.)

@kfaraz
Copy link
Contributor Author

kfaraz commented Feb 5, 2025

Thanks for the suggestion, @AmatyaAvadhanula . I have added an Upgrade Notes section.

Copy link
Contributor

@AmatyaAvadhanula AmatyaAvadhanula left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I had a couple of questions about certain choices, which I have left as comments.
I was also hoping to understand the pros and cons of having pending segments and segments being accessed by the same ReadWriteLock in the cache.

Otherwise, the segment allocation changes look good to me.

Will try to wrap up the review of the caching mechanism soon.


HeapMemoryDatasourceSegmentCache(String dataSource)
{
super(true);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please add more details explaining the usage of a fair lock?
I recall a discussion where we wanted to change the lock in VersionedIntervalTimeline to false as well.

Copy link
Contributor Author

@kfaraz kfaraz Feb 6, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, will add that in the javadoc.

Edit: I haven't really given much thought to whether we should go with fair or not for the cache. Reading through the javadoc of ReentrantReadWriteLock, fair had seemed an appropriate choice. But I will take another look and document whatever we decide to do.

/**
* Not being used right now. Could allow lookup of visible segments for a given interval.
*/
private final SegmentTimeline usedSegmentTimeline = SegmentTimeline.forSegments(Set.of());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Adding a timeline with potentially expensive operations because of its own lock seems risky and wasteful given that we are not using it.

Is there a reason we aren't using a TreeMap: Interval -> (DataSegment / SegmentId) and use it instead?
I believe the perf impact would be significant when there are several intervals and segments.
We are creating a Timeline after fetching segments anyway.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, the timeline is not really being used in the code right now. I will just get rid of it for the time being.

Copy link
Contributor Author

@kfaraz kfaraz Feb 6, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a reason we aren't using a TreeMap: Interval -> (DataSegment / SegmentId) and use it instead?
I believe the perf impact would be significant when there are several intervals and segments.

Hmm, let me evaluate this once. I had decided against it originally since most segment searches are for overlapping intervals rather than exact matches. But I guess we can have some additional logic to prune out intervals which are disjoint, thus benefiting perf as you point out.

Other searches are by segment ID, so keeping an id -> segment map helped.

Eventually, we would most likely end up keeping a timeline itself.
That timeline could also be used to replace the timeline maintained in SqlSegmentsMetadataManager used by CompactionScheduler (and also the coordinator).

segmentAllocationQueue.becomeLeader();
taskMaster.becomeHalfLeader(taskRunner, taskQueue);
}

@Override
public void stop()
{
segmentMetadataCache.stopBeingLeader();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Generally, the order of items in stop() should be the reverse of the order in start(), in case there are dependencies.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed.

@@ -228,6 +230,10 @@ public void configure(Binder binder)
JsonConfigProvider.bind(binder, "druid.indexer.task.default", DefaultTaskConfig.class);
binder.bind(RetryPolicyFactory.class).in(LazySingleton.class);

binder.bind(SegmentMetadataCache.class)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this meant to override the binding to NoopSegmentMetadataCache in SQLMetadataStorageDruidModule? I thought multiple bindings typically weren't allowed.

I wonder why we need the binding outside of CliOverlord at all- why do other server types need to create one?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ideally, the other server types shouldn't need the binding at all.
But we have this dep graph:
CoreInjectorBuilder -> DerbyMetadataStorageModule -> SQLMetadataStorageModule -> SqlSegmentMetadataTransactionFactory (required for IndexerSQLMetadataStorageCoordinator) -> SegmentMetadataCache.

CoreInjectorBuilder should not even load DerbyMetadataStorageModule, it should be loaded only in CliOverlord and CliCoordinator. But I decided to make this change in a separate PR so that I could test it out properly, just to be on the safe side.

Please let me know if you think I should include it in this PR itself.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. IMO, it's ok the way it is, although a comment explaining this would be nice.

{
return pollDuration;
this.pollDuration = Configs.valueOrDefault(pollDuration, Period.minutes(1));
this.useCache = Configs.valueOrDefault(useCache, true);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this is what controls whether the new caching feature is enabled, then want this to be false by default for now. We could change the default to true when it's more proven out.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I had enabled this temporarily to have all UTs and ITs work with the cache enabled.
I will disable it now as all tests seem to work as expected.
There are already some tests which run in both modes: cache enabled and disabled.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed, default is now false (disabled).

int maxPartitionNum = -1;
for (String id : unusedSegmentIds) {
final SegmentId segmentId = SegmentId.tryParse(datasource, id);
if (segmentId == null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Warn (or perhaps even throw?) if the segment ID is unparseable, since in that case, the method may not be returning the correct answer.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed, throws exception now.

for (List<String> partition : partitionedSegmentIds) {
fetchedSegments.addAll(retrieveSegmentBatchById(datasource, partition, false));
fetchedSegments.add(retrieveSegmentBatchById(datasource, partition, false));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder what this code will do exactly. There will be multiple CloseableIterator from retrieveSegmentBatchById existing at once. What effect does that have?

Does the metadata query get made lazily when the iterator first has hasNext() called? If so then it would lead to the metadata queries being issued sequentially, which seems fine. But, if the query is issued as part of iterator creation, this would lead to quite a lot of simultaneously open queries, which might cause problems with the metadata store.

Copy link
Contributor Author

@kfaraz kfaraz Feb 8, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the suggestion! Added a CloseableIterator which keeps only one result set open at a time.

while (currentCacheState == CacheState.LEADER_FIRST_SYNC_PENDING
|| currentCacheState == CacheState.LEADER_FIRST_SYNC_STARTED) {
try {
cacheStateLock.wait(5 * 60_000);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use a static constant please. Btw, if the intent here is to have a specific timeout on waiting for sync, it should be checked again after wait returns (and then continue to wait if the timeout hasn't been reached yet). It is possible for wait to return early in case of spurious wakeup.

To ensure the thread wakes up timely, all of the cache state transitions should include a notifyAll.

Copy link
Contributor Author

@kfaraz kfaraz Feb 8, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, the intent is just to avoid waiting forever.
The spurious wakeup is handled by verifyCacheIsReady (renamed to verifyCacheIsUsableAndAwaitSync()) itself.

cacheStateLock.wait(5 * 60_000);
}
catch (Exception e) {
log.noStackTrace().error(e, "Error while waiting for cache to be ready");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the only error wait will throw during normal operation is InterruptedException. Consider special-casing that, and logging at a lower level. Other exception types can continue to be logged at error level and re-thrown.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added catch for InterruptedException.

}
catch (Throwable t) {
log.error(t, "Error occurred while polling metadata store");
log.makeAlert(t, "Error occurred while polling metadata store");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Missing call to emit() after makeAlert. Also, no reason to call both log.error and log.makeAlert. The alert is logged when it is emitted.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for catching this!

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed.

throw DruidException.defensive("Cache has not been started yet");
}

currentCacheState = CacheState.LEADER_FIRST_SYNC_PENDING;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Interrupt and re-start the current sync? That could help the leader gain leadership faster.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated.

* <li>Emit metrics</li>
* </ul>
*/
private void syncWithMetadataStore()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Have you been able to benchmark this method with a cluster with lots of segments (e.g. millions)? It will need to complete before allocation can work post-leadership-election, so I'm wondering how much time that will take.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not yet. I am about to start testing on some clusters with a large number of segments. I will share the results here once the testing is done.

Copy link
Contributor Author

@kfaraz kfaraz Feb 10, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ran some tests on a local cluster with 600K segments.
Full sync takes about 1 min.
Delta sync (fetch all segment IDs but payloads only for updated segments) takes about 3s.
(updated PR description to include these details)

So, after an Overlord is elected leader, it should be able to sync up within a few seconds.
I will do some more testing involving leader re-elections.

@kfaraz
Copy link
Contributor Author

kfaraz commented Feb 9, 2025

@gianm , thanks for the review!
I have updated the PR based on your feedback.

Copy link
Contributor

@gianm gianm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Latest patch LGTM, I just had a note on the error messages for "service unavailable".

throw InternalServerError.exception("Not leader anymore. Failing transaction.");
throw DruidException.forPersona(DruidException.Persona.USER)
.ofCategory(DruidException.Category.SERVICE_UNAVAILABLE)
.build("Not leader anymore. Failing transaction.");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This message isn't great for a user-facing error. The error message should help the user figure out what to do next. Better would be like: "This API is not currently available. Please try your request again."

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated.

@gianm gianm merged commit bb3fb1b into apache:master Feb 11, 2025
74 checks passed
@kfaraz kfaraz deleted the cache_segments_on_overlord branch February 11, 2025 17:09
@kfaraz
Copy link
Contributor Author

kfaraz commented Mar 4, 2025

For anyone trying out this patch, please make sure to include #17772 as well.

kfaraz added a commit that referenced this pull request Mar 26, 2025
Description
-----------
#17653 introduces a cache for segment metadata on the Overlord.
This patch is a follow up to that to make the cache more robust, performant and debug-friendly.

Changes
---------
- Do not cache unused segments
This significantly reduces sync time in cases where the cluster has a lot of unused segments.
Unused segments are needed only during segment allocation to ensure that a duplicate ID is not allocated.
This is a rare DB query which is supported by sufficient indexes and thus need not be cached at the moment.
- Update cache directly when segments are marked as unused to avoid race conditions with DB sync.
- Fix NPE when using segment metadata cache with concurrent locks.
- Atomically update segment IDs and pending segments in a `HeapMemoryDatasourceSegmentCache`
using methods `syncSegmentIds()` and `syncPendingSegments()` rather than updating one by one.
This ensures that the locks are held for a shorter period and the update made to the cache is atomic.

Main updated classes
----------------------
- `IndexerMetadataStorageCoordinator`
- `OverlordDataSourcesResource`
- `HeapMemorySegmentMetadataCache`
- `HeapMemoryDatasourceSegmentCache`

Cleaner cache sync
--------------------
In every sync, the following steps are performed for each datasource:

- Retrieve ALL used segment IDs from metadata store
- Atomically update segment IDs in cache and determine list of segment IDs which need to be refreshed.
- Fetch payloads of segments that need to be refreshed
- Atomically update fetched payloads into the cache
- Fetch ALL pending segments
- Atomically update pending segments into the cache
- Clean up empty intervals from datasource caches
cecemei pushed a commit to cecemei/druid that referenced this pull request Mar 31, 2025
Description
-----------
apache#17653 introduces a cache for segment metadata on the Overlord.
This patch is a follow up to that to make the cache more robust, performant and debug-friendly.

Changes
---------
- Do not cache unused segments
This significantly reduces sync time in cases where the cluster has a lot of unused segments.
Unused segments are needed only during segment allocation to ensure that a duplicate ID is not allocated.
This is a rare DB query which is supported by sufficient indexes and thus need not be cached at the moment.
- Update cache directly when segments are marked as unused to avoid race conditions with DB sync.
- Fix NPE when using segment metadata cache with concurrent locks.
- Atomically update segment IDs and pending segments in a `HeapMemoryDatasourceSegmentCache`
using methods `syncSegmentIds()` and `syncPendingSegments()` rather than updating one by one.
This ensures that the locks are held for a shorter period and the update made to the cache is atomic.

Main updated classes
----------------------
- `IndexerMetadataStorageCoordinator`
- `OverlordDataSourcesResource`
- `HeapMemorySegmentMetadataCache`
- `HeapMemoryDatasourceSegmentCache`

Cleaner cache sync
--------------------
In every sync, the following steps are performed for each datasource:

- Retrieve ALL used segment IDs from metadata store
- Atomically update segment IDs in cache and determine list of segment IDs which need to be refreshed.
- Fetch payloads of segments that need to be refreshed
- Atomically update fetched payloads into the cache
- Fetch ALL pending segments
- Atomically update pending segments into the cache
- Clean up empty intervals from datasource caches
@kgyrtkirk kgyrtkirk added this to the 33.0.0 milestone Apr 14, 2025
gianm added a commit that referenced this pull request Apr 25, 2025
)

* Some debug configs

* use postgresql as the default metadata store and set a few debug log

* Add s3 extension, update local storage directory, use emoji in website title

* Update favicon, easier to find the console tab

* Add indexer server, add some basic security config, updated historical and broker to use the common druid root directory

* Some policy config

* add checks for SegmentMetadataQuery

* Add thread.sleep for flaky.

* auth config

* format, and remove temp folder rules

* added NoopPolicyEnforcer and RestrictAllTablesPolicyEnforcer class

* Support pushing and streaming task payload for HDFS (#17742)

Implement pushTaskPayload/streamTaskPayload as introduced in #14887
for HDFS storage to allow larger mm-less ingestion payloads when using
HDFS as the deep storage location.

* Remove usages of deprecated API Files.write() (#17761)

* Add deprecated com.google.common.io.Files#write to forbiddenApis

* Replace deprecated Files.write()

* Doc: Fix description typo for sqlserver metadata store (#17771)

Mistakenly categories under deep storage instead of metadata store.

* Fix binding of segment metadata cache on CliOverlord (#17772)

Changes
---------
- Bind `SegmentMetadataCache` only once to
`HeapMemorySegmentMetadataCache` in `SQLMetadataStorageDruidModule`
- Invoke start and stop of the cache from `DruidOverlord` rather than on lifecycle start/stop
- Do not override the binding in `CliOverlord`

* Docs: Remove semicolon from example (#17759)

* Restrict segment metadata kill query till maxInterval from last kill task time (#17770)

Changes
---------
- Use `maxIntervalToKill` to determine search interval for killing unused segments.
- If no segment has been killed for the datasource yet, use durationToRetain

* Update the Supervisor endpoint to not restart the Supervisor if the spec was unmodified (#17707)

Add an optional query parameter called skipRestartIfUnmodified to the
/druid/indexer/v1/supervisor endpoint. Callers can set skipRestartIfUnmodified=true
to not restart the supervisor if the spec is unchanged.

Example:

curl -X POST --header "Content-Type: application/json" -d @supervisor.json
localhost:8888/druid/indexer/v1/supervisor?skipRestartIfUnmodified=true

* Reduce noisy coordinator logs (#17779)

* Emit time lag from Kafka supervisor (#17735)

Changes
---------
- Emit time lag from Kafka similar to Kinesis as metrics `ingest/kafka/lag/time`,
`ingest/kafka/maxLag/time`, `ingest/kafka/avgLag/time`
- Add new method in `KafkaSupervisor` to fetch timestamps of latest records in stream to compute time lag
- Add new field `emitTimeLagMetrics` in `KafkaSupervisorIOConfig` to toggle emission of new metrics

* fix processed row formatting (#17756)

* Web console: add suggestions for table status filtering. (#17765)

* suggest filter values when known

* update snapshots

* add more d

* fix load rule clamp

* better segment timeline init

* Remove all usages of skife config (#17776)


Changes
---------
- Usages of skife config had been deprecated in #14695 and
`LegacyBrokerParallelMergeConfig` is the last config class that still uses it.
- Remove `org.skife.config` from pom, licenses, log4j2.xml, etc.
- Add validation for deleted property paths in `StartupInjectorBuilder.PropertiesValidator`
- Use the replacement flattened configs (which remove the `.task` and `.pool` substring)

* Add field `taskLimits` to worker select strategies (#16889)

Changes
---------
- Add field `taskLimits` to the following worker select strategies
`equalDistribution`, `equalDistributionWithCategorySpec`, `fillCapacityWithCategorySpec`, `fillCapacity`
- Add sub-fields `maxSlotCountByType` and `maxSlotRatioByType` to `taskLimits`
- Apply these limits per worker when assigning new tasks

---------
Co-authored-by: sviatahorau <mikhail.sviatahorau@deep.bi>
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

* remove NullValueHandlingConfig, NullHandlingModule, NullHandling (#17778)

* Docs: Add SQL query example (#17593)

* Docs: Add query example

* Update after review

* Update query

* Update docs/api-reference/sql-api.md

---------

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>

* More logging cleanup on Overlord (#17780)

* Remove maven.twttr repo from pom (#17797)

remove usage of dependency:go-offline from build scripts - as it tries to download excluded artifacts

---------

Co-authored-by: Zoltan Haindrich <kirk@rxd.hu>

* fix bug (#17791)

* Log query stack traces for DEVELOPER and OPERATOR personas. (#17790)

Currently, query stack traces are logged only when "debug: true" is set
in the query context. This patch additionally logs stack traces targeted
at the DEVELOPER or OPERATOR personas, because for these personas, stack
traces are useful more often than not.

We continue to omit stack traces by default for USER and ADMIN, because
these personas are meant to interact with the API, not with code or logs.
Skipping stack traces minimizes clutter in the logs.

* Set useMaxMemoryEstimates=false for MSQ tasks (#17792)

* Web console: fix go to task selecting correct task type (#17788)

* fix go to task selecting correct task type

* support autocompact also

* support scheduled_batch, refactor

* one more state and update tests

* Enable ComponentSuppliers to run queries using Dart (#17787)



Enables Calcite*Test-s and quidem tests to run queries with Dart.

needed some minor tweaks:

    changed to use interfaces at some places
    renamed DartWorkerClient to DartWorkerClientImpl and made DartWorkerClient an interface
    reused existing parts of the MSQ test system to run the query

* Fix single container config creates failing peon tasks (#17794)

* Fix single container config creates failing peon tasks

* More obvious array error output

* Update `k8s-jobs.md` reference (#17805)

Signed-off-by: Emmanuel Ferdman <emmanuelferdman@gmail.com>

* Footer Copyright Year Update (#17751)

* Update docusaurus.config.js

* Update docusaurus.config.js

* [Revert] Reduce number of metadata transaction retries (#17808)

* Revert "Run JDK 21 workflows with latest JDK. (#17694)" (#17806)

* Revert "Run JDK 21 workflows with latest JDK. (#17694)"

This reverts commit 31ede5c

* Review comments.

* Review comments.

* Revert "reject publishing actions with a retriable error code if a earlier task is still publishing (#17509)"

This reverts commit aca56d6.

* Fix unstable tests after #17787 and dart usage in quidem-ut (#17814)

* fixes

* fix cleanup

* Use "mix" shuffle spec for target size with nil clusterBy. (#17810)

When a nil clusterBy is used, we have no way of achieving a particular
target size, so we need to fall back to a "mix" spec (unsorted single
partition).

This comes up for queries like "SELECT COUNT(*) FROM FOO LIMIT 1" when
results use a target size, such as when we are inserting into another
table or when we are writing to durable storage.

* Docs: Recommend using runtime property javaOptsArray instead of javaOpts

* Add minor checks in jetty utils (#17817)

Add minor checks in jetty utils class

* CI improvement: Leverage cancelled() instead of always() for CI jobs (#17819)

* Make MSQ tests use the same datasets as other similar tests (#17818)

MSQ tests had their own way of creating the segments/etc - this have lead to that custom datasets didn't worked with them.
This patch alters a few things to make it possible to access CompleteSegment for the active segments - which fixed the issue and also enabled the removal of the extra loading codes.

* Add unnest tests to quidem (#17825)

This PR adds the sql-native unnest tests to quidem. This set of tests has 6392 queries in total, with 5247 positive tests and 1145 negative tests.

* Web console: show loader on aux queries (#17804)

* show loader on aux queries

* show supervisors if not on page 0

* refactor

* fix bug fetching data when columns are added or removed

* update test

* Use compaction dynamic config to enable compaction supervisors (#17782)

Changes
---------
- Remove runtime property object `CompactionSupervisorConfig`
- Add fields `useSupervisors` and `engine` to cluster-level compaction dynamic config
- Remove unused field `useAutoScaleSlots`

* Retry segment publish task actions without holding locks (#17816)

#17802 reverted a retry of failed segment publish actions.

This patch attempts to address the original issue by retrying the segment publish task actions
on the client (i.e. task) side without holding any locks so that other transactions are not blocked.
Changes

    Add retries to TransactionalSegmentPublisher
    Add field retryable to SegmentPublishResult
    Remove class DataStoreMetadataUpdateResult and use SegmentPublishResult instead

* Add the capability to turboload segments onto historicals (#17775)

Add the capability to set Historicals into a turbo loading mode,
to focus on loading segments at the cost of query performance.

Context
--------
Currently, when a new Historical is started, it initially starts out using a bootstrap thread pool.
It uses this thread pool to load any existing cached segments and broadcast segments.
Once it loads any segments from both these sources, the historical switches to a smaller thread-pool
and begins to serve queries.

In certain cases, it would be useful to have the historical switch back to this mode,
and focus on loading segments, either to continue loading the initial non-bootstrap segments,
or to catch up with assigned segments.

This PR adds a coordinator dynamic config that allows servers to be configured to use
the larger bootstrap threadpool to load segments faster.

Changes
---------
- Added a new dynamic coordinator configuration, `turboLoadingNodes`.
- Ignore  `druid.coordinator.loadqueuepeon.http.batchSize` for servers in `turboLoadingNodes`
- Add API on historical to return loading capabilities i.e. num loading threads in normal and turbo mode

* Fix resource leak for GroupBy query merge buffer when query matched result cache (#17823)

* Fix resource leak for GroupBy query merge buffer when match result cache

* Fix resource leak for GroupBy query merge buffer when match result cache

* Add test

* Add test

* Add comment

* Add test

* Add metric and simulation test for turbo loading mode (#17830)

Changes
---------
- Add field `loadingMode` to `SegmentChangeStatus`
- Including loading mode in `DataSegmentChangeResponse`
- Include loading mode in the `description` of metrics emitted from `HttpLoadQueuePeon`
- Add simulation test to verify loading mode metrics

* Update query example (#17811)

* String util upgrade for jdk9+ (#17795)

* Update StringUtils.replace() after fix in JDK9

* Upgrade optimized string replace algorithm

* Update methods by re-using declared StringUtils#replace method

* Replace hard-coded UTF-8 encodings with StandardCharsets

* Documentation Fix (#17826)

* Enable to run quidem tests against multiple configurations; add conditionals; cleanup framework init (#17829)

* cleans up `SqlTestFramework` initialization to leave the `OverrideModule` empty - so that tests could more easily take over parts
* remove the `QueryComponentSupplier#createEngine`  factory method - instead uses a `Class<SqlEngine>` and use the `injector` to initialize it
* enables the usage of `!disabled <supplier> <message>` - to mark cases which are not yet supported with a specific configuration for some reason
* fixes that `datasets` was not respecting the `rollup` specification of the ingest
* enables to use `MultiComponentSupplier` backed tests - these will turn into matrix tests over multiple componentsuppliers - enabling running the same testcase in different scenarios

* Fix failing test in DimensionSchemaUtilsTest (#17832)

* Improve performance of segment metadata cache on Overlord (#17785)

Description
-----------
#17653 introduces a cache for segment metadata on the Overlord.
This patch is a follow up to that to make the cache more robust, performant and debug-friendly.

Changes
---------
- Do not cache unused segments
This significantly reduces sync time in cases where the cluster has a lot of unused segments.
Unused segments are needed only during segment allocation to ensure that a duplicate ID is not allocated.
This is a rare DB query which is supported by sufficient indexes and thus need not be cached at the moment.
- Update cache directly when segments are marked as unused to avoid race conditions with DB sync.
- Fix NPE when using segment metadata cache with concurrent locks.
- Atomically update segment IDs and pending segments in a `HeapMemoryDatasourceSegmentCache`
using methods `syncSegmentIds()` and `syncPendingSegments()` rather than updating one by one.
This ensures that the locks are held for a shorter period and the update made to the cache is atomic.

Main updated classes
----------------------
- `IndexerMetadataStorageCoordinator`
- `OverlordDataSourcesResource`
- `HeapMemorySegmentMetadataCache`
- `HeapMemoryDatasourceSegmentCache`

Cleaner cache sync
--------------------
In every sync, the following steps are performed for each datasource:

- Retrieve ALL used segment IDs from metadata store
- Atomically update segment IDs in cache and determine list of segment IDs which need to be refreshed.
- Fetch payloads of segments that need to be refreshed
- Atomically update fetched payloads into the cache
- Fetch ALL pending segments
- Atomically update pending segments into the cache
- Clean up empty intervals from datasource caches

* GroupBy: Fix offsets on outer queries. (#17837)

Prior to this patch, an offset specified on a groupBy that itself has an
inner groupBy would lead to an error like "Cannot push down offsets". This
happened because of a violated assumption: the processing logic assumes that
offsets have been pushed into limits (so limit pushdown optimizations can
safely be used).

This patch adjusts processing to incorporate offsets into limits during
processing of subqueries. Later on, in post-processing, offsets are applied
as written.

* Enable build cache for web-console (#17831)

* run audit fix (#17836)

* Do not block task actions on Overlord if segment metadata cache is syncing (#17824)

* Do not use segment metadata cache until leader has synced

* Read from cache only when synced, but write even if sync is pending

* Fix compilation

* Fix checkstyle, test

* Revert some extra changes

* Add 3 modes of cache usage

* Move enum to SegmentMetadataCache

* Run tests in all 3 cache modes

* Fix docs and IT configs

* Fix config binding

* Remove forbidden api

* Fix typos, docs and enum casing

* Fix doc

* Add json, array, aggregation function tests to quidem (#17842)

This PR adds the sql-native portion of the json, array, and aggregation function tests to quidem.  It adds a total of 9965 queries, with 6752 positive tests and 3213 negative tests.

* Optionally include Content-Disposition header in statement results API response (#17840)

Adds support for an optional filename query parameter to the /druid/v2/sql/statements/{queryId}/results API. When provided, the response will include a header Content-Disposition: attachment; filename="{filename}", which will instruct a web browser to save the response as a file rather than displaying it inline.

This save-as-attachment behavior could be achieved by adding a "download" attribute to the results link, but this only works for same-origin URLs (as in the Web Console). If the UI origin is different from the Druid API origin, browsers will ignore the attribute and serve the results inline, which is poor UX for files that are potentially very large.

For the sake of consistency, all successful responses in SqlStatementResource.doGetResults may include this header, even if there are no results.
Release note

Improved: The "Get query results" statements API supports an optional filename query parameter. When provided, the response will instruct web browsers to save the results as a file instead of showing them inline (via the Content-Disposition header).

* Web console: download follow up (#17845)

* set filename

* update download button

* added markdown support

* add test

* better download

* fix TSV

* better download behaviour and tests

* always show download all button

* Fix flaky unit tests in SegmentBootstrapperTest and KinesisIndexTaskTest (#17841)

Changes:
- Fix flakiness in SegmentBootstrapperTest
- Make TestSegmentCacheManager thread safe by moving from ArrayList to CopyOnWriteArrayList
- Modify assertions to disregard list ordering since order of list modifications is not always deterministic
- Fix flaky KinesisIndexTask tests.

* Web console: responding to user feedback about the explore view and fixing bugs (#17844)

* better debounce

* better cumpose filter

* hook up preview filters

* better stack handling

* fix some props

* refactor stack to facet

* fix hover part 1

* line hover part 2

* start adding moduleWhere

* info popover

* add filter icon

* toggle button

* module filter bar

* update TestSegmentCacheManager

* revert some style changes

* validate datasource in CachingClusteredClient as well

* fix build failure and update style

* changes

* add inlineds test

* add sanity check on segment

* inject policy enforcer

* add PolicyEnforcer binding in MSQTestBase

* add check in SinkQuerySegmentWalker

* more tests in realtime server

* revert config change in examples

* revert config change in integration test config

* more tests in msq

* another test for unnest in msq

* add support for policy from extension

* more test

* refactor MSQTaskQueryMakerTest to use an instance of MSQTaskQueryMaker

* Add test for JoinDataSource

* add policyEnforcer to withPolicies, and validate segment after segment mapping

* fix binding and test

* add policy module

* mock planner toolbox

* revert some injection

* add test for stream appenderator

* update PolicyEnforcer to take ReferenceCountingSegment as param

* update to QueryLifecycleTest

* update to SqlTestFramework

* pass enforcer to BroadcastJoinSegmentMapFnProcessor and add test. PolicyEnforcer should also deal with multiple layer wrapped segments/

* ReferenceCountingSegment is not allowed to wrap with a SegmentReference, and PolicyEnforcer now validates all segments, remove test cases for inline/lookup.

* moving ReferenceCountingSegment to another pr

* Revert "Merge remote-tracking branch 'cecemei/debug' into policy"

This reverts commit 25ffb7c, reversing
changes made to 1e6632f.

---------

Signed-off-by: Emmanuel Ferdman <emmanuelferdman@gmail.com>
Co-authored-by: Virushade <70288012+GWphua@users.noreply.github.com>
Co-authored-by: Eyal Yurman <eyal.yurman@gmail.com>
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
Co-authored-by: Frank Chen <frank.chen021@outlook.com>
Co-authored-by: Chetan Patidar <122344823+chetanpatidar26@users.noreply.github.com>
Co-authored-by: aho135 <ash023@ucsd.edu>
Co-authored-by: Adithya Chakilam <35785271+adithyachakilam@users.noreply.github.com>
Co-authored-by: Vadim Ogievetsky <vadim@ogievetsky.com>
Co-authored-by: Misha <mikhailsviatohorof@gmail.com>
Co-authored-by: sviatahorau <mikhail.sviatahorau@deep.bi>
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Clint Wylie <cwylie@apache.org>
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
Co-authored-by: Zoltan Haindrich <kirk@rxd.hu>
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
Co-authored-by: Emmanuel Ferdman <emmanuelferdman@gmail.com>
Co-authored-by: Om Kenge <88768848+omkenge@users.noreply.github.com>
Co-authored-by: Karan Kumar <karankumar1100@gmail.com>
Co-authored-by: Lars Francke <lars.francke@stackable.tech>
Co-authored-by: Adarsh Sanjeev <adarshsanjeev@gmail.com>
Co-authored-by: Akshat Jain <akjn11@gmail.com>
Co-authored-by: Andy Tsai <61856143+weishiuntsai@users.noreply.github.com>
Co-authored-by: Maytas Monsereenusorn <maytasm@apache.org>
Co-authored-by: jtuglu-netflix <jtuglu@netflix.com>
Co-authored-by: Lucas Capistrant <capistrant@users.noreply.github.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants