You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2020/01/15 00:38:16 UTC

[GitHub] [druid] maytasm3 opened a new pull request #9187: Implement ANY aggregator

maytasm3 opened a new pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187
 
 
   <!-- 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. -->
   
   Fixes #XXXX.
   
   <!-- 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: -->
   
   #### Fixed the bug ...
   #### Renamed the class ...
   #### Added a forbidden-apis entry ...
   
   <!--
   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. -->
   
   <hr>
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/licenses.yaml)
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths.
   - [ ] added integration tests.
   - [ ] been tested in a test Druid cluster.
   
   <!-- 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 above are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `MyFoo`
    * `OurBar`
    * `TheirBaz`
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367162064
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
 
 Review comment:
   Not sure if the storeDoubleAsFloat is even needed.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r366709685
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java
 ##########
 @@ -121,6 +121,11 @@
 
   public static final byte MEAN_CACHE_TYPE_ID = 0x41;
 
+  public static final byte LONG_ANY_CACHE_TYPE_ID = 0x42;
+  public static final byte DOUBLE_ANY_CACHE_TYPE_ID = 0x42;
 
 Review comment:
   I think you need a different byte for each type id here.
   
   Also nitpicking here - can you add a comment above the group saying this is for the any 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367020210
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
 
 Review comment:
   nit: javadocs please - I know most of the aggregators don't have docs, but I think explaining how nulls should be handled in this class is valuable

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367208078
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
 
 Review comment:
   Can you also add entries for the new aggregators under `docs/querying/aggregations.md`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367255974
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
+|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 
 Review comment:
   What do you mean default? You need to always pass the value. There is no default value for maxBytesPerString

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367116049
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
+
+  public DoubleAnyAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+    foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null && !valueSelector.isNull()) {
 
 Review comment:
   `valueSelector.isNull()` will never be true since this the factory is `NullableNumericAggregatorFactory`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367263397
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
 ##########
 @@ -1297,6 +1301,46 @@ public void testLatestAggregators() throws Exception
     );
   }
 
+  // This test the on-heap version of the AnyAggregator (Double/Float/Long/String)
+  @Test
+  public void testAnyAggregator() throws Exception
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367042756
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.putDouble(position, NULL_VALUE);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    if (buf.getDouble(position) == NULL_VALUE && !valueSelector.isNull()) {
+      buf.putDouble(position, valueSelector.getDouble());
+    }
+  }
+
+  @Override
+  public Object get(ByteBuffer buf, int position)
+  {
+    return buf.getDouble(position);
+  }
+
+  @Override
+  public float getFloat(ByteBuffer buf, int position)
+  {
+    return (float) buf.getDouble(position);
+  }
+
+  @Override
+  public long getLong(ByteBuffer buf, int position)
+  {
+    return (long) buf.getDouble(position);
+  }
+
+  @Override
+  public double getDouble(ByteBuffer buf, int position)
+  {
+    return buf.getDouble(position);
+  }
+
+  @Override
+  public void close()
+  {
+
 
 Review comment:
   nit: Comment for empty function please

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367255751
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367204738
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,99 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This Aggregator is created by the {@link DoubleAnyAggregatorFactory} which extends from
+ * {@link NullableNumericAggregatorFactory}. If null needs to be handle, then {@link NullableNumericAggregatorFactory}
+ * will wrap this aggregator in {@link NullableNumericAggregator} and can handle all null in that class.
+ * Hence, no null will ever be pass into this aggregator from the valueSelector.
+ */
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final byte BYTE_FLAG_IS_NOT_SET = 0;
+  private static final byte BYTE_FLAG_IS_SET = 1;
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.put(position, BYTE_FLAG_IS_NOT_SET);
+    buf.putDouble(position + Byte.BYTES, NULL_VALUE);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    if (buf.get(position) == BYTE_FLAG_IS_NOT_SET) {
+      buf.putDouble(position + Byte.BYTES, valueSelector.getDouble());
+      buf.put(position, BYTE_FLAG_IS_SET);
 
 Review comment:
   nit: suggest setting the IS_SET byte first before the double value, since that's the order they appear in the buffer

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on issue #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on issue #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#issuecomment-574914486
 
 
   Note: ANY on Double/Float/Long column with the useDefaultValueForNull=true will **not** prefer non-null values over default value for null (i.e. 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367603604
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
 ##########
 @@ -377,6 +377,15 @@ public AuthenticationResult createEscalatedAuthenticationResult()
   );
 
   public static final List<InputRow> ROWS1_WITH_NUMERIC_DIMS = ImmutableList.of(
+      createRow(
 
 Review comment:
   Seems like the VarianceSqlAggregatorTest is using this data too and when the ordering of the rows changed, the variance also changed. I created a new datasource that have the numeric dim first for my test and changed the numfoo datasource back to how it was. The reason I wanted to have numeric null first is because the ANY will select the first row and skip everything after. So if the first row is not null, then there is not really any point in testing (if we want to test the numeric null stuff)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367167409
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final double NULL_VALUE = 0;
 
 Review comment:
   Added byte to indicate whether or not the value has been found. We actually do not get null due to the NullableNumericAggregatorFactory/ NullableNumericAggregator stuff

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367040205
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
 
 Review comment:
   Is this so that missing values show up first? Is that the behavior we want? I don't know the answer...

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367628285
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
 ##########
 @@ -377,6 +377,15 @@ public AuthenticationResult createEscalatedAuthenticationResult()
   );
 
   public static final List<InputRow> ROWS1_WITH_NUMERIC_DIMS = ImmutableList.of(
+      createRow(
 
 Review comment:
   Actually, I think it's fine to just test with the same numfoo datasource (with first row being non-null)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367207795
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
 
 Review comment:
   "this can returns the default" -> "this can return the default", similarly for "then this will returns"

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367257549
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
 
 Review comment:
   Done. Btw I saw filterNullValues for stringLast and stringFirst. Is that still true?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367624288
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
+|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 
 Review comment:
   Currently, the implementation for LATEST, EARLIEST (and ANY since I based it off LATEST, EARLIEST) is that if you use the json stuff, then maxStringBytes is optional and if not present will default to 1024 (as per the docs in docs/querying/aggregations.md). 
   However, this does not work the same if you issue the query through SQL. To use LATEST, EARLIEST (and ANY) in SQL, you must give the maxStringBytes as the second argument. If you do not, then the column actually gets cast into double (super weird).  

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367160539
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
 
 Review comment:
   There's javadoc on AggregatorFactory class. But i think it's like reusing input output fields 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367207961
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
+|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 
 Review comment:
   This should mention that the default maxBytesPerString is 1024

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367036857
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
 
 Review comment:
   I don't understand why both the parameters here are `fieldName` and both the parameters in getCombiningFactory are `name` - I see this pattern used in all the 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367110861
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java
 ##########
 @@ -121,6 +121,11 @@
 
   public static final byte MEAN_CACHE_TYPE_ID = 0x41;
 
+  public static final byte LONG_ANY_CACHE_TYPE_ID = 0x42;
+  public static final byte DOUBLE_ANY_CACHE_TYPE_ID = 0x42;
 
 Review comment:
   yes these need to be different 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367232247
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregator.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.first.StringAggregatorUtils;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+
+public class StringAnyAggregator implements Aggregator
+{
+  private final BaseObjectColumnValueSelector valueSelector;
+  private final int maxStringBytes;
+
+  private String foundValue;
+
+  public StringAnyAggregator(BaseObjectColumnValueSelector valueSelector, int maxStringBytes)
+  {
+    this.valueSelector = valueSelector;
+    this.maxStringBytes = maxStringBytes;
+    this.foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null) {
+      final Object object = valueSelector.getObject();
+      if (object != null) {
+        foundValue = DimensionHandlerUtils.convertObjectToString(object);
+        if (foundValue != null && foundValue.length() > maxStringBytes) {
+          foundValue = foundValue.substring(0, maxStringBytes);
+        }
+      }
+    }
+  }
+
+  @Override
+  public Object get()
+  {
+    return StringAggregatorUtils.chop(foundValue, maxStringBytes);
 
 Review comment:
   nit: It is probably worth pushing `chop` down into `StringUtils` rather than renaming and widening the usage of `StringAggregatorUtils`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367163512
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.putDouble(position, NULL_VALUE);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    if (buf.getDouble(position) == NULL_VALUE && !valueSelector.isNull()) {
+      buf.putDouble(position, valueSelector.getDouble());
+    }
+  }
+
+  @Override
+  public Object get(ByteBuffer buf, int position)
+  {
+    return buf.getDouble(position);
+  }
+
+  @Override
+  public float getFloat(ByteBuffer buf, int position)
+  {
+    return (float) buf.getDouble(position);
+  }
+
+  @Override
+  public long getLong(ByteBuffer buf, int position)
+  {
+    return (long) buf.getDouble(position);
+  }
+
+  @Override
+  public double getDouble(ByteBuffer buf, int position)
+  {
+    return buf.getDouble(position);
+  }
+
+  @Override
+  public void close()
+  {
+
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367231030
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,99 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This Aggregator is created by the {@link DoubleAnyAggregatorFactory} which extends from
+ * {@link NullableNumericAggregatorFactory}. If null needs to be handle, then {@link NullableNumericAggregatorFactory}
+ * will wrap this aggregator in {@link NullableNumericAggregator} and can handle all null in that class.
+ * Hence, no null will ever be pass into this aggregator from the valueSelector.
+ */
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final byte BYTE_FLAG_IS_NOT_SET = 0;
+  private static final byte BYTE_FLAG_IS_SET = 1;
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.put(position, BYTE_FLAG_IS_NOT_SET);
+    buf.putDouble(position + Byte.BYTES, NULL_VALUE);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    if (buf.get(position) == BYTE_FLAG_IS_NOT_SET) {
+      buf.putDouble(position + Byte.BYTES, valueSelector.getDouble());
+      buf.put(position, BYTE_FLAG_IS_SET);
 
 Review comment:
   I agree on put ordering

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367263080
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregator.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.first.StringAggregatorUtils;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+
+public class StringAnyAggregator implements Aggregator
+{
+  private final BaseObjectColumnValueSelector valueSelector;
+  private final int maxStringBytes;
+
+  private String foundValue;
+
+  public StringAnyAggregator(BaseObjectColumnValueSelector valueSelector, int maxStringBytes)
+  {
+    this.valueSelector = valueSelector;
+    this.maxStringBytes = maxStringBytes;
+    this.foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null) {
+      final Object object = valueSelector.getObject();
+      if (object != null) {
+        foundValue = DimensionHandlerUtils.convertObjectToString(object);
+        if (foundValue != null && foundValue.length() > maxStringBytes) {
+          foundValue = foundValue.substring(0, maxStringBytes);
+        }
+      }
+    }
+  }
+
+  @Override
+  public Object get()
+  {
+    return StringAggregatorUtils.chop(foundValue, maxStringBytes);
 
 Review comment:
   sounds good to me. 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367031257
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DoubleAnyAggregatorFactory that = (DoubleAnyAggregatorFactory) o;
+
+    return name.equals(that.name) && fieldName.equals(that.fieldName);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, fieldName);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "DoubleAnyAggregatorFactory{" +
+           "name='" + name + '\'' +
+           ", fieldName='" + fieldName + '\'' +
+           '}';
+  }
 
 Review comment:
   Similar comment to above...
   
   Ugh I can't wait till we can hide all of this away with lombok

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367026592
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
+
+  public DoubleAnyAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+    foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null && !valueSelector.isNull()) {
+      foundValue = valueSelector.getDouble();
+    }
+  }
+
+  @Override
+  public Object get()
+  {
+    return foundValue;
+  }
+
+  @Override
+  public float getFloat()
+  {
+    return foundValue.floatValue();
+  }
+
+  @Override
+  public long getLong()
+  {
+    return foundValue.longValue();
+  }
+
+  @Override
+  public double getDouble()
+  {
+    return foundValue;
+  }
+
+  @Override
+  public void close()
+  {
+
 
 Review comment:
   nit: Add a comment explaining why this function is empty to indicate it is intentional

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367060153
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
 
 Review comment:
   As mentioned before we don't have to worry about null. This is just considering which is more optimize between checking if Double is null or checking if boolean is T/F (and I guess memory for storing one Double vs. storing one double and one Boolean)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367126090
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
 
 Review comment:
   I would also go with primitive double and a boolean found, we can avoid the boxing and I don't think using one Double object would save memory compared to double+boolean

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367043979
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final double NULL_VALUE = 0;
 
 Review comment:
   0 is a valid value for a double. I think `Double.NaN` is safer to indicate that the value is not found. 
   
   Similar comments to `DoubleAnyAggregator` - I think you need to store a byte to indicate whether or not the value has been found since 0, NaN, etc. are all valid values that can show up in a double column.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367260857
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,99 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This Aggregator is created by the {@link DoubleAnyAggregatorFactory} which extends from
+ * {@link NullableNumericAggregatorFactory}. If null needs to be handle, then {@link NullableNumericAggregatorFactory}
+ * will wrap this aggregator in {@link NullableNumericAggregator} and can handle all null in that class.
+ * Hence, no null will ever be pass into this aggregator from the valueSelector.
+ */
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final byte BYTE_FLAG_IS_NOT_SET = 0;
+  private static final byte BYTE_FLAG_IS_SET = 1;
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.put(position, BYTE_FLAG_IS_NOT_SET);
+    buf.putDouble(position + Byte.BYTES, NULL_VALUE);
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367120449
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
 
 Review comment:
   I think this could be `SimpleDoubleAggregatorFactory`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367204550
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,137 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.DoubleMaxAggregator;
+import org.apache.druid.query.aggregation.DoubleMaxBufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.aggregation.SimpleDoubleAggregatorFactory;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends SimpleDoubleAggregatorFactory
+{
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName,
+      @JsonProperty("expression") @Nullable String expression,
+      @JacksonInject ExprMacroTable macroTable
+  )
+  {
+    super(macroTable, name, fieldName, expression);
+  }
+
+  public DoubleAnyAggregatorFactory(String name, String fieldName)
+  {
+    this(name, fieldName, null, ExprMacroTable.nil());
+  }
+
+  @Override
+  protected double nullValue()
+  {
+    return Double.NaN;
+  }
+
+  @Override
+  protected Aggregator buildAggregator(BaseDoubleColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator buildBufferAggregator(BaseDoubleColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name, null, macroTable);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName, expression, macroTable));
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8WithNullToEmpty(fieldName);
 
 Review comment:
   This method and similar could use `CacheKeyBuilder` 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367165057
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DoubleAnyAggregatorFactory that = (DoubleAnyAggregatorFactory) o;
+
+    return name.equals(that.name) && fieldName.equals(that.fieldName);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, fieldName);
+  }
 
 Review comment:
   Actually gonna remove storeDoubleAsFloat

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367032686
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
 
 Review comment:
   not sure if this makes a difference or not, but won't the max size be `Float.BYTES` if storeDoubleAsFloat is true? 
   
   I see the pattern you used is the same as what's used in `DoubleFirstAggregatorFactory` - probably a bug in both?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367168182
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
+
+  public DoubleAnyAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+    foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null && !valueSelector.isNull()) {
 
 Review comment:
   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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367188578
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367160604
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
 
 Review comment:
   getCombiningFactory are combining between segments 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei merged pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei merged pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367165160
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
 
 Review comment:
   Actually gonna remove storeDoubleAsFloat
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367233250
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
 ##########
 @@ -1297,6 +1301,46 @@ public void testLatestAggregators() throws Exception
     );
   }
 
+  // This test the on-heap version of the AnyAggregator (Double/Float/Long/String)
+  @Test
+  public void testAnyAggregator() throws Exception
 
 Review comment:
   nit: It would probably be worth adding an additional test that tests numeric columns agains `druid.numfoo` table since it contains numeric columns that have null values when run in sql compatible null mode, and also tests for ordering by each 'any' aggregator.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367303950
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
 ##########
 @@ -377,6 +377,15 @@ public AuthenticationResult createEscalatedAuthenticationResult()
   );
 
   public static final List<InputRow> ROWS1_WITH_NUMERIC_DIMS = ImmutableList.of(
+      createRow(
 
 Review comment:
   Hmm, it seems like this change is causing some unrelated test failures

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367026167
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
+
+  public DoubleAnyAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+    foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null && !valueSelector.isNull()) {
+      foundValue = valueSelector.getDouble();
+    }
+  }
+
+  @Override
+  public Object get()
+  {
+    return foundValue;
+  }
+
+  @Override
+  public float getFloat()
+  {
+    return foundValue.floatValue();
 
 Review comment:
   This can throw an NPE if isNull is true. I can't tell if this is called from a hot loop or not. @clintropolis or @jon-wei might know how we should deal with this. I see this potential NPE in other aggregators as well.
   
   If you switch to using a primitive in the class, then all these getters can simply cast the local variable

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367232554
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/StringAggregatorUtils.java
 ##########
 @@ -29,7 +29,7 @@
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 
-public class StringFirstLastUtils
+public class StringAggregatorUtils
 
 Review comment:
   If you decide to end up moving `chop` to `StringUtils`, please revert this rename

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367161045
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
 
 Review comment:
   Not sure I understand the question. This is for sorting the aggregated result. I think it does not really matters

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367154652
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java
 ##########
 @@ -121,6 +121,11 @@
 
   public static final byte MEAN_CACHE_TYPE_ID = 0x41;
 
+  public static final byte LONG_ANY_CACHE_TYPE_ID = 0x42;
+  public static final byte DOUBLE_ANY_CACHE_TYPE_ID = 0x42;
 
 Review comment:
   Copy+Paste and forgot to change the value. 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367230483
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,99 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This Aggregator is created by the {@link DoubleAnyAggregatorFactory} which extends from
+ * {@link NullableNumericAggregatorFactory}. If null needs to be handle, then {@link NullableNumericAggregatorFactory}
+ * will wrap this aggregator in {@link NullableNumericAggregator} and can handle all null in that class.
+ * Hence, no null will ever be pass into this aggregator from the valueSelector.
+ */
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final byte BYTE_FLAG_IS_NOT_SET = 0;
+  private static final byte BYTE_FLAG_IS_SET = 1;
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.put(position, BYTE_FLAG_IS_NOT_SET);
+    buf.putDouble(position + Byte.BYTES, NULL_VALUE);
 
 Review comment:
   instead of `NULL_VALUE` maybe use `NullHandling.ZERO_DOUBLE` or like just 0 since this is the only place this is used

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367042441
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
 
 Review comment:
   I know this is an existing pattern, but I prefer having a package private all args constructor to make unit testing easier and have the JsonCreator constructor call the all args constructor. Otherwise the tests need to rely on the static implementation of `ColumnHolder#storeDoubleAsFloat` which can be a huge pain to try and mock correctly.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367157886
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
+
+  public DoubleAnyAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+    foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null && !valueSelector.isNull()) {
+      foundValue = valueSelector.getDouble();
+    }
+  }
+
+  @Override
+  public Object get()
+  {
+    return foundValue;
+  }
+
+  @Override
+  public float getFloat()
+  {
+    return foundValue.floatValue();
+  }
+
+  @Override
+  public long getLong()
+  {
+    return foundValue.longValue();
+  }
+
+  @Override
+  public double getDouble()
+  {
+    return foundValue;
+  }
+
+  @Override
+  public void close()
+  {
+
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367253148
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,137 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.DoubleMaxAggregator;
+import org.apache.druid.query.aggregation.DoubleMaxBufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.aggregation.SimpleDoubleAggregatorFactory;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends SimpleDoubleAggregatorFactory
+{
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName,
+      @JsonProperty("expression") @Nullable String expression,
+      @JacksonInject ExprMacroTable macroTable
+  )
+  {
+    super(macroTable, name, fieldName, expression);
+  }
+
+  public DoubleAnyAggregatorFactory(String name, String fieldName)
+  {
+    this(name, fieldName, null, ExprMacroTable.nil());
+  }
+
+  @Override
+  protected double nullValue()
+  {
+    return Double.NaN;
+  }
+
+  @Override
+  protected Aggregator buildAggregator(BaseDoubleColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator buildBufferAggregator(BaseDoubleColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name, null, macroTable);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName, expression, macroTable));
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8WithNullToEmpty(fieldName);
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367263109
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/StringAggregatorUtils.java
 ##########
 @@ -29,7 +29,7 @@
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 
-public class StringFirstLastUtils
+public class StringAggregatorUtils
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367301502
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregator.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.any;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+
+public class StringAnyAggregator implements Aggregator
+{
+  private final BaseObjectColumnValueSelector valueSelector;
+  private final int maxStringBytes;
+
+  private String foundValue;
+
+  public StringAnyAggregator(BaseObjectColumnValueSelector valueSelector, int maxStringBytes)
+  {
+    this.valueSelector = valueSelector;
+    this.maxStringBytes = maxStringBytes;
+    this.foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null) {
+      final Object object = valueSelector.getObject();
+      if (object != null) {
 
 Review comment:
   sorry I missed this earlier, this check isn't necessary, `DimensionHandlerUtils.convertObjectToString` has it's own null check

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367586511
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,104 @@
+/*
+ * 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.any;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+
+import java.nio.ByteBuffer;
+
+public class StringAnyBufferAggregator implements BufferAggregator
+{
+  private static final int NULL_STRING_LENGTH = -1;
+  private final BaseObjectColumnValueSelector valueSelector;
+  private final int maxStringBytes;
+
+  public StringAnyBufferAggregator(BaseObjectColumnValueSelector valueSelector, int maxStringBytes)
+  {
+    this.valueSelector = valueSelector;
+    this.maxStringBytes = maxStringBytes;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.putInt(position, NULL_STRING_LENGTH);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    int stringSizeBytes = buf.getInt(position);
+    if (stringSizeBytes < 0) {
+      final Object object = valueSelector.getObject();
+      if (object != null) {
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367059233
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
+
+  public DoubleAnyAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+    foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null && !valueSelector.isNull()) {
+      foundValue = valueSelector.getDouble();
+    }
+  }
+
+  @Override
+  public Object get()
+  {
+    return foundValue;
+  }
+
+  @Override
+  public float getFloat()
+  {
+    return foundValue.floatValue();
 
 Review comment:
   (Double/Float/Long)AnyAggregators are created by the (Double/Float/Long)AnyAggregatorFactory. The (Double/Float/Long)AnyAggregatorFactory extends from NullableNumericAggregatorFactory. NullableNumericAggregatorFactory will handle the Null for the (Double/Float/Long)AnyAggregators. If useDefaultValueForNull=true then we don't have to worry about null since all null will be convert to non-null (i.e. 0) before aggregation. If useDefaultValueForNull=false then the NullableNumericAggregatorFactory will wrap the AnyAggregators in NullableNumericAggregator which already have isNull check. (Those methods like getFloat also won't be call if isNull is true) 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367302006
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,104 @@
+/*
+ * 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.any;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+
+import java.nio.ByteBuffer;
+
+public class StringAnyBufferAggregator implements BufferAggregator
+{
+  private static final int NULL_STRING_LENGTH = -1;
+  private final BaseObjectColumnValueSelector valueSelector;
+  private final int maxStringBytes;
+
+  public StringAnyBufferAggregator(BaseObjectColumnValueSelector valueSelector, int maxStringBytes)
+  {
+    this.valueSelector = valueSelector;
+    this.maxStringBytes = maxStringBytes;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.putInt(position, NULL_STRING_LENGTH);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    int stringSizeBytes = buf.getInt(position);
+    if (stringSizeBytes < 0) {
+      final Object object = valueSelector.getObject();
+      if (object != null) {
 
 Review comment:
   same comment about unnecessary check

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367255431
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyBufferAggregator.java
 ##########
 @@ -0,0 +1,99 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This Aggregator is created by the {@link DoubleAnyAggregatorFactory} which extends from
+ * {@link NullableNumericAggregatorFactory}. If null needs to be handle, then {@link NullableNumericAggregatorFactory}
+ * will wrap this aggregator in {@link NullableNumericAggregator} and can handle all null in that class.
+ * Hence, no null will ever be pass into this aggregator from the valueSelector.
+ */
+public class DoubleAnyBufferAggregator implements BufferAggregator
+{
+  private static final byte BYTE_FLAG_IS_NOT_SET = 0;
+  private static final byte BYTE_FLAG_IS_SET = 1;
+  private static final double NULL_VALUE = 0;
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  public DoubleAnyBufferAggregator(BaseDoubleColumnValueSelector valueSelector)
+  {
+    this.valueSelector = valueSelector;
+  }
+
+  @Override
+  public void init(ByteBuffer buf, int position)
+  {
+    buf.put(position, BYTE_FLAG_IS_NOT_SET);
+    buf.putDouble(position + Byte.BYTES, NULL_VALUE);
+  }
+
+  @Override
+  public void aggregate(ByteBuffer buf, int position)
+  {
+    if (buf.get(position) == BYTE_FLAG_IS_NOT_SET) {
+      buf.putDouble(position + Byte.BYTES, valueSelector.getDouble());
+      buf.put(position, BYTE_FLAG_IS_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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367166074
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
 
 Review comment:
   Actually gonna remove storeDoubleAsFloat
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367033326
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
 
 Review comment:
   nit: `Collections.singletonList(fieldName)`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367652349
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
 
 Review comment:
   Hm, looks like the docs are out of date for those, we can fix those later

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367159577
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367157480
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367158997
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DoubleAnyAggregatorFactory that = (DoubleAnyAggregatorFactory) o;
+
+    return name.equals(that.name) && fieldName.equals(that.fieldName);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, fieldName);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "DoubleAnyAggregatorFactory{" +
+           "name='" + name + '\'' +
+           ", fieldName='" + fieldName + '\'' +
+           '}';
+  }
 
 Review comment:
   No idea. DoubleFirstAggregatorFactory does not look at storeDoubleAsFloat for hashCode.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] jon-wei commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367616811
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
+|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 
 Review comment:
   you have this block in StringAnyAggregatorFactory:
   
   ```
   
       this.maxStringBytes = maxStringBytes == null
                             ? StringFirstAggregatorFactory.DEFAULT_MAX_STRING_SIZE
                             : maxStringBytes;
   ```
   
   I would give the SQL function consistent behavior
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367165128
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DoubleAnyAggregatorFactory that = (DoubleAnyAggregatorFactory) o;
+
+    return name.equals(that.name) && fieldName.equals(that.fieldName);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, fieldName);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "DoubleAnyAggregatorFactory{" +
+           "name='" + name + '\'' +
+           ", fieldName='" + fieldName + '\'' +
+           '}';
+  }
 
 Review comment:
   Actually gonna remove storeDoubleAsFloat
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367030965
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DoubleAnyAggregatorFactory that = (DoubleAnyAggregatorFactory) o;
+
+    return name.equals(that.name) && fieldName.equals(that.fieldName);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, fieldName);
+  }
 
 Review comment:
   Shouldn't we be looking at `storeDoubleAsFloat ` as well for equalsAndHashCode?
   
   Can we add an EqualsVerifierTest 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367154903
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367024014
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
 
 Review comment:
   I think you should store this as a primitive so we don't have to do un-necessary boxing. Also I think we need another variable in here to track whether the value is null or not. We would need to implement `isNull` for the aggregator.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367586499
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregator.java
 ##########
 @@ -0,0 +1,84 @@
+/*
+ * 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.any;
+
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseObjectColumnValueSelector;
+import org.apache.druid.segment.DimensionHandlerUtils;
+
+public class StringAnyAggregator implements Aggregator
+{
+  private final BaseObjectColumnValueSelector valueSelector;
+  private final int maxStringBytes;
+
+  private String foundValue;
+
+  public StringAnyAggregator(BaseObjectColumnValueSelector valueSelector, int maxStringBytes)
+  {
+    this.valueSelector = valueSelector;
+    this.maxStringBytes = maxStringBytes;
+    this.foundValue = null;
+  }
+
+  @Override
+  public void aggregate()
+  {
+    if (foundValue == null) {
+      final Object object = valueSelector.getObject();
+      if (object != null) {
 
 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367624495
 
 

 ##########
 File path: docs/querying/sql.md
 ##########
 @@ -203,6 +203,10 @@ Only the COUNT aggregation can accept DISTINCT.
 |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 |`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
 |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
+|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can returns the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will returns any non-null value of `expr`|
+|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
 
 Review comment:
   Let's discuss. We can change this behaviour for LATEST, EARLIEST (and ANY)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367167598
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/StringAggregatorUtils.java
 ##########
 @@ -87,7 +87,7 @@ public static void writePair(
 
     if (pair.rhs != null) {
       mutationBuffer.position(position + Long.BYTES + Integer.BYTES);
-      mutationBuffer.limit(maxStringBytes);
+      mutationBuffer.limit(position + Long.BYTES + Integer.BYTES + maxStringBytes);
 
 Review comment:
   Removed this. Will fix as part of separate PR

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367158326
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java
 ##########
 @@ -0,0 +1,200 @@
+/*
+ * 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.any;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnHolder;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class DoubleAnyAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+{
+  private static final Comparator<Number> VALUE_COMPARATOR = Comparator.nullsFirst(
+      Comparator.comparingDouble(Number::doubleValue)
+  );
+
+  private final String fieldName;
+  private final String name;
+  private final boolean storeDoubleAsFloat;
+
+  @JsonCreator
+  public DoubleAnyAggregatorFactory(
+      @JsonProperty("name") String name,
+      @JsonProperty("fieldName") final String fieldName
+  )
+  {
+    Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
+    Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
+    this.name = name;
+    this.fieldName = fieldName;
+    this.storeDoubleAsFloat = ColumnHolder.storeDoubleAsFloat();
+  }
+
+  @Override
+  protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory)
+  {
+    return metricFactory.makeColumnValueSelector(fieldName);
+  }
+
+  @Override
+  protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyAggregator(selector);
+  }
+
+  @Override
+  protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+  {
+    return new DoubleAnyBufferAggregator(selector);
+  }
+
+  @Override
+  public Comparator getComparator()
+  {
+    return DoubleAnyAggregatorFactory.VALUE_COMPARATOR;
+  }
+
+  @Override
+  @Nullable
+  public Object combine(@Nullable Object lhs, @Nullable Object rhs)
+  {
+    if (lhs != null) {
+      return lhs;
+    } else {
+      return rhs;
+    }
+  }
+
+  @Override
+  public AggregatorFactory getCombiningFactory()
+  {
+    return new DoubleAnyAggregatorFactory(name, name);
+  }
+
+  @Override
+  public List<AggregatorFactory> getRequiredColumns()
+  {
+    return Collections.singletonList(new DoubleAnyAggregatorFactory(fieldName, fieldName));
+  }
+
+  @Override
+  public Object deserialize(Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @Nullable
+  public Object finalizeComputation(@Nullable Object object)
+  {
+    return object;
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @JsonProperty
+  public String getFieldName()
+  {
+    return fieldName;
+  }
+
+  @Override
+  public List<String> requiredFields()
+  {
+    return Arrays.asList(fieldName);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
+
+    return ByteBuffer.allocate(1 + fieldNameBytes.length)
+                     .put(AggregatorUtil.DOUBLE_ANY_CACHE_TYPE_ID)
+                     .put(fieldNameBytes)
+                     .array();
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    if (storeDoubleAsFloat) {
+      return "float";
+    }
+    return "double";
+  }
+
+  @Override
+  public int getMaxIntermediateSize()
+  {
+    return Double.BYTES;
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    DoubleAnyAggregatorFactory that = (DoubleAnyAggregatorFactory) o;
+
+    return name.equals(that.name) && fieldName.equals(that.fieldName);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, fieldName);
+  }
 
 Review comment:
   No idea. DoubleFirstAggregatorFactory does not look at storeDoubleAsFloat for hashCode. 

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] maytasm3 commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
maytasm3 commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367085110
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/StringAggregatorUtils.java
 ##########
 @@ -87,7 +87,7 @@ public static void writePair(
 
     if (pair.rhs != null) {
       mutationBuffer.position(position + Long.BYTES + Integer.BYTES);
-      mutationBuffer.limit(maxStringBytes);
+      mutationBuffer.limit(position + Long.BYTES + Integer.BYTES + maxStringBytes);
 
 Review comment:
   This is to fix an existing bug i found. The limit method for buffer sets it at absolute position. The correct limit should be maxStringBytes after where we are writing the String to the buffer (which is position + Long.BYTES + Integer.BYTES). Hence, limit should be position + Long.BYTES + Integer.BYTES + maxStringBytes

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] suneet-s commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367047216
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/StringAggregatorUtils.java
 ##########
 @@ -87,7 +87,7 @@ public static void writePair(
 
     if (pair.rhs != null) {
       mutationBuffer.position(position + Long.BYTES + Integer.BYTES);
-      mutationBuffer.limit(maxStringBytes);
+      mutationBuffer.limit(position + Long.BYTES + Integer.BYTES + maxStringBytes);
 
 Review comment:
   Why was this change needed?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [druid] clintropolis commented on a change in pull request #9187: Implement ANY aggregator

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9187: Implement ANY aggregator
URL: https://github.com/apache/druid/pull/9187#discussion_r367119977
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregator.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.any;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.BaseDoubleColumnValueSelector;
+
+public class DoubleAnyAggregator implements Aggregator
+{
+  private final BaseDoubleColumnValueSelector valueSelector;
+
+  private Double foundValue;
 
 Review comment:
   Since this is created by `NullableNumericAggregatorFactory` it can be a primitive. `NullableNumericAggregator` and `NullableNumericBufferAggregator` will initialize to a null value, so `aggregate` will never be called unless you encounter a not null value.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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