Skip to content
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

Merged
merged 6 commits into from
Jan 23, 2024
Merged

Conversation

hfukada
Copy link
Contributor

@hfukada hfukada commented Sep 27, 2023

Description

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 = 1000
  • DDSketchAggregator: Does the accepting of new values and merging of other DDSketches.
  • DDSketchBufferAggregator: same as above but using the buffer.
  • DDSketchComplexMetricSerde: registers the serde
  • DDSketchObjectStrategy: Strategy for managing the objects/serialization/deserialization
  • DDSketchToQuantilePostAggregator: Sets up the postAggregator quantileFromDDSketch
  • DDSketchToQuantilesPostAggregator: Sets up the postAggregator quantilesFromDDSketch
  • DDSketchUtils: Utility functions around serialization/deserialization using protobuf

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.

@hfukada hfukada changed the title New: Add DDSketch-Druid extensions-contrib New: Add DDSketch in extensions-contrib Sep 27, 2023
@abhishekagarwal87
Copy link
Contributor

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?

@hfukada
Copy link
Contributor Author

hfukada commented Sep 28, 2023

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 quantilesDoublesSketch for over a year. The QuantilesSketches provided out of the box perform well for ranks in the middle as shown in this chart etc


This is due to the distributions around the p25-p75 are far more forgiving when operating with rank error (the error guarantee that QuantilesSketches provide). p50 values do not differ much from p51.

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 sketches-java library that Datadog released has different storage strategies to manage memory. It includes strategies for unbounded storage, storage for maintaining error guarantees at higher and lower bins. described in the comments on this source: https://github.com/DataDog/sketches-java/blob/master/src/main/java/com/datadoghq/sketch/ddsketch/DDSketches.java

This extension makes use of the collapsingLowestDense strategy to preserve error gurantees at the highest quantiles.

@hfukada hfukada requested a review from LakshSingla September 29, 2023 14:11
@abhishekagarwal87
Copy link
Contributor

Thank you for the explanation @hfukada. I will look at this PR soon.

@abhishekagarwal87
Copy link
Contributor

@hfukada - did you also try out ReqSketch (it's not integrated into druid yet) but also claims to offer higher accuracy as rank increases?
REQSketch is based on https://arxiv.org/pdf/2004.01668.pdf and there is a comparison with DD-Sketch.

@hfukada
Copy link
Contributor Author

hfukada commented Oct 4, 2023

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:

This definition only makes sense for data universes with a notion of magnitude and distance
(e.g., numerical data)

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.

@hfukada
Copy link
Contributor Author

hfukada commented Oct 24, 2023

Is there anything else you need to get this merged?

@abhishekagarwal87
Copy link
Contributor

@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|
Copy link
Contributor

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?

<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<version>31.1-jre</version>
Copy link
Contributor

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.

<dependency>
<groupId>com.google.code.findbugs</groupId>
<artifactId>jsr305</artifactId>
<version>2.0.1</version>
Copy link
Contributor

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.

Comment on lines 62 to 74
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()
);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
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()
);
}

Copy link
Contributor

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.
Copy link
Contributor

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?

Copy link
Contributor Author

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.

Comment on lines 285 to 287
if (combined == null) {
combined = DDSketches.collapsingLowestDense(relativeError, numBins);
}
Copy link
Contributor

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.

Copy link
Contributor

@adarshsanjeev adarshsanjeev left a 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) {
Copy link
Contributor

@adarshsanjeev adarshsanjeev Nov 21, 2023

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.

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

This arithmetic expression depends on a
user-provided value
, potentially causing an overflow.
This arithmetic expression depends on a
user-provided value
, potentially causing an overflow.
This arithmetic expression depends on a
user-provided value
, potentially causing an overflow.
@adarshsanjeev
Copy link
Contributor

@hfukada Thanks for the changes. The PR looks good overall other than the build issues.

@hfukada
Copy link
Contributor Author

hfukada commented Jan 3, 2024

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.

Done.

Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java:116 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java:52 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQueryWorkerFaultTolerance.java:65 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java:125 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java:55 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITKeyStatisticsSketchMergeMode.java:196 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/AzureTestUtil.java:122 -- The declared exception <code>URISyntaxException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/AzureTestUtil.java:75 -- The declared exception <code>URISyntaxException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/AzureTestUtil.java:75 -- The declared exception <code>InvalidKeyException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/AzureTestUtil.java:85 -- The declared exception <code>URISyntaxException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/AzureTestUtil.java:109 -- The declared exception <code>URISyntaxException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/GcsTestUtil.java:108 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/GcsTestUtil.java:92 -- The declared exception <code>IOException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/GcsTestUtil.java:75 -- The declared exception <code>GeneralSecurityException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/GcsTestUtil.java:75 -- The declared exception <code>IOException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/GcsTestUtil.java:102 -- The declared exception <code>IOException</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java:78 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/DruidTestRunner.java:60 -- The declared exception <code>Throwable</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java:94 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java:43 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java:80 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java:108 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/ITLocalInputSourceAllFormatSchemalessTest.java:122 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/indexer/AbstractITBatchIndexTest.java:185 -- The declared exception <code>Exception</code> is never thrown
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java:352 -- Call to <code>asList</code> with only one argument #loc
Error:  integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/utils/DruidClusterAdminClient.java:250 -- Suspicious call to 'List.contains'
Error:  integration-tests-ex/cases/pom.xml:21 -- Failed to read artifact descriptor for org.yaml:snakeyaml:jar:1.21 #loc
Error: Process completed with exit code 1.

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):
Details

[INFO] 
[INFO] -------------------------------------------------------
[INFO]  T E S T S
[INFO] -------------------------------------------------------
[INFO] Running TestSuite
2024-01-03T04:12:16,541 INFO [main] org.hibernate.validator.internal.util.Version - HV000001: Hibernate Validator 6.2.5.Final
2024-01-03T04:12:18,605 INFO [main] org.apache.druid.storage.google.GoogleStorageDruidModule - Getting jackson modules...
2024-01-03T04:12:18,706 WARN [main] org.apache.druid.initialization.Log4jShutterDownerModule - Shutdown callback registry expected class [org.apache.druid.common.config.Log4jShutdown] found [org.apache.logging.log4j.core.util.DefaultShutdownCallbackRegistry]. Skipping shutdown registry
2024-01-03T04:12:19,577 INFO [main] org.apache.druid.storage.google.GoogleStorageDruidModule - Configuring GoogleStorageDruidModule...
2024-01-03T04:12:19,954 WARN [main] org.apache.hadoop.util.NativeCodeLoader - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2024-01-03T04:12:21,313 INFO [main] org.apache.druid.server.metrics.MetricsModule - Loaded 2 monitors: org.apache.druid.curator.DruidConnectionStateListener, org.apache.druid.server.initialization.jetty.JettyServerModule$JettyMonitor
2024-01-03T04:12:21,536 INFO [main] org.apache.druid.https.SSLContextProvider - Creating SslContext for https client using config [SSLClientConfig{protocol='null', trustStoreType='null', trustStorePath='client_tls/truststore.jks', trustStoreAlgorithm='null', keyStorePath='client_tls/client.jks', keyStoreType='null', certAlias='druid', keyManagerFactoryAlgorithm='null', validateHostnames='null'}]
2024-01-03T04:12:22,825 INFO [main] org.apache.druid.https.SSLContextProvider - Creating SslContext for https client using config [SSLClientConfig{protocol='null', trustStoreType='null', trustStorePath='client_tls/truststore.jks', trustStoreAlgorithm='null', keyStorePath='client_tls/client.jks', keyStoreType='null', certAlias='druid', keyManagerFactoryAlgorithm='null', validateHostnames='null'}]
2024-01-03T04:12:23,452 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[0/240]...
2024-01-03T04:12:23,635 WARN [main] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:23,636 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Error while waiting for [http://127.0.0.1:8081/] to be ready
java.util.concurrent.ExecutionException: java.nio.channels.ClosedChannelException
	at com.google.common.util.concurrent.AbstractFuture.getDoneValue(AbstractFuture.java:592) ~[guava-32.0.1-jre.jar:?]
	at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:551) ~[guava-32.0.1-jre.jar:?]
	at com.google.common.util.concurrent.AbstractFuture$TrustedFuture.get(AbstractFuture.java:111) ~[guava-32.0.1-jre.jar:?]
	at org.apache.druid.testing.utils.DruidClusterAdminClient.lambda$waitUntilInstanceReady$1(DruidClusterAdminClient.java:268) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.testing.utils.ITRetryUtil.retryUntil(ITRetryUtil.java:61) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.testing.utils.ITRetryUtil.retryUntilTrue(ITRetryUtil.java:39) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.testing.utils.DruidClusterAdminClient.waitUntilInstanceReady(DruidClusterAdminClient.java:262) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.testing.utils.DruidClusterAdminClient.waitUntilCoordinatorReady(DruidClusterAdminClient.java:124) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.testing.utils.SuiteListener.onStart(SuiteListener.java:41) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.testng.SuiteRunner.invokeListeners(SuiteRunner.java:223) ~[testng-7.3.0.jar:?]
	at org.testng.SuiteRunner.run(SuiteRunner.java:284) ~[testng-7.3.0.jar:?]
	at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:53) ~[testng-7.3.0.jar:?]
	at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:96) ~[testng-7.3.0.jar:?]
	at org.testng.TestNG.runSuitesSequentially(TestNG.java:1218) ~[testng-7.3.0.jar:?]
	at org.testng.TestNG.runSuitesLocally(TestNG.java:1140) ~[testng-7.3.0.jar:?]
	at org.testng.TestNG.runSuites(TestNG.java:1069) ~[testng-7.3.0.jar:?]
	at org.testng.TestNG.run(TestNG.java:1037) ~[testng-7.3.0.jar:?]
	at org.apache.maven.surefire.testng.TestNGExecutor.run(TestNGExecutor.java:283) ~[surefire-testng-2.22.2.jar:2.22.2]
	at org.apache.maven.surefire.testng.TestNGXmlTestSuite.execute(TestNGXmlTestSuite.java:75) ~[surefire-testng-2.22.2.jar:2.22.2]
	at org.apache.maven.surefire.testng.TestNGProvider.invoke(TestNGProvider.java:120) ~[surefire-testng-2.22.2.jar:2.22.2]
	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384) ~[surefire-booter-2.22.2.jar:2.22.2]
	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345) ~[surefire-booter-2.22.2.jar:2.22.2]
	at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126) ~[surefire-booter-2.22.2.jar:2.22.2]
	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418) ~[surefire-booter-2.22.2.jar:2.22.2]
Caused by: java.nio.channels.ClosedChannelException
	at org.jboss.netty.channel.socket.nio.AbstractNioWorker.cleanUpWriteBuffer(AbstractNioWorker.java:433) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.socket.nio.AbstractNioWorker.writeFromUserCode(AbstractNioWorker.java:128) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink.eventSunk(NioClientSocketPipelineSink.java:84) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendDownstream(DefaultChannelPipeline.java:779) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.Channels.write(Channels.java:725) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.handler.codec.oneone.OneToOneEncoder.doEncode(OneToOneEncoder.java:71) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.handler.codec.oneone.OneToOneEncoder.handleDownstream(OneToOneEncoder.java:59) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.handler.codec.http.HttpClientCodec.handleDownstream(HttpClientCodec.java:97) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.DefaultChannelPipeline.sendDownstream(DefaultChannelPipeline.java:591) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.DefaultChannelPipeline.sendDownstream(DefaultChannelPipeline.java:582) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.Channels.write(Channels.java:704) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.Channels.write(Channels.java:671) ~[netty-3.10.6.Final.jar:?]
	at org.jboss.netty.channel.AbstractChannel.write(AbstractChannel.java:347) ~[netty-3.10.6.Final.jar:?]
	at org.apache.druid.java.util.http.client.NettyHttpClient.go(NettyHttpClient.java:396) ~[druid-processing-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.java.util.http.client.CredentialedHttpClient.go(CredentialedHttpClient.java:48) ~[druid-processing-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.java.util.http.client.AbstractHttpClient.go(AbstractHttpClient.java:33) ~[druid-processing-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	at org.apache.druid.testing.utils.DruidClusterAdminClient.lambda$waitUntilInstanceReady$1(DruidClusterAdminClient.java:265) ~[druid-integration-tests-29.0.0-SNAPSHOT.jar:29.0.0-SNAPSHOT]
	... 20 more
2024-01-03T04:12:23,680 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[0/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:28,680 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[1/240]...
2024-01-03T04:12:28,683 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Channel Closed
2024-01-03T04:12:28,683 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[1/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:28,684 WARN [HttpClient-Netty-Worker-20] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:33,683 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[2/240]...
2024-01-03T04:12:33,689 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Channel Closed
2024-01-03T04:12:33,689 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[2/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:33,690 WARN [HttpClient-Netty-Worker-21] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:38,690 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[3/240]...
2024-01-03T04:12:38,692 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Channel Closed
2024-01-03T04:12:38,692 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[3/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:38,693 WARN [HttpClient-Netty-Worker-22] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:43,693 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[4/240]...
2024-01-03T04:12:43,695 WARN [main] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:43,697 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Channel disconnected
2024-01-03T04:12:43,697 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[4/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:48,698 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[5/240]...
2024-01-03T04:12:48,700 WARN [main] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:48,700 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Channel disconnected
2024-01-03T04:12:48,700 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[5/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:53,701 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[6/240]...
2024-01-03T04:12:53,702 ERROR [main] org.apache.druid.testing.utils.DruidClusterAdminClient - Channel Closed
2024-01-03T04:12:53,702 WARN [HttpClient-Netty-Worker-25] org.apache.druid.java.util.http.client.pool.ResourcePool - Resource at key[http://127.0.0.1:8081/] was returned multiple times?
2024-01-03T04:12:53,703 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Attempt[6/240] did not pass: Task Waiting for instance to be ready: [http://127.0.0.1:8081/] still not complete. Next retry in 5000 ms
2024-01-03T04:12:58,703 INFO [main] org.apache.druid.testing.utils.ITRetryUtil - Trying attempt[7/240]...

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
Details
I don't know how other PRs pass this given that this limit(position + numBytes) is copied. I don't actually think it'll even throw an overflow exception on this line either. it's just limit. I pulled this example from the other already-merged sketches as inspiration. The buffer is readonly, and the java protobuf library implements various memory safeguards when using parseFrom(bytes).

@adarshsanjeev
Copy link
Contributor

@hfukada
The static checks would probably be resolved by rebasing the PR with master. It appears to be quite behind anyway.

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)|
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
|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)|

Copy link
Contributor Author

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.
Copy link
Member

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.

Copy link
Contributor Author

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?|
Copy link
Member

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.

Copy link
Contributor Author

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.

Copy link
Member

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?|
Copy link
Member

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

Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
Use `quantileFromDDSketch` to fetch a single quantile.

Comment on lines +83 to +75
```json
{
"type" : "quantilesFromDDSketch",
"name" : <output_name>,
"field" : <reference to DDSketch>,
"fractions" : <array of doubles in [0,1]>
}
```
Copy link
Member

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.
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
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:
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
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

Copy link
Contributor Author

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:

Copy link
Member

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:

Comment on lines 61 to 63
Users can query for a set of quantiles using the `quantilesFromDDSketch` post-aggregator on the sketches created by the `ddSketch` aggregators.

#### quantilesFromDDSketch
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
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.

Copy link
Member

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|
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
|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|
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
|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|

Copy link
Member

@vtlim vtlim left a 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!

@vtlim
Copy link
Member

vtlim commented Jan 12, 2024

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
@adarshsanjeev adarshsanjeev merged commit 3fe3a65 into apache:master Jan 23, 2024
80 of 83 checks passed
@asdf2014 asdf2014 added this to the 29.0.0 milestone Jan 24, 2024
adarshsanjeev pushed a commit to adarshsanjeev/druid that referenced this pull request Jan 24, 2024
* 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
abhishekagarwal87 pushed a commit that referenced this pull request Jan 29, 2024
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.

6 participants