Skip to content

Conversation

@gianm
Copy link
Contributor

@gianm gianm commented Jan 6, 2026

This patch adds query reports, and a mapping from SQL query ID to Dart query ID, to DartControllerRegistry. It is made available through a new getQueryReport method on SqlEngine, and through a new /druid/v2/sql/queries/<sqlQueryId>/reports endpoint on SqlResource.

The format of the response is a JSON object with two keys, "query" and "report". The "query" key is the same info that is available from the existing /druid/v2/sql/queries endpoint. The "report" key is a report map including an MSQ report.

New Broker configurations control retention of reports:

  • druid.msq.dart.controller.maxRetainedReportCount (default 0, meaning no reports are retained)
  • druid.msq.dart.controller.maxRetainedReportDuration (default PT0S, meaning time-based expiration is disabled)

This patch adds query reports, and a mapping from SQL query ID to
Dart query ID, to DartControllerRegistry. It is made available through
a new getQueryReport method on SqlEngine, and through a new
"/druid/v2/sql/queries/<sqlQueryId>/report" endpoint on SqlResource.

The format of the response is a JSON object with two keys, "query"
and "report". The "query" key is the same info that is available from
the existing "/druid/v2/sql/queries" endpoint. The "report" key is an
MSQ query report.

New Broker configurations control retention of reports:

- druid.msq.dart.controller.maxRetainedReportCount (default 0, meaning
  no reports are retained)
- druid.msq.dart.controller.maxRetainedReportDuration (default PT0S,
  meaning time-based expiration is disabled)
@github-actions github-actions bot added Area - Batch Ingestion Area - Querying Area - MSQ For multi stage queries - https://github.com/apache/druid/issues/12262 labels Jan 6, 2026
@gianm
Copy link
Contributor Author

gianm commented Jan 6, 2026

Added jacoco:skip because most of the files that are being complained about, are being tested with embedded tests.

@gianm gianm closed this Jan 6, 2026
@gianm gianm reopened this Jan 6, 2026
@gianm gianm added the Needs web console change Backend API changes that would benefit from frontend support in the web console label Jan 7, 2026
Comment on lines 122 to 123
synchronized (snapshotsMap) {
synchronized (that.snapshotsMap) {
Copy link
Member

Choose a reason for hiding this comment

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

is this deadlocky if multiple things are calling equals? i'm nervous at least

Copy link
Contributor Author

@gianm gianm Jan 8, 2026

Choose a reason for hiding this comment

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

Hmm, I think in theory yes. equals is only going to be used in unit tests though so it probably doesn't matter (this was added to make a test case easier to write). But, still not great, so I'll change it to copyMap().equals(that.copyMap()).

return new DartSqlClientImpl(client, jsonMapper);
}

private static String removeTrailingSlash(String path)
Copy link
Member

Choose a reason for hiding this comment

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

we also have StringUtils.maybeRemoveTrailingSlash

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ha, I thought there was something but couldn't find it. Updated.

Comment on lines 582 to 583
this.complete = true;
this.report = report;
Copy link
Member

Choose a reason for hiding this comment

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

do the order these are set in matter? I can't see any overlap, but .. didn't follow all around everything that is using complete so just wanting to make sure nothing will try to get the report after checking completed

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't believe it matters, but I switched the order anyway because that's easier than trying to prove it doesn't matter.

Copy link
Contributor

@kfaraz kfaraz left a comment

Choose a reason for hiding this comment

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

Minor nitpicks, otherwise LGTM 👍🏻

/**
* Class returned by {@link SqlResource#doGetQueryReport}, the "get query report" API.
*/
public class GetReportResponse
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: rename to GetQueryReportResponse instead?

/**
* Creates a {@link BrokerClient} that targets a specific broker using a {@link FixedServiceLocator}.
*/
private static BrokerClient createBrokerClientForBroker(EmbeddedBroker targetBroker)
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: I think it may be useful to move this method to EmbeddedServiceClient as a variation of onAnyBroker(). The new method could be onTargetBroker() which accepts the target EmbeddedBroker and the API lambda to be called on the underlying client.

{
final long timeout = 30_000;
final long deadline = System.currentTimeMillis() + timeout;
while (System.currentTimeMillis() < deadline) {
Copy link
Contributor

@kfaraz kfaraz Jan 8, 2026

Choose a reason for hiding this comment

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

Is there any "query has started running" metric that we could wait for instead of sleeping?
Nit: You could also use Stopwatch.hasMillisElapsed() instead of System.currentTimeMillis().

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 really, the metrics are emitted when a query finishes. This could likely be refactored to use the waitForResult utility in #18766 (comment) once that PR is merged.

/**
* Object returned by {@link DartControllerRegistry#getQueryInfoAndReport(String)}.
*/
public class QueryInfoAndReport
Copy link
Contributor

Choose a reason for hiding this comment

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

This class seems very similar to GetReportResponse. Do we need the two to be distinct?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I didn't want to have an unnecessary timestamp field polluting the object that represents what the API returns. This class has the timestamp so it can be used for expiration of reports.

private double heapFraction = DEFAULT_HEAP_FRACTION;

@JsonProperty("maxRetainedReportCount")
private int maxRetainedReportCount = 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we default to a small positive number, say 10? This seems like a useful feature to have out of the box.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I was being conservative here with the new functionality, and also thinking that a small number might not be that useful. A cluster that gets real usage will probably go through 10 queries pretty quickly. In a production situation it probably makes sense to set it much higher and to have a time-based expiration of at least a few minutes.

// Schedule periodic cleanup of expired reports.
if (!config.getMaxRetainedReportDuration().equals(Period.ZERO)) {
final String threadNameFormat = StringUtils.format("%s-ReportCleanupExec-%%s", getClass().getSimpleName());
final long cleanupPeriodMs = Math.max(
Copy link
Contributor

Choose a reason for hiding this comment

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

Just curious, is it necessary to do periodic cleanup?
Isn't it sufficient to just have a limit on the number of reports?

Copy link
Contributor Author

@gianm gianm Jan 8, 2026

Choose a reason for hiding this comment

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

I was thinking that in a production scenario, an admin probably wants to think about things in terms of time-based expiration (i.e. I'm going to have reports available for 5 minutes). I was thinking the count-based retention is more just to protect against the broker going OOM if there is a burst of queries. It would be better for the limit to be footprint-based, but the code for that is more complex :|

Copy link
Contributor

Choose a reason for hiding this comment

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

That makes sense. thanks!

Comment on lines +148 to +149
int reportsToRemove = completeReports.size() - config.getMaxRetainedReportCount() + 1;
if (reportsToRemove > 0) {
Copy link
Contributor

Choose a reason for hiding this comment

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

We might be able to get rid of this logic by overriding the removeEldestEntry method on the LinkedHashMap completeReports.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

True but I always find that approach ugly.

if (didRemove && completeReport != null && config.getMaxRetainedReportCount() > 0) {
synchronized (completeReports) {
// Remove reports if size is greater than maxRetainedReportCount - 1.
int reportsToRemove = completeReports.size() - config.getMaxRetainedReportCount() + 1;
Copy link
Contributor

@kfaraz kfaraz Jan 8, 2026

Choose a reason for hiding this comment

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

Since we are acquiring a lock before this, I suppose this will always be <= 1.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think so too, just being defensive.

@gianm gianm merged commit 93735f5 into apache:master Jan 8, 2026
125 of 130 checks passed
@gianm gianm deleted the sql-engines-get-reports branch January 8, 2026 19:04
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Area - Batch Ingestion Area - MSQ For multi stage queries - https://github.com/apache/druid/issues/12262 Area - Querying jacoco:skip Needs web console change Backend API changes that would benefit from frontend support in the web console

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants