You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "anshu-makkar (via GitHub)" <gi...@apache.org> on 2023/02/17 08:57:34 UTC

[GitHub] [druid] anshu-makkar opened a new pull request, #13819: Add Post Aggregators for Tuple Sketches

anshu-makkar opened a new pull request, #13819:
URL: https://github.com/apache/druid/pull/13819

   <!-- Thanks for trying to help us make Apache Druid be the best it can be! Please fill out as much of the following information as is possible (where relevant, and remove it when irrelevant) to help make the intention and scope of this PR clear in order to ease review. -->
   
   <!-- Please read the doc for contribution (https://github.com/apache/druid/blob/master/CONTRIBUTING.md) before making this PR. Also, once you open a PR, please _avoid using force pushes and rebasing_ since these make it difficult for reviewers to see what you've changed in response to their reviews. See [the 'If your pull request shows conflicts with master' section](https://github.com/apache/druid/blob/master/CONTRIBUTING.md#if-your-pull-request-shows-conflicts-with-master) for more details. -->
   
   
   <!-- Replace XXXX with the id of the issue fixed in this PR. Remove this section if there is no corresponding issue. Don't reference the issue in the title of this pull-request. -->
   
   <!-- If you are a committer, follow the PR action item checklist for committers:
   https://github.com/apache/druid/blob/master/dev/committer-instructions.md#pr-and-issue-action-item-checklist-for-committers. -->
   
   ### Description
   
   <!-- Describe the goal of this PR, what problem are you fixing. If there is a corresponding issue (referenced above), it's not necessary to repeat the description here, however, you may choose to keep one summary sentence. -->
   
   <!-- Describe your patch: what did you change in code? How did you fix the problem? -->
   
   <!-- If there are several relatively logically separate changes in this PR, create a mini-section for each of them. For example: -->
   
   #### Added New Post Aggregators for Tuple Sketches
   
   <!--
   In each section, please describe design decisions made, including:
    - Choice of algorithms
    - Behavioral aspects. What configuration values are acceptable? How are corner cases and error conditions handled, such as when there are insufficient resources?
    - Class organization and design (how the logic is split between classes, inheritance, composition, design patterns)
    - Method organization and design (how the logic is split between methods, parameters and return types)
    - Naming (class, method, API, configuration, HTTP endpoint, names of emitted metrics)
   -->
   
   
   <!-- It's good to describe an alternative design (or mention an alternative name) for every design (or naming) decision point and compare the alternatives with the designs that you've implemented (or the names you've chosen) to highlight the advantages of the chosen designs and names. -->
   
   <!-- If there was a discussion of the design of the feature implemented in this PR elsewhere (e. g. a "Proposal" issue, any other issue, or a thread in the development mailing list), link to that discussion from this PR description and explain what have changed in your final design compared to your original proposal or the consensus version in the end of the discussion. If something hasn't changed since the original discussion, you can omit a detailed discussion of those aspects of the design here, perhaps apart from brief mentioning for the sake of readability of this PR description. -->
   
   <!-- Some of the aspects mentioned above may be omitted for simple and small changes. -->
   
   #### Release note<br/>   
   
   #### New: You can now do the following operations with TupleSketches in Post Aggregation Step
   - Get the Sketch Output as Base64 String
   - Provide a constant Tuple Sketch in post-aggregation step that can be used in Set Operations
   - Get the Estimated Value(Sum) of Summary/Metrics Objects associated with Tuple Sketch
   
   
   
   
   
   <hr>
   
   ##### Key changed/added classes in this PR
   ##### Added
    * `ArrayOfDoublesSketchToEncodedStringPostAggregator.java`
    * `ArrayOfDoublesSketchToMetricsEstimatePostAggregator.java`
    * `ArrayOfDoublesSketchConstantPostAggregator.java`
    * `ArrayOfDoublesSketchToMetricsEstimatePostAggregatorTest.java`
    * `ArrayOfDoublesSketchToEncodedStringPostAggregatorTest.java`
   ##### Modified
   * `AggregatorUtil.java`
   * `ArrayOfDoublesSketchModule.java`
   
   <hr>
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist below are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   This PR has:
   
   - [X] been self-reviewed.
   - [ ] added documentation for new or modified features or behaviors.
   - [X] a release note entry in the PR description.
   - [X] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [X ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [X] been tested in a test Druid cluster.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1449923126

   
   @abhishekagarwal87 :  I have added the tests as suggested for native query testing for newly added post-aggs. 
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] vtlim commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "vtlim (via GitHub)" <gi...@apache.org>.
vtlim commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1113471425


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.

Review Comment:
   ```suggestion
   This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
   ```
   Not sure if what is meant by "adds". Would it be accurate to say "generates" or "outputs"?



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 

Review Comment:
   ```suggestion
   ### Base64 output of ArrayOfDoublesSketch 
   ```



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE

Review Comment:
   ```suggestion
     "type": "arrayOfDoublesSketchConstant",
     "name": DESTINATION_COLUMN_NAME,
     "value": CONSTANT_SKETCH_VALUE
   ```



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 

Review Comment:
   ```suggestion
   This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
   ```



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated Metrics values for each column of ArrayOfDoublesSketch
+
+You can use this post aggregator to return a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key.

Review Comment:
   What does the `(sum)` mean here?



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>

Review Comment:
   ```suggestion
     "type": "arrayOfDoublesSketchToBase64String",
     "name": DESTINATION_COLUMN_NAME,
     "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
   ```



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated Metrics values for each column of ArrayOfDoublesSketch
+
+You can use this post aggregator to return a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key.
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToMetricsEstimate",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>

Review Comment:
   ```suggestion
     "type": "arrayOfDoublesSketchToMetricsEstimate",
     "name": DESTINATION_COLUMN_NAME,
     "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
   ```



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated Metrics values for each column of ArrayOfDoublesSketch
+
+You can use this post aggregator to return a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key.

Review Comment:
   ```suggestion
   This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.
   ```



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated Metrics values for each column of ArrayOfDoublesSketch

Review Comment:
   ```suggestion
   ### Estimated metrics values for each column of ArrayOfDoublesSketch
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1117043966


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEncodedStringPostAggregator.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Base64;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a base64 encoded string of a given {@link ArrayOfDoublesSketch}.
+ * This is a string returned by  encoding the output of toByteArray() using Base64 method of the sketch.
+ * This can be useful for debugging and using the sketch output in other operations.
+ */
+public class ArrayOfDoublesSketchToEncodedStringPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator

Review Comment:
   done
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] vtlim commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "vtlim (via GitHub)" <gi...@apache.org>.
vtlim commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1122396533


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated sum for each metric value from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   Thanks for clarifying @anshu-makkar. Suggested wording, if this is accurate:
   
   `````suggestion
   For the key-value pairs in the given ArrayOfDoublesSketch, this post aggregator estimates the sum for each set of values across the keys. For example, the post aggregator returns `{3.0, 8.0}` for the following key-value pairs:
   
   ```
   Key_1, {1.0, 3.0}
   Key_2, {2.0, 5.0}
   ```
   
   The post aggregator returns _N_ double values, where _N_ is the number of values associated with each key.
   `````



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1111479695


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+
+/**

Review Comment:
   done
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1111471404


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ *
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeSafe(value);
+  }
+
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
+  {
+    return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+  }
+
+  private ArrayOfDoublesSketch deserializeFromByteArray(final byte[] data)
+  {
+    return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(data));
+  }
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedStringSafe(final String str)
+  {
+    return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+  }
+
+  private ArrayOfDoublesSketch deserializeFromByteArraySafe(final byte[] data)
+  {
+    return ArrayOfDoublesSketches.wrapSketch(SafeWritableMemory.wrap(data));
+  }
+
+
+  private ArrayOfDoublesSketch deserialize(final Object serializedSketch)
+  {
+    if (serializedSketch instanceof String) {
+      return deserializeFromBase64EncodedString((String) serializedSketch);
+    } else if (serializedSketch instanceof byte[]) {
+      return deserializeFromByteArray((byte[]) serializedSketch);
+    } else if (serializedSketch instanceof ArrayOfDoublesSketch) {
+      return (ArrayOfDoublesSketch) serializedSketch;
+    }
+    throw new ISE("Object is not of a type that can deserialize to sketch: %s", serializedSketch.getClass());
+  }
+
+  private ArrayOfDoublesSketch deserializeSafe(final Object serializedSketch)
+  {
+    if (serializedSketch instanceof String) {

Review Comment:
   Yes, the user input is a string, there should not be any case of not having a string value. I will remove the dead code. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1110565322


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+
+/**

Review Comment:
   can you add some java-docs? 



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ *
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeSafe(value);
+  }
+
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
+  {
+    return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+  }
+
+  private ArrayOfDoublesSketch deserializeFromByteArray(final byte[] data)
+  {
+    return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(data));
+  }
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedStringSafe(final String str)
+  {
+    return deserializeFromByteArraySafe(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
+  }
+
+  private ArrayOfDoublesSketch deserializeFromByteArraySafe(final byte[] data)
+  {
+    return ArrayOfDoublesSketches.wrapSketch(SafeWritableMemory.wrap(data));
+  }
+
+
+  private ArrayOfDoublesSketch deserialize(final Object serializedSketch)
+  {
+    if (serializedSketch instanceof String) {
+      return deserializeFromBase64EncodedString((String) serializedSketch);
+    } else if (serializedSketch instanceof byte[]) {
+      return deserializeFromByteArray((byte[]) serializedSketch);
+    } else if (serializedSketch instanceof ArrayOfDoublesSketch) {
+      return (ArrayOfDoublesSketch) serializedSketch;
+    }
+    throw new ISE("Object is not of a type that can deserialize to sketch: %s", serializedSketch.getClass());
+  }
+
+  private ArrayOfDoublesSketch deserializeSafe(final Object serializedSketch)
+  {
+    if (serializedSketch instanceof String) {

Review Comment:
   when is this serializedSketch not a String? 



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ *
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeSafe(value);
+  }
+
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
+  {
+    return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));

Review Comment:
   you can use StringUtils.toUtf8(str) method instead. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1119680213


##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregatorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class ArrayOfDoublesSketchConstantPostAggregatorTest

Review Comment:
   This is WIP, will update once it is done. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1119591483


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketchIterator;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a list of estimate values of metrics column from a given {@link ArrayOfDoublesSketch}.
+ * The result will be N double values, where N is the number of double values kept in the sketch per key.
+ */
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] rohangarg commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "rohangarg (via GitHub)" <gi...@apache.org>.
rohangarg commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116742868


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedStringSafe(final String str)

Review Comment:
   can this be re-used from `ArrayOfDoublesSketchOperations` ?



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEncodedStringPostAggregator.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Base64;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a base64 encoded string of a given {@link ArrayOfDoublesSketch}.
+ * This is a string returned by  encoding the output of toByteArray() using Base64 method of the sketch.
+ * This can be useful for debugging and using the sketch output in other operations.
+ */
+public class ArrayOfDoublesSketchToEncodedStringPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
+{
+
+  @JsonCreator
+  public ArrayOfDoublesSketchToEncodedStringPostAggregator(
+      @JsonProperty("name") final String name,
+      @JsonProperty("field") final PostAggregator field
+  )
+  {
+    super(name, field);
+  }
+
+  @Override
+  public String compute(final Map<String, Object> combinedAggregators)
+  {
+    final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
+    return Base64.getEncoder().encodeToString(sketch.toByteArray());

Review Comment:
   there's already a utility method in `StringUtils` class to do base64 encoding of a string - that can be re-used here.



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketchIterator;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a list of estimate values of metrics column from a given {@link ArrayOfDoublesSketch}.
+ * The result will be N double values, where N is the number of double values kept in the sketch per key.
+ */
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
+{
+
+  @JsonCreator
+  public ArrayOfDoublesSketchToMetricsEstimatePostAggregator(
+      @JsonProperty("name") final String name,
+      @JsonProperty("field") final PostAggregator field
+  )
+  {
+    super(name, field);
+  }
+
+  @Override
+  public double[] compute(final Map<String, Object> combinedAggregators)
+  {
+    final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
+    final SummaryStatistics[] stats = new SummaryStatistics[sketch.getNumValues()];
+    Arrays.setAll(stats, i -> new SummaryStatistics());
+    final ArrayOfDoublesSketchIterator it = sketch.iterator();
+    while (it.next()) {
+      final double[] values = it.getValues();
+      for (int i = 0; i < values.length; i++) {
+        stats[i].addValue(values[i]);
+      }
+    }
+    final double[] estimates = new double[sketch.getNumValues()];
+    Arrays.setAll(estimates, i -> ((stats[i].getMean()) * (sketch.getRetainedEntries())) / (sketch.getTheta()));

Review Comment:
   could this be just `stats[i].getSum() / sketch.getTheta()`? 



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEncodedStringPostAggregator.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Base64;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a base64 encoded string of a given {@link ArrayOfDoublesSketch}.
+ * This is a string returned by  encoding the output of toByteArray() using Base64 method of the sketch.
+ * This can be useful for debugging and using the sketch output in other operations.
+ */
+public class ArrayOfDoublesSketchToEncodedStringPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator

Review Comment:
   name could be `ArrayOfDoublesSketchToBase64StringPostAggregator` for clarity



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1117050044


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   This is an example use case, where we want to obtain time series data of Cricket Users and `sum` total of the clicks they have made in the app screen pixel.
   
   
   ```
   {
     "queryType": "groupBy",
     "dataSource": "ds1",
     "granularity": "hour",
     "dimensions": [],
     "filter": {
       "type": "and",
       "fields": [
         { "type": "selector", "dimension": "country", "value": "US" },
         { "type": "selector", "dimension": "type", "value": 1 }
   
         , {"type": "in","dimension": "region",  "values": ["FLORIDA","CALIFORNIA"]}  ]
     },
     "aggregations": [
       { "type" : "arrayOfDoublesSketch", "name" : "filtered_user_sketch",  "fieldName": "filtered_user_sketch", "nominalEntries": 4096},
     ],
       "postAggregations": [
   
       {
         "type": "arrayOfDoublesSketchToSumMetricsEstimate",
         "name": "cricket_users_count",
         "field":
         {
           "type": "arrayOfDoublesSketchSetOp",
           "name": "filtered_cricket_users_tuple_sketch",
           "operation": "INTERSECT",
           "fields": [
           {
   
                           "type"  : "arrayOfDoublesSketchConstant",
                           "name"  : "cricket_users_tuple_sketch",
                           "value" : "AQEJAwgBzJP/////////fwIAAAAAAAAAzT6NGdX0aWUOJvS5EIhpLwAAAAAAAAAAAAAAAAAAAAA="
   
   
             },
             {
                           "type"  : "fieldAccess",
                           "fieldName"  : "filtered_user_sketch"
             }
           ],
           "nominalEntries":4096
         }
       }
     ],
     "intervals": ["2022-11-01/2023-02-01"]
   }
   
   
   
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1115255614


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeSafe(value);

Review Comment:
   you could just call `deserializeFromBase64EncodedStringSafe` and remove `deserializeSafe`



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   is `arrayOfDoublesSketchToMetricsSumEstimate` a better name for this? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] rohangarg commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "rohangarg (via GitHub)" <gi...@apache.org>.
rohangarg commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1118596955


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java:
##########
@@ -48,6 +48,14 @@ public class ArrayOfDoublesSketchModule implements DruidModule
   public static final ColumnType BUILD_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG);
   public static final ColumnType MERGE_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG);
 
+  public static final String ARRAY_OF_DOUBLES_SKETCH_CONSTANT = "arrayOfDoublesSketchConstant";
+
+  public static final String ARRAY_OF_DOUBLES_SKETCH_TO_BASE64_STRING = "arrayOfDoublesSketchToBase64String";
+
+  public static final String ARRAY_OF_DOUBLES_SKETCH_METRICS_ESTIMATE = "arrayOfDoublesSketchToMetricsSumEstimate";

Review Comment:
   the name of the var needs update



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = ArrayOfDoublesSketchOperations.deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return Comparators.alwaysEqual();
+  }
+
+  @Override
+  public Object compute(Map<String, Object> combinedAggregators)
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public ArrayOfDoublesSketchConstantPostAggregator decorate(Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty("value")
+  public ArrayOfDoublesSketch getSketchValue()
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ArrayOfDoublesSketchConstantPostAggregator{name='" + this.getName() + "', value='" + value + "'}";
+  }
+  
+  private String getRawSketchValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ArrayOfDoublesSketchConstantPostAggregator that = (ArrayOfDoublesSketchConstantPostAggregator) o;
+    if (!this.getSketchValue().equals(that.getSketchValue())) {
+      return false;
+    }
+    if (this.getName() != null ? !this.getName().equals(that.getName()) : that.getName() != null) {
+      return false;
+    }
+
+    if (this.getRawSketchValue() != null ? !this.getRawSketchValue().equals(that.getRawSketchValue()) : that.getRawSketchValue() != null) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode()
+  {
+    int result = getName() != null ? getName().hashCode() : 0;

Review Comment:
   can this be converted to `Objects.hash(name, value)`?



##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregatorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class ArrayOfDoublesSketchConstantPostAggregatorTest

Review Comment:
   Please consider adding the new post aggs as a part of the `ArrayOfDoublesSketchAggregationTest` so that they be tested as a part of a complete native query as well.



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   Thanks a lot for explaining the use-case @anshu-makkar - it was very helpful!
   Ideally, I think it would have been better if we could just plug a deserialization function and make `ExpressionPostAggregator` work with it as @abhishekagarwal87 had mentioned.
   But I think it is ok to have a dedicated post agg for this now. Once there is a consolidate method of combining serdes with post aggregations, we can consider deprecating some of these post aggregations.



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = ArrayOfDoublesSketchOperations.deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return Comparators.alwaysEqual();
+  }
+
+  @Override
+  public Object compute(Map<String, Object> combinedAggregators)
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public ArrayOfDoublesSketchConstantPostAggregator decorate(Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty("value")
+  public ArrayOfDoublesSketch getSketchValue()

Review Comment:
   sorry, I didn't understand the need for making the json property method on sketch object and not on the string value object. Can you please explain this? 



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketchIterator;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a list of estimate values of metrics column from a given {@link ArrayOfDoublesSketch}.
+ * The result will be N double values, where N is the number of double values kept in the sketch per key.
+ */
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator

Review Comment:
   Some of the occurrences of `MetricsEstimate` have not been updated to `MetricsSumEstimate`. Can you please do that in the changes?



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   could `This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch.` be more like `This post aggregator returns a list of estimated sum for each metric value from a given ArrayOfDoublesSketch.` ?



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = ArrayOfDoublesSketchOperations.deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return Comparators.alwaysEqual();
+  }
+
+  @Override
+  public Object compute(Map<String, Object> combinedAggregators)
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public ArrayOfDoublesSketchConstantPostAggregator decorate(Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty("value")
+  public ArrayOfDoublesSketch getSketchValue()
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ArrayOfDoublesSketchConstantPostAggregator{name='" + this.getName() + "', value='" + value + "'}";
+  }
+  
+  private String getRawSketchValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ArrayOfDoublesSketchConstantPostAggregator that = (ArrayOfDoublesSketchConstantPostAggregator) o;

Review Comment:
   can the below code be converted into `Objects.equals(this.name, o.name) && Objects.equals(this.value, o.value)` ? 



##########
processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java:
##########
@@ -100,7 +100,9 @@
   public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_QUANTILES_SKETCH_CACHE_TYPE_ID = 0x28;
   public static final byte ARRAY_OF_DOUBLES_SKETCH_T_TEST_CACHE_TYPE_ID = 0x29;
   public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID = 0x2A;
-
+  public static final byte ARRAY_OF_DOUBLES_SKETCH_CONSTANT_SKETCH_CACHE_TYPE_ID = 0x4D;

Review Comment:
   I think it would be better to append these at the last in a separate section in increasing hex value order. This makes it hard to know the next available id and can very possibly lead to collisions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1447722787

   @abhishekagarwal87 @rohangarg  Thanks for taking time to review this. I will work on remaining review comment to add a test case to `ArrayOfDoublesSketchAggregationTest`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1434736161

   @techdocsmith Hello, if you could help me tag a reviewer for these new features. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1122588047


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = ArrayOfDoublesSketchOperations.deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return Comparators.alwaysEqual();
+  }
+
+  @Override
+  public Object compute(Map<String, Object> combinedAggregators)
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public ArrayOfDoublesSketchConstantPostAggregator decorate(Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty("value")
+  public ArrayOfDoublesSketch getSketchValue()
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ArrayOfDoublesSketchConstantPostAggregator{name='" + this.getName() + "', value='" + value + "'}";
+  }
+  
+  private String getRawSketchValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ArrayOfDoublesSketchConstantPostAggregator that = (ArrayOfDoublesSketchConstantPostAggregator) o;
+    if (!this.getSketchValue().equals(that.getSketchValue())) {
+      return false;
+    }
+    if (this.getName() != null ? !this.getName().equals(that.getName()) : that.getName() != null) {
+      return false;
+    }
+
+    if (this.getRawSketchValue() != null ? !this.getRawSketchValue().equals(that.getRawSketchValue()) : that.getRawSketchValue() != null) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode()
+  {
+    int result = getName() != null ? getName().hashCode() : 0;

Review Comment:
   done
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1111479573


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ *
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeSafe(value);
+  }
+
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
+  {
+    return deserializeFromByteArray(StringUtils.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));

Review Comment:
   done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1115303426


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   You mean the `type` field in the post-aggregator?



##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   @abhishekagarwal87 : Are we discussing about the `type` field



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1113859215


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+You can use the this post aggregator to add a Base64-encoded constant ArrayOfDoublesSketch value for use in other post-aggregators.
+```json
+{
+  "type"  : "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value"  : CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 Output of ArrayOfDoublesSketch 
+
+You can use the this post aggregator to output an ArrayOfDoublesSketch as Base64-encoded string constant tuple sketch value for use in other post-aggregators. 
+
+```json
+{
+  "type"  : "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field"  : <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated Metrics values for each column of ArrayOfDoublesSketch
+
+You can use this post aggregator to return a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result will be N double values, where N is the number of double values kept in the sketch per key.

Review Comment:
   For each key in a `sketch`, there is an array of doubles, while calculating the estimate, we are doing sum of the values. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1441233971

   @abhishekagarwal87 : Request to you review if there are more changes required. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1435472101

   @anshu-makkar - Thank you for your contribution. can you please address build failures? You also need to update docs - https://github.com/apache/druid/blob/master/docs/development/extensions-core/datasketches-tuple.md#post-aggregators


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1115303444


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   ```suggestion
   This post aggregator returns a list of estimated values(sum of distincts) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116857258


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   @rohangarg : Requirement here is to provide an external sketch to the Native Query, this external sketch is non-time series in nature, so it is not present in any of the dimensions/fields. External Sketch is added at post-aggregation step so that we could do set operations on the sketches( time series sketch output in aggregation step vs non time series data) 
   
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116861980


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   @rohangarg : Let me know if that clarifies or more clarity is required. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] vtlim commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "vtlim (via GitHub)" <gi...@apache.org>.
vtlim commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1120990146


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated sum for each metric value from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   ```suggestion
   For each key-value pair in the given ArrayOfDoublesSketch, this post aggregator estimates the sum of the values associated with the key. The post aggregator returns _N_ double values, where _N_ is the number of double values associated with each key.
   ```
   
   Based on the example in https://github.com/apache/druid/pull/13819#discussion_r1115305144, if the sum is done for each key (row-wise), then I think we should say `_N_ is the number of keys in the ArrayOfDoublesSketch` but if the sum is done across different keys (column-wise), then the description for _N_ is fine as is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] vtlim commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "vtlim (via GitHub)" <gi...@apache.org>.
vtlim commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1120990962


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   Sorry, just saw this. The example isn't clear to me; is the sum being added row-wise or column-wise?
   
   For this example, 
   ```
   Key_1, {1.0, 3.0}
   Key_2, {2.0, 5.0}
   ```
   
   Would the result be `{3.0, 8.0}` or `{4.0, 7.0}`?
   
   Commented a suggestion in https://github.com/apache/druid/pull/13819/files?file-filters%5B%5D=.md&show-viewed-files=true#r1120990146. It would be great to include this example in the docs as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1435473035

   > @anshu-makkar - Thank you for your contribution. can you please address build failures? You also need to update docs - https://github.com/apache/druid/blob/master/docs/development/extensions-core/datasketches-tuple.md#post-aggregators
   
   @abhishekagarwal87 : Yes I will work on adding the documentation, and address the build failures. 
   
   Any other comments if you have I can address those. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1119658493


##########
processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java:
##########
@@ -100,7 +100,9 @@
   public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_QUANTILES_SKETCH_CACHE_TYPE_ID = 0x28;
   public static final byte ARRAY_OF_DOUBLES_SKETCH_T_TEST_CACHE_TYPE_ID = 0x29;
   public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID = 0x2A;
-
+  public static final byte ARRAY_OF_DOUBLES_SKETCH_CONSTANT_SKETCH_CACHE_TYPE_ID = 0x4D;

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116853470


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketchIterator;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a list of estimate values of metrics column from a given {@link ArrayOfDoublesSketch}.
+ * The result will be N double values, where N is the number of double values kept in the sketch per key.
+ */
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
+{
+
+  @JsonCreator
+  public ArrayOfDoublesSketchToMetricsEstimatePostAggregator(
+      @JsonProperty("name") final String name,
+      @JsonProperty("field") final PostAggregator field
+  )
+  {
+    super(name, field);
+  }
+
+  @Override
+  public double[] compute(final Map<String, Object> combinedAggregators)
+  {
+    final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
+    final SummaryStatistics[] stats = new SummaryStatistics[sketch.getNumValues()];
+    Arrays.setAll(stats, i -> new SummaryStatistics());
+    final ArrayOfDoublesSketchIterator it = sketch.iterator();
+    while (it.next()) {
+      final double[] values = it.getValues();
+      for (int i = 0; i < values.length; i++) {
+        stats[i].addValue(values[i]);
+      }
+    }
+    final double[] estimates = new double[sketch.getNumValues()];
+    Arrays.setAll(estimates, i -> ((stats[i].getMean()) * (sketch.getRetainedEntries())) / (sketch.getTheta()));

Review Comment:
   Yes it could optimize it a bit. Thanks I will modify it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1117000510


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   Basically assume we have a time series data as an output from aggregator output.
   Filter => country ='US' and region In(Florida, California). Based on this we have sketches created while ingesting data. With userId being the key of the sketch and metrics in Double sketch is the frequency of the user appearing in the time series Data.
   
   External sketch = Tuplesketch of all the users who watches Cricket.- This is non time series data. 
   
   Now we take want to intersect the external sketch with aggregator output in post-aggregator step.
   
   @abhishekagarwal87 @rohangarg 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1115303426


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   You mean the `type` field in the post-aggregator?



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeSafe(value);

Review Comment:
   ok.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1115305144


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   I think this needs an overall rephrase, basically with each distinct value, there is a `array` of `doubles`, with this post-aggregator we want to provide a double array with sum. 
   
   For E.g.
   
   `Key_1, {1.0, 2.0}`
   `Key_2, {2.0, 5.0}`
   
   Output from Post-aggregator is `{3.0, 7.0}`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1452931990

   Merged. Thank you for your contribution @anshu-makkar 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 merged pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 merged PR #13819:
URL: https://github.com/apache/druid/pull/13819


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1452911292

   Request to merge, since we have another development activity on top of this to add `SQL` support to `Tuple Sketches`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1119591269


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java:
##########
@@ -48,6 +48,14 @@ public class ArrayOfDoublesSketchModule implements DruidModule
   public static final ColumnType BUILD_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG);
   public static final ColumnType MERGE_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG);
 
+  public static final String ARRAY_OF_DOUBLES_SKETCH_CONSTANT = "arrayOfDoublesSketchConstant";
+
+  public static final String ARRAY_OF_DOUBLES_SKETCH_TO_BASE64_STRING = "arrayOfDoublesSketchToBase64String";
+
+  public static final String ARRAY_OF_DOUBLES_SKETCH_METRICS_ESTIMATE = "arrayOfDoublesSketchToMetricsSumEstimate";

Review Comment:
   done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] rohangarg commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "rohangarg (via GitHub)" <gi...@apache.org>.
rohangarg commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116853211


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   I didn't quite understand the need for this `PostAggregator` - can you please provide any reference or example explaining this a bit more in detail?
   Also, given that this post agg just converts a user given string to a sketch, is it necessary to make it a post aggregator and not a simple function?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116994146


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   The external sketch is generated by a backend service external to druid. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116997831


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   This explanation works. Though it does need to be called out that result is an estimated sum and not accurate sum. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1438682066

   @anshu-makkar - are you going to update the code later? You have resolved the comments. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] github-code-scanning[bot] commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "github-code-scanning[bot] (via GitHub)" <gi...@apache.org>.
github-code-scanning[bot] commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1109543348


##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEncodedStringPostAggregatorTest.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class ArrayOfDoublesSketchToEncodedStringPostAggregatorTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ExpectedException.none](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4270)



##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregatorTest.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregatorTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();

Review Comment:
   ## Deprecated method or constructor invocation
   
   Invoking [ExpectedException.none](1) should be avoided because it has been deprecated.
   
   [Show more details](https://github.com/apache/druid/security/code-scanning/4271)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] techdocsmith commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "techdocsmith (via GitHub)" <gi...@apache.org>.
techdocsmith commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1434925397

   Thanks for the contribution @anshu-makkar . I'm not sure who should do this review--I focus mostly on markdown :D . @abhishekagarwal87 do you have a recommendation?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116988274


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   An `ExpressionPostAggregator` could have been used as well where the expression is a function to convert string to sketch. But then, we have similar post aggregators already. E.g. ArrayOfDoublesSketchToStringPostAggregator



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116991958


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator

Review Comment:
   But then, those post aggregators make sense since they operate on a sketch which is typically the output of an aggregation. This external sketch is coming out of an aggregate function? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1119591140


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = ArrayOfDoublesSketchOperations.deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return Comparators.alwaysEqual();
+  }
+
+  @Override
+  public Object compute(Map<String, Object> combinedAggregators)
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public ArrayOfDoublesSketchConstantPostAggregator decorate(Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty("value")
+  public ArrayOfDoublesSketch getSketchValue()

Review Comment:
   This was a typo. removed. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116869683


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketches;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.data.SafeWritableMemory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  private ArrayOfDoublesSketch deserializeFromBase64EncodedStringSafe(final String str)

Review Comment:
   @rohangarg : Yes makes sense



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1121379084


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated sum for each metric value from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   @vtlim : The sum is performed column wise. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1122587933


##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregatorTest.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class ArrayOfDoublesSketchConstantPostAggregatorTest

Review Comment:
   completed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1449753441

   thanks @anshu-makkar. Please let us know once you think it's ready for review. I think we are almost there. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "abhishekagarwal87 (via GitHub)" <gi...@apache.org>.
abhishekagarwal87 commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1122643836


##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregationTest.java:
##########
@@ -1244,4 +1244,97 @@ public void buildingSketchesAtQueryTimeWithNullsTest() throws Exception
     Assert.assertEquals(0.0, ds3.getMinValue(), 0);
     Assert.assertEquals(3.0, ds3.getMaxValue(), 0);
   }
+
+
+  //Test ConstantTupleSketchPost-Agg and Base64 Encoding
+
+  @Test
+  public void testConstantAndBase64WithEstimateSumPostAgg() throws Exception
+  {
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
+        new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_sketch_data_two_values.tsv")
+            .getFile()),
+        String.join(
+            "\n",
+            "{",
+            "  \"type\": \"string\",",
+            "  \"parseSpec\": {",
+            "    \"format\": \"tsv\",",
+            "    \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
+            "    \"dimensionsSpec\": {",
+            "      \"dimensions\": [\"product\"],",
+            "      \"dimensionExclusions\": [],",
+            "      \"spatialDimensions\": []",
+            "    },",
+            "    \"columns\": [\"timestamp\", \"product\", \"sketch\"]",
+            "  }",
+            "}"
+        ),
+        String.join(
+            "\n",
+            "[",
+            "  {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
+            "]"
+        ),
+        0, // minTimestamp
+        Granularities.NONE,
+        10, // maxRowCount
+        String.join(
+            "\n",
+            "{",
+            "  \"queryType\": \"groupBy\",",
+            "  \"dataSource\": \"test_datasource\",",
+            "  \"granularity\": \"ALL\",",
+            "  \"dimensions\": [],",
+            "  \"aggregations\": [",
+            "    {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
+            "  ],",
+            "  \"postAggregations\": [",
+            "    {\"type\": \"arrayOfDoublesSketchToMetricsEstimate\", \"name\": \"estimateSum\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
+            "    {\"type\": \"arrayOfDoublesSketchToMetricsEstimate\", \"name\": \"intersection\", \"field\": {",

Review Comment:
   ```suggestion
               "    {\"type\": \"arrayOfDoublesSketchToMetricsSumEstimate\", \"name\": \"intersection\", \"field\": {",
   ```



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java:
##########
@@ -48,6 +48,13 @@ public class ArrayOfDoublesSketchModule implements DruidModule
   public static final ColumnType BUILD_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG);
   public static final ColumnType MERGE_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG);
 
+  public static final String ARRAY_OF_DOUBLES_SKETCH_CONSTANT = "arrayOfDoublesSketchConstant";
+
+  public static final String ARRAY_OF_DOUBLES_SKETCH_TO_BASE64_STRING = "arrayOfDoublesSketchToBase64String";
+
+  public static final String ARRAY_OF_DOUBLES_SKETCH_METRICS_SUM_ESTIMATE = "arrayOfDoublesSketchToMetricsEstimate";

Review Comment:
   ```suggestion
     public static final String ARRAY_OF_DOUBLES_SKETCH_METRICS_SUM_ESTIMATE = "arrayOfDoublesSketchToMetricsSumEstimate";
   ```



##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregationTest.java:
##########
@@ -1244,4 +1244,97 @@ public void buildingSketchesAtQueryTimeWithNullsTest() throws Exception
     Assert.assertEquals(0.0, ds3.getMinValue(), 0);
     Assert.assertEquals(3.0, ds3.getMaxValue(), 0);
   }
+
+
+  //Test ConstantTupleSketchPost-Agg and Base64 Encoding
+
+  @Test
+  public void testConstantAndBase64WithEstimateSumPostAgg() throws Exception
+  {
+    Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
+        new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_sketch_data_two_values.tsv")
+            .getFile()),
+        String.join(
+            "\n",
+            "{",
+            "  \"type\": \"string\",",
+            "  \"parseSpec\": {",
+            "    \"format\": \"tsv\",",
+            "    \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
+            "    \"dimensionsSpec\": {",
+            "      \"dimensions\": [\"product\"],",
+            "      \"dimensionExclusions\": [],",
+            "      \"spatialDimensions\": []",
+            "    },",
+            "    \"columns\": [\"timestamp\", \"product\", \"sketch\"]",
+            "  }",
+            "}"
+        ),
+        String.join(
+            "\n",
+            "[",
+            "  {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
+            "]"
+        ),
+        0, // minTimestamp
+        Granularities.NONE,
+        10, // maxRowCount
+        String.join(
+            "\n",
+            "{",
+            "  \"queryType\": \"groupBy\",",
+            "  \"dataSource\": \"test_datasource\",",
+            "  \"granularity\": \"ALL\",",
+            "  \"dimensions\": [],",
+            "  \"aggregations\": [",
+            "    {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
+            "  ],",
+            "  \"postAggregations\": [",
+            "    {\"type\": \"arrayOfDoublesSketchToMetricsEstimate\", \"name\": \"estimateSum\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",

Review Comment:
   ```suggestion
               "    {\"type\": \"arrayOfDoublesSketchToMetricsSumEstimate\", \"name\": \"estimateSum\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1119656998


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchConstantPostAggregator.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This post-aggregator converts a given Base64 encoded string to an ArrayOfDoublesSketch.
+ * The input column contains name of post-aggregator output and base64 encoded input string.
+ * The output is a deserialized {@link ArrayOfDoublesSketch} .
+ */
+public class ArrayOfDoublesSketchConstantPostAggregator extends ArrayOfDoublesSketchPostAggregator
+{
+
+  private final String value;
+  private final ArrayOfDoublesSketch sketchValue;
+
+  @JsonCreator
+  public ArrayOfDoublesSketchConstantPostAggregator(@JsonProperty("name") String name, @JsonProperty("value") String value)
+  {
+    super(name);
+    Preconditions.checkArgument(value != null && !value.isEmpty(),
+        "Constant value cannot be null or empty, expecting base64 encoded sketch string");
+    this.value = value;
+    this.sketchValue = ArrayOfDoublesSketchOperations.deserializeFromBase64EncodedStringSafe(value);
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return Collections.emptySet();
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return Comparators.alwaysEqual();
+  }
+
+  @Override
+  public Object compute(Map<String, Object> combinedAggregators)
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public ArrayOfDoublesSketchConstantPostAggregator decorate(Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty("value")
+  public ArrayOfDoublesSketch getSketchValue()
+  {
+    return sketchValue;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "ArrayOfDoublesSketchConstantPostAggregator{name='" + this.getName() + "', value='" + value + "'}";
+  }
+  
+  private String getRawSketchValue()
+  {
+    return value;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ArrayOfDoublesSketchConstantPostAggregator that = (ArrayOfDoublesSketchConstantPostAggregator) o;

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1121543787


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   > thanks @anshu-makkar. Please let us know once you think it's ready for review. I think we are almost there.
   
   @abhishekagarwal87 : I have added tests for native query as suggested. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1121380792


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   Result would be `{3.0, 8.0}`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on PR #13819:
URL: https://github.com/apache/druid/pull/13819#issuecomment-1438684769

   > @anshu-makkar - are you going to update the code later? You have resolved the comments.
   
   Hi @abhishekagarwal87 I will be pushing the code shortly. I was working on unit tests


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1109667916


##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregatorTest.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregatorTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();

Review Comment:
   Fixed



##########
extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEncodedStringPostAggregatorTest.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.aggregation.post.ConstantPostAggregator;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class ArrayOfDoublesSketchToEncodedStringPostAggregatorTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();

Review Comment:
   Fixed
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1115938543


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   @techdocsmith @vtlim Can you help rephrase this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116923409


##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEncodedStringPostAggregator.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Base64;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a base64 encoded string of a given {@link ArrayOfDoublesSketch}.
+ * This is a string returned by  encoding the output of toByteArray() using Base64 method of the sketch.
+ * This can be useful for debugging and using the sketch output in other operations.
+ */
+public class ArrayOfDoublesSketchToEncodedStringPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
+{
+
+  @JsonCreator
+  public ArrayOfDoublesSketchToEncodedStringPostAggregator(
+      @JsonProperty("name") final String name,
+      @JsonProperty("field") final PostAggregator field
+  )
+  {
+    super(name, field);
+  }
+
+  @Override
+  public String compute(final Map<String, Object> combinedAggregators)
+  {
+    final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
+    return Base64.getEncoder().encodeToString(sketch.toByteArray());

Review Comment:
   done
   



##########
extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMetricsEstimatePostAggregator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.tuple;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch;
+import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketchIterator;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnInspector;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Returns a list of estimate values of metrics column from a given {@link ArrayOfDoublesSketch}.
+ * The result will be N double values, where N is the number of double values kept in the sketch per key.
+ */
+public class ArrayOfDoublesSketchToMetricsEstimatePostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
+{
+
+  @JsonCreator
+  public ArrayOfDoublesSketchToMetricsEstimatePostAggregator(
+      @JsonProperty("name") final String name,
+      @JsonProperty("field") final PostAggregator field
+  )
+  {
+    super(name, field);
+  }
+
+  @Override
+  public double[] compute(final Map<String, Object> combinedAggregators)
+  {
+    final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
+    final SummaryStatistics[] stats = new SummaryStatistics[sketch.getNumValues()];
+    Arrays.setAll(stats, i -> new SummaryStatistics());
+    final ArrayOfDoublesSketchIterator it = sketch.iterator();
+    while (it.next()) {
+      final double[] values = it.getValues();
+      for (int i = 0; i < values.length; i++) {
+        stats[i].addValue(values[i]);
+      }
+    }
+    final double[] estimates = new double[sketch.getNumValues()];
+    Arrays.setAll(estimates, i -> ((stats[i].getMean()) * (sketch.getRetainedEntries())) / (sketch.getTheta()));

Review Comment:
   done
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] anshu-makkar commented on a diff in pull request #13819: Add Post Aggregators for Tuple Sketches

Posted by "anshu-makkar (via GitHub)" <gi...@apache.org>.
anshu-makkar commented on code in PR #13819:
URL: https://github.com/apache/druid/pull/13819#discussion_r1116923080


##########
docs/development/extensions-core/datasketches-tuple.md:
##########
@@ -207,3 +207,39 @@ Returns a human-readable summary of a given ArrayOfDoublesSketch. This is a stri
   "field"  : <post aggregator that refers to an ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
 }
 ```
+
+
+### Constant ArrayOfDoublesSketch 
+
+This post aggregator adds a Base64-encoded constant ArrayOfDoublesSketch value that you can use in other post aggregators.
+```json
+{
+  "type": "arrayOfDoublesSketchConstant",
+  "name": DESTINATION_COLUMN_NAME,
+  "value": CONSTANT_SKETCH_VALUE
+}
+```
+
+### Base64 output of ArrayOfDoublesSketch 
+
+This post aggregator outputs an ArrayOfDoublesSketch as a Base64-encoded string storing the constant tuple sketch value that you can use in other post aggregators. 
+
+```json
+{
+  "type": "arrayOfDoublesSketchToBase64String",
+  "name": DESTINATION_COLUMN_NAME,
+  "field": <post aggregator that refers to a ArrayOfDoublesSketch (fieldAccess or another post aggregator)>
+}
+```
+
+### Estimated metrics values for each column of ArrayOfDoublesSketch
+
+This post aggregator returns a list of estimated values(sum) from a given ArrayOfDoublesSketch. The result is _N_ double values, where _N_ is the number of double values kept in the sketch per key.

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org