-
Notifications
You must be signed in to change notification settings - Fork 3.7k
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
New: Add DDSketch in extensions-contrib #15049
Conversation
...b/ddsketch/src/main/java/org/apache/druid/query/aggregation/ddsketch/DDSketchAggregator.java
Fixed
Show fixed
Hide fixed
...b/ddsketch/src/main/java/org/apache/druid/query/aggregation/ddsketch/DDSketchAggregator.java
Fixed
Show fixed
Hide fixed
...t/java/org/apache/druid/query/aggregation/ddsketch/DDSketchToQuantilePostAggregatorTest.java
Fixed
Show fixed
Hide fixed
.../java/org/apache/druid/query/aggregation/ddsketch/DDSketchToQuantilesPostAggregatorTest.java
Fixed
Show fixed
Hide fixed
...tch/src/main/java/org/apache/druid/query/aggregation/ddsketch/DDSketchAggregatorFactory.java
Fixed
Show resolved
Hide resolved
Nice. I had taken a cursory look at this paper a while back. But it wasn't clear to me if DDSketche has any advantage over https://datasketches.apache.org/docs/Quantiles/QuantilesSketchOverview.html. We already use data sketches library in druid. Have you looked at this library? |
We have been trying to make QuantilesSketches, specifically the
but as the DDSketch paper points out, when the main operation is capturing the p9x values, it is not as accurate when using sketches that operate on rank error. a p98 can vary wildly from p99 in a long-tail distribution. This is where DDSketch comes in, it provides a relative-to-actual-value error guarantee (uniform error). We are happy with its performance and accuracy with a long tail distribution that spans from a few milliseconds to 10 minutes. As an example, it ultimately does not matter if the true p99 is 550 seconds but the calculated p99 is 555 seconds. A big painpoint on datasketches is tuning and understanding K. Tuning K for queries for calculating a stable p9x is painful. Since QuantilesSketches return values are non-deterministic, the same query with the same underlying dataset can return values that change every time a query is fired. This K value can be bumped upwards to provide higher value stability, but at the cost of higher memory consumption (often causing heap-to-disk events) and return much slower. Furthermore, the This extension makes use of the |
Thank you for the explanation @hfukada. I will look at this PR soon. |
@hfukada - did you also try out ReqSketch (it's not integrated into druid yet) but also claims to offer higher accuracy as rank increases? |
I did not try it no. I did try to use KLL sketches, which ReqSketch is based off of, in Druid but the insertion/merge time guarantees were not as strong as DDSketch. I am ultimately not interested in rank-error. Because that ReqSketch paper talks about "no generalizations about data distribution" it loses me on interest about its ability to accurately characterize long-tail distributions. Casually mentioning that [p9X values are exactly that ReqSketch is good at], but not demonstrating its ability to handle these hard cases leaves me unsatisfied with the results presented. In addition to this, it's another "random" algorithm. as in there is some element of randomness built into the sketch. I believe there is a strong value in being able to return deterministic values. I had to chuckle at this line talking about DDSketch's shortcomings:
DDSketch is not the answer for every problem, I agree. Certainly not sorting or taking p99 of strings. Maybe even not generally applicable. However, I deal exactly with percentiles >0.5, care most about p9x values and doing these calculations fast. It is the algorithm I desire most after having played with it in my own clusters, I am happy with its accuracy, storage, and query performance. |
Is there anything else you need to get this merged? |
@hfukada - It's on my radar. I haven't got the time yet. will look into it soon. |
|name|A String for the output (result) name of the calculation.|yes| | ||
|fieldName|A String for the name of the input field (can contain sketches or raw numeric values).|yes| | ||
|relativeError||Describes the precision in which to store the sketch. Must be a number between 0 and 1.|no, defaults to 0.01 (1% error)| | ||
|numBins|Total number of bins the sketch is allowed to use to describe the distribution. This has a direct impact on max memory used|no, defaults to 1000| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can you add some more details about the trade-offs?
extensions-contrib/ddsketch/pom.xml
Outdated
<dependency> | ||
<groupId>com.google.guava</groupId> | ||
<artifactId>guava</artifactId> | ||
<version>31.1-jre</version> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
need not be hardcoded here.
extensions-contrib/ddsketch/pom.xml
Outdated
<dependency> | ||
<groupId>com.google.code.findbugs</groupId> | ||
<artifactId>jsr305</artifactId> | ||
<version>2.0.1</version> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
lets not hardcode the version here.
if (obj instanceof Number) { | ||
synchronized (this) { | ||
this.histogram.accept(((Number) obj).doubleValue()); | ||
} | ||
} else if (obj instanceof DDSketch) { | ||
synchronized (this) { | ||
this.histogram.mergeWith((DDSketch) obj); | ||
} | ||
} else { | ||
throw new IAE( | ||
"Expected a number or an instance of DDSketch, but received [%s] of type [%s]", | ||
obj, | ||
obj.getClass() | ||
); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if (obj instanceof Number) { | |
synchronized (this) { | |
this.histogram.accept(((Number) obj).doubleValue()); | |
} | |
} else if (obj instanceof DDSketch) { | |
synchronized (this) { | |
this.histogram.mergeWith((DDSketch) obj); | |
} | |
} else { | |
throw new IAE( | |
"Expected a number or an instance of DDSketch, but received [%s] of type [%s]", | |
obj, | |
obj.getClass() | |
); | |
} | |
synchronized (this) { | |
if (obj instanceof Number) { | |
this.histogram.accept(((Number) obj).doubleValue()); | |
} else if (obj instanceof DDSketch) { | |
this.histogram.mergeWith((DDSketch) obj); | |
} else { | |
throw new IAE( | |
"Expected a number or an instance of DDSketch, but received [%s] of type [%s]", | |
obj, | |
obj.getClass() | |
); | |
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit : above seems a better structure
* Each bounded lower collapsing store yields a max size of numBins * 8 bytes (size Of Double) in terms of size. | ||
* Since the sketch contains a store for positive values and negative values, a fully filled sketch at maximum would contain: | ||
* 2 * numBins * 8Bytes for storage. Other tracked members of the serialized sketch are constant, | ||
* so we add 1k as buffer for these members. These members include mapping reconstruction, and zero counts. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
where are you adding this buffer? And is this something you have measured?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, that was an old comment. I have updating the doc to reference the 12 descriptor bytes.
if (combined == null) { | ||
combined = DDSketches.collapsingLowestDense(relativeError, numBins); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this seems unnecessary since combined will never be null.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Took a look at this, the changes look good other than the comments left and merge conflict!
Thanks for the PR!
@Override | ||
public byte[] toBytes(@Nullable DDSketch val) | ||
{ | ||
if (val == null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should the sketch serialize to a null on an empty sketch as well? It would save a bit of time serde (though I'm not sure how much time it would save), and help while merging sketches. I believe this is how quantile sketch extension does it as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would prefer to keep nulls explicitly null. if the value is null, I feel like it should be retained as null?
This is me trying to be an advocate for keeping the representation of the data be true.
return null; | ||
} | ||
ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); | ||
readOnlyBuffer.limit(buffer.position() + numBytes); |
Check failure
Code scanning / CodeQL
User-controlled data in arithmetic expression High
user-provided value
This arithmetic expression depends on a
user-provided value
This arithmetic expression depends on a
user-provided value
@hfukada Thanks for the changes. The PR looks good overall other than the build issues. |
A couple of build failures seem unrelated to my changes? Static Checks CI / intellij-inspections (pull_request): https://github.com/apache/druid/actions/runs/7388106247/job/20112155029?pr=15049 I did not touch any of these files and I'm confused why these are failing.
Unit & Integration tests CI / standard-its / integration-query-tests-middleManager (centralized-datasource-schema) / centralized-datasource-schema integration test (Compile=jdk8, Run=jdk8, Indexer=middleManager, Mysql=com.mysql.jdbc.Driver) (pull_request):
Also does not look to be my fault, and I cannot retry it. Code scanning results / CodeQL Failing after 4s — 1 new alert including 1 high severity security vulnerability |
@hfukada Looks like the test failure was a flaky test, and seems to be resolved now, I'll document that test as flaky. On going through the code, the security issue seems to be a false positive as well, we can ignore that failure. |
|type|Must be "ddSketch" |yes| | ||
|name|A String for the output (result) name of the calculation.|yes| | ||
|fieldName|A String for the name of the input field (can contain sketches or raw numeric values).|yes| | ||
|relativeError||Describes the precision in which to store the sketch. Must be a number between 0 and 1.|no, defaults to 0.01 (1% error)| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
|relativeError||Describes the precision in which to store the sketch. Must be a number between 0 and 1.|no, defaults to 0.01 (1% error)| | |
|relativeError|Describes the precision in which to store the sketch. Must be a number between 0 and 1.|no, defaults to 0.01 (1% error)| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thank you, I'm sorry I missed that second go around
|relativeError||Describes the precision in which to store the sketch. Must be a number between 0 and 1.|no, defaults to 0.01 (1% error)| | ||
|numBins|Total number of bins the sketch is allowed to use to describe the distribution. This has a direct impact on max memory used. The more total bins available, the larger the range of accurate quantiles.* |no, defaults to 1000| | ||
|
||
* Examples Tuning: With relative accuracy of 2%, only 275 bins are required to cover values between 1 millisecond and 1 minute. 800 bins are required to cover values between 1 nanosecond and 1 day. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The asterisk renders in Markdown as a bullet point here. Since this is a relative short description, considering moving this line to the numBins
description.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
moved into the table 👍
|field|A computed ddSketch.|yes| | ||
|fractions|list of doubles from 0 to 1 of the quantiles to compute|yes| | ||
|
||
|property|description|required?| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's hard to differentiate these two tables. Consider a single table in which fractions
is described similar to below:
To compute [...], supply a list of doubles ranging from 0 to 1. To compute a single quantile, supply a single double value between 0 and 1.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
oh, whoops this should actually read quantileFromDDsketch
which takes fraction
instead of fractions
similar to the other sketch aggregators. Updated with fixed documentation and a header per aggregator.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for updating the header for each. Since it can still be easily overlooked (quantile(s) and fraction(s)), consider adding an intro to each section. I'll leave some suggestions.
|field|A computed ddSketch.|yes| | ||
|fractions|list of doubles from 0 to 1 of the quantiles to compute|yes| | ||
|
||
|property|description|required?| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for updating the header for each. Since it can still be easily overlooked (quantile(s) and fraction(s)), consider adding an intro to each section. I'll leave some suggestions.
|fractions|list of doubles from 0 to 1 of the quantiles to compute|yes| | ||
|
||
#### quantileFromDDSketch | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Use `quantileFromDDSketch` to fetch a single quantile. | |
```json | ||
{ | ||
"type" : "quantilesFromDDSketch", | ||
"name" : <output_name>, | ||
"field" : <reference to DDSketch>, | ||
"fractions" : <array of doubles in [0,1]> | ||
} | ||
``` |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Move this into the previous section so all quantilesFromDDSketch
content stays together
} | ||
``` | ||
|
||
Single quantiles may be fetched as well. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Single quantiles may be fetched as well. |
Remove this line here, and move it as the introduction to the single quantile section
} | ||
``` | ||
|
||
and make queries using the following aggregator + post-aggregator: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
and make queries using the following aggregator + post-aggregator: | |
You can query pre-aggregated sketches using the following aggregator and post-aggregator: |
I'm not sure if this changes the meaning of the sentence; please confirm
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What person perspective should I be writing these docs? I feel like the sudden introduction of You
here and in the postAggregator section feels jarring to me.
I have altered the suggested changes to:
To compute approximate quantiles, use `quantilesFromDDSketch` to query for a set of quantiles or `quantileFromDDSketch` to query for a single quantile. Call these post-aggregators on the sketches created by the `ddSketch` aggregators.
and
Quantiles can queried out from the pre-aggregated sketches using the following aggregator + post-aggregator:
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Try to use second person in the docs -- "you" instead of "we." See the docs style guide here: https://github.com/apache/druid/blob/master/docs/development/docs-contribute.md#style-checklist
The first change looks good. For the second one, "can queried out" seems odd, but perhaps this instead:
Compute quantiles from the pre-aggregated sketches using the following aggregator and post-aggregator:
Users can query for a set of quantiles using the `quantilesFromDDSketch` post-aggregator on the sketches created by the `ddSketch` aggregators. | ||
|
||
#### quantilesFromDDSketch |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Users can query for a set of quantiles using the `quantilesFromDDSketch` post-aggregator on the sketches created by the `ddSketch` aggregators. | |
#### quantilesFromDDSketch | |
To compute approximate quantiles, you can use the `quantilesFromDDSketch` to query for a set of quantiles or `quantileFromDDSketch` to query for a single quantile. Call these post-aggregators on the sketches created by the `ddSketch` aggregators. | |
#### quantilesFromDDSketch | |
Use `quantilesFromDDSketch` to compute a set of quantiles. | |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Mention both post-aggregators in the introduction to the section
|type|Must be "quantilesFromDDSketch" |yes| | ||
|name|A String for the output (result) name of the calculation.|yes| | ||
|field|A computed ddSketch.|yes| | ||
|fractions|list of doubles from 0 to 1 of the quantiles to compute|yes| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
|fractions|list of doubles from 0 to 1 of the quantiles to compute|yes| | |
|fractions|Array of doubles from 0 to 1 of the quantiles to compute|yes| |
|type|Must be "quantileFromDDSketch" |yes| | ||
|name|A String for the output (result) name of the calculation.|yes| | ||
|field|A computed ddSketch.|yes| | ||
|fraction|double from 0 to 1 of the quantile to compute|yes| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
|fraction|double from 0 to 1 of the quantile to compute|yes| | |
|fraction|A double from 0 to 1 of the quantile to compute|yes| |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you for the doc updates!
Sorry for the follow up comment. Please also add a new entry to the extensions doc: https://github.com/apache/druid/blob/master/docs/configuration/extensions.md |
- Based off of http://www.vldb.org/pvldb/vol12/p2195-masson.pdf and uses the corresponding https://github.com/DataDog/sketches-java library - contains tests for post building and using aggregation/post aggregation. - New aggregator: `ddSketch` - New post aggregators: `quantileFromDDSketch` and `quantilesFromDDSketch`
Also moved aggregator ids to AggregatorUtil and PostAggregatorIds
* New: Add DDSketch-Druid extension - Based off of http://www.vldb.org/pvldb/vol12/p2195-masson.pdf and uses the corresponding https://github.com/DataDog/sketches-java library - contains tests for post building and using aggregation/post aggregation. - New aggregator: `ddSketch` - New post aggregators: `quantileFromDDSketch` and `quantilesFromDDSketch` * Fixing easy CodeQL warnings/errors * Fixing docs, and dependencies Also moved aggregator ids to AggregatorUtil and PostAggregatorIds * Adding more Docs and better null/empty handling for aggregators * Fixing docs, and pom version * DDSketch documentation format and wording
Description
ddSketch
quantileFromDDSketch
andquantilesFromDDSketch
Release note
New:
druid-ddsketch
Extension. DDSketches can be built off of raw numeric data or merged from other DDSketches that have been preaggregated. DDSketches are then used to calculate quantiles with relative error guarantees with bounded space.Key changed/added classes in this PR
DDSketchAggregatorFactory
: Sets defaults up for aggregators. notably relativeError = 0.01 and numBins = 1000DDSketchAggregator
: Does the accepting of new values and merging of other DDSketches.DDSketchBufferAggregator
: same as above but using the buffer.DDSketchComplexMetricSerde
: registers the serdeDDSketchObjectStrategy
: Strategy for managing the objects/serialization/deserializationDDSketchToQuantilePostAggregator
: Sets up the postAggregatorquantileFromDDSketch
DDSketchToQuantilesPostAggregator
: Sets up the postAggregatorquantilesFromDDSketch
DDSketchUtils
: Utility functions around serialization/deserialization usingprotobuf
This PR has: