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/10 11:28:39 UTC

[GitHub] [druid] clintropolis opened a new pull request #9161: refactor numeric first/last aggregators to fix null handling bugs

clintropolis opened a new pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161
 
 
   ### Description
   
   I did some additional digging related to findings of #9154 and #9159, and managed to hit an npe while sorting a top-n query by a 'long first' aggregator. Checking out the comparators, I assumed it had similar issues to #9159 and would be a quick/simple change, but how ever wrong I was.
   
   It turns out, the 'first'/'last' family of aggregators was building on top of `NullableNumericAggregatorFactory`, following #8834, though the lack of correct SQL compatible null handling definitely predates that change. Anyway, the problem with using `NullableNumericAggregatorFactory` is that the 'first'/'last' aggregators aren't aggregating numbers, but rather a complex type of a `SerializablePair` containing the timestamp and the number column value. This would make the aggregator result for a row itself be null rather than a pair containing the timestamp and the null right hand side value, which caused all sorts of funny business down the line when sorting or combining values.
   
   For a solution, I did some heavy refactoring, pulling out `NumericFirstAggregator`, `NumericFirstBufferAggregator`, `NumericLastAggregator`, and `NumericLastBufferAggregator`. Further refactoring could likely pull out an additional base type between the first and last aggs and buffer aggs, because the only differences is the time comparison and the initial time values, but it gets kind of messy because they aren't in the same package...
   
   Anyway, common code for tracking if the current value is null for on heap and buffer aggs is in these types, and so on and so forth, comparators fixed up, tests added, and so on, so I _think_ these aggs should now handle null numbers correctly.
   
   
   <hr>
   
   This PR has:
   - [x] been self-reviewed.
   - [ ] 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 comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368727641
 
 

 ##########
 File path: processing/src/test/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregationTest.java
 ##########
 @@ -30,14 +30,16 @@
 import org.apache.druid.query.aggregation.TestObjectColumnSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.testing.InitializedNullHandlingTest;
 import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.nio.ByteBuffer;
+import java.util.Comparator;
 
-public class DoubleFirstAggregationTest
+public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
 
 Review comment:
   it looks like these tests will only check `useDefaultValuesForNull = true` or is there some config I'm not seeing that sets it to `false` as well in another run?

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368720995
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java
 ##########
 @@ -45,10 +46,34 @@
 import java.util.Map;
 import java.util.Objects;
 
-public class DoubleFirstAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector>
+public class DoubleFirstAggregatorFactory extends AggregatorFactory
 {
+  private static final Aggregator NIL_AGGREGATOR = new DoubleFirstAggregator(
+      NilColumnValueSelector.instance(),
+      NilColumnValueSelector.instance()
+  )
+  {
+    @Override
+    public void aggregate()
+    {
+      // no-op
+    }
+  };
+
+  private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new DoubleFirstBufferAggregator(
+      NilColumnValueSelector.instance(),
+      NilColumnValueSelector.instance()
+  )
+  {
+    @Override
+    public void aggregate(ByteBuffer buf, int position)
+    {
+      // no-op
+    }
+  };
+
   public static final Comparator<SerializablePair<Long, Double>> VALUE_COMPARATOR =
-      Comparator.comparingDouble(o -> o.rhs);
+      SerializablePair.createNullHandlingComparator(Double::compare, true);
 
 Review comment:
   It took me a long time to try and figure out what the comparator was used for. I got wrapped up in the fact that the aggregator was meant compare timestamps, that I didn't realize this was for ordering. I think a javadoc on  #`AggregatorFactory#getComparator` would have cleared up my confusion pretty quickly

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
jon-wei merged pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161
 
 
   

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368708486
 
 

 ##########
 File path: core/src/main/java/org/apache/druid/collections/SerializablePair.java
 ##########
 @@ -45,4 +46,25 @@ public T2 getRhs()
   {
     return rhs;
   }
+
+  public static <T1, T2> Comparator<SerializablePair<T1, T2>> createNullHandlingComparator(
 
 Review comment:
   looks like this is missing unit tests? Also javadocs since this is a utility that would be used by many other classes

----------------------------------------------------------------
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 #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r368154212
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java
 ##########
 @@ -230,7 +248,7 @@ public String getTypeName()
   @Override
   public int getMaxIntermediateSize()
   {
-    return Long.BYTES + Double.BYTES;
+    return Long.BYTES + Double.BYTES + 1;
 
 Review comment:
   it seems unlikely to me that the number of bytes in a byte is going to change, but ok :p

----------------------------------------------------------------
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 #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r365493458
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/last/NumericLastBufferAggregator.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.last;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public abstract class NumericLastBufferAggregator<TSelector extends BaseNullableColumnValueSelector>
+    implements BufferAggregator
+{
+  static final int NULL_OFFSET = Long.BYTES;
+  static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
+  static byte RHS_NOT_NULL = 0x00;
+  static byte RHS_NULL = 0x01;
+
+  final boolean useDefault = NullHandling.replaceWithDefault();
+
+  final BaseLongColumnValueSelector timeSelector;
+  final TSelector valueSelector;
+
+  public NumericLastBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
+  {
+    this.timeSelector = timeSelector;
+    this.valueSelector = valueSelector;
+  }
+
+  abstract void initValue(ByteBuffer buf, int position);
+
+  abstract void putValue(ByteBuffer buf, int position);
+
+  boolean isValueNull(ByteBuffer buf, int position)
+  {
+    return buf.get(position + NULL_OFFSET) == 1;
+  }
+
+  void updateTimeWithValue(ByteBuffer buf, int position, long time)
+  {
+    buf.putLong(position, time);
+    putValue(buf, position);
 
 Review comment:
   nit: to follow the ordering in the buffer, maybe move putValue call after the null marker setting

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368717243
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregator.java
 ##########
 @@ -20,42 +20,29 @@
 package org.apache.druid.query.aggregation.first;
 
 import org.apache.druid.collections.SerializablePair;
-import org.apache.druid.query.aggregation.Aggregator;
 import org.apache.druid.segment.BaseDoubleColumnValueSelector;
 import org.apache.druid.segment.BaseLongColumnValueSelector;
 
-public class DoubleFirstAggregator implements Aggregator
+public class DoubleFirstAggregator extends NumericFirstAggregator<BaseDoubleColumnValueSelector>
 
 Review comment:
   Nice abstraction! 🎉 
   
   note to self: can the get call be abstracted into the base class?

----------------------------------------------------------------
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] jihoonson commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r368145651
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/NumericFirstBufferAggregator.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.first;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public abstract class NumericFirstBufferAggregator<TSelector extends BaseNullableColumnValueSelector>
+    implements BufferAggregator
+{
+  static final int NULL_OFFSET = Long.BYTES;
+  static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
+  static byte RHS_NOT_NULL = 0x00;
+  static byte RHS_NULL = 0x01;
+
+  final boolean useDefault = NullHandling.replaceWithDefault();
+
+  final BaseLongColumnValueSelector timeSelector;
+  final TSelector valueSelector;
+
+  public NumericFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
+  {
+    this.timeSelector = timeSelector;
+    this.valueSelector = valueSelector;
+  }
+
+  abstract void initValue(ByteBuffer buf, int position);
+
+  abstract void putValue(ByteBuffer buf, int position);
+
+  boolean isValueNull(ByteBuffer buf, int position)
+  {
+    return buf.get(position + NULL_OFFSET) == 1;
 
 Review comment:
   `== RHS_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] jihoonson commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r368151055
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java
 ##########
 @@ -230,7 +248,7 @@ public String getTypeName()
   @Override
   public int getMaxIntermediateSize()
   {
-    return Long.BYTES + Double.BYTES;
+    return Long.BYTES + Double.BYTES + 1;
 
 Review comment:
   `Byte.BYTES`. Or perhaps adding a new variable `NULL_SIZE`?

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368789965
 
 

 ##########
 File path: core/src/main/java/org/apache/druid/collections/SerializablePair.java
 ##########
 @@ -45,4 +46,25 @@ public T2 getRhs()
   {
     return rhs;
   }
+
+  public static <T1, T2> Comparator<SerializablePair<T1, T2>> createNullHandlingComparator(
 
 Review comment:
   This is covered by calcite query tests that order by the first/last aggregators; afaik SerializedPair is only used by first/last despite it's generic name, to store a timestamp and 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


[GitHub] [druid] clintropolis commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r368154212
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java
 ##########
 @@ -230,7 +248,7 @@ public String getTypeName()
   @Override
   public int getMaxIntermediateSize()
   {
-    return Long.BYTES + Double.BYTES;
+    return Long.BYTES + Double.BYTES + 1;
 
 Review comment:
   it seems unlikely to me that the number of bytes in a byte is not going to change, but ok :p

----------------------------------------------------------------
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 #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
jon-wei commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r365493390
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/last/NumericLastBufferAggregator.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.last;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public abstract class NumericLastBufferAggregator<TSelector extends BaseNullableColumnValueSelector>
+    implements BufferAggregator
+{
+  static final int NULL_OFFSET = Long.BYTES;
+  static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
+  static byte RHS_NOT_NULL = 0x00;
+  static byte RHS_NULL = 0x01;
+
+  final boolean useDefault = NullHandling.replaceWithDefault();
+
+  final BaseLongColumnValueSelector timeSelector;
+  final TSelector valueSelector;
+
+  public NumericLastBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
+  {
+    this.timeSelector = timeSelector;
+    this.valueSelector = valueSelector;
+  }
+
+  abstract void initValue(ByteBuffer buf, int position);
 
 Review comment:
   Can you add javadocs for these buffer methods?

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368788599
 
 

 ##########
 File path: processing/src/test/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregationTest.java
 ##########
 @@ -30,14 +30,16 @@
 import org.apache.druid.query.aggregation.TestObjectColumnSelector;
 import org.apache.druid.segment.ColumnSelectorFactory;
 import org.apache.druid.segment.column.ColumnHolder;
+import org.apache.druid.testing.InitializedNullHandlingTest;
 import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.nio.ByteBuffer;
+import java.util.Comparator;
 
-public class DoubleFirstAggregationTest
+public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
 
 Review comment:
   travis runs tests in both modes, this is to let tests run in intellij which doesn't get initialized 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] suneet-s commented on a change in pull request #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368713952
 
 

 ##########
 File path: docs/querying/aggregations.md
 ##########
 @@ -136,7 +136,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
 
 #### `doubleFirst` aggregator
 
-`doubleFirst` computes the metric value with the minimum timestamp or 0 if no row exist
+`doubleFirst` computes the metric value with the minimum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
 
 Review comment:
   Are you referring to this property - https://github.com/apache/druid/blob/master/docs/configuration/index.md#sql-compatible-null-handling ? Would be nice to link to the configuration here.
   
   nit: I'd re-phrase slightly
   
   ```
   computes the metric value with the minimum timestamp. If no row exists, it will return 0 or `null` if [SQL compatible mode](../configuration/index.md#sql-compatible-null-handling) is enabled
   ``` 

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
suneet-s commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368722885
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java
 ##########
 @@ -126,35 +158,54 @@ public AggregatorFactory getCombiningFactory()
     return new DoubleFirstAggregatorFactory(name, name)
     {
       @Override
-      public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+      public Aggregator factorize(ColumnSelectorFactory metricFactory)
       {
+        final ColumnValueSelector<SerializablePair<Long, Double>> selector =
 
 Review comment:
   Based on javadocs in `makeColumnValueSelector` this selector can be `NilColumnValueSelector`
   
   in which case `selector.getObject()` on line 170 would return null and line 171 would throw an NPE?
   similar comment for the `factorizeBuffered` method

----------------------------------------------------------------
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 #9161: refactor numeric first/last aggregators to fix null handling bugs

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: refactor numeric first/last aggregators to fix null handling bugs
URL: https://github.com/apache/druid/pull/9161#discussion_r368153982
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/NumericFirstBufferAggregator.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.first;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.BaseLongColumnValueSelector;
+import org.apache.druid.segment.BaseNullableColumnValueSelector;
+
+import java.nio.ByteBuffer;
+
+public abstract class NumericFirstBufferAggregator<TSelector extends BaseNullableColumnValueSelector>
+    implements BufferAggregator
+{
+  static final int NULL_OFFSET = Long.BYTES;
+  static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
+  static byte RHS_NOT_NULL = 0x00;
+  static byte RHS_NULL = 0x01;
+
+  final boolean useDefault = NullHandling.replaceWithDefault();
+
+  final BaseLongColumnValueSelector timeSelector;
+  final TSelector valueSelector;
+
+  public NumericFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
+  {
+    this.timeSelector = timeSelector;
+    this.valueSelector = valueSelector;
+  }
+
+  abstract void initValue(ByteBuffer buf, int position);
+
+  abstract void putValue(ByteBuffer buf, int position);
+
+  boolean isValueNull(ByteBuffer buf, int position)
+  {
+    return buf.get(position + NULL_OFFSET) == 1;
 
 Review comment:
   switching these to use `NullHandling.IS_NULL_BYTE` and `NullHandling.IS_NOT_NULL_BYTE`

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368789474
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregator.java
 ##########
 @@ -20,42 +20,29 @@
 package org.apache.druid.query.aggregation.first;
 
 import org.apache.druid.collections.SerializablePair;
-import org.apache.druid.query.aggregation.Aggregator;
 import org.apache.druid.segment.BaseDoubleColumnValueSelector;
 import org.apache.druid.segment.BaseLongColumnValueSelector;
 
-public class DoubleFirstAggregator implements Aggregator
+public class DoubleFirstAggregator extends NumericFirstAggregator<BaseDoubleColumnValueSelector>
 
 Review comment:
   Not without boxing the primitive

----------------------------------------------------------------
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 #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368789089
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java
 ##########
 @@ -126,35 +158,54 @@ public AggregatorFactory getCombiningFactory()
     return new DoubleFirstAggregatorFactory(name, name)
     {
       @Override
-      public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector)
+      public Aggregator factorize(ColumnSelectorFactory metricFactory)
       {
+        final ColumnValueSelector<SerializablePair<Long, Double>> selector =
 
 Review comment:
   Since this is for combining, the selector will be of the serialized pair complex objects, which will not be 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] clintropolis commented on a change in pull request #9161: first/last aggregators and nulls

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9161: first/last aggregators and nulls
URL: https://github.com/apache/druid/pull/9161#discussion_r368790320
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregator.java
 ##########
 @@ -20,42 +20,29 @@
 package org.apache.druid.query.aggregation.first;
 
 import org.apache.druid.collections.SerializablePair;
-import org.apache.druid.query.aggregation.Aggregator;
 import org.apache.druid.segment.BaseDoubleColumnValueSelector;
 import org.apache.druid.segment.BaseLongColumnValueSelector;
 
-public class DoubleFirstAggregator implements Aggregator
+public class DoubleFirstAggregator extends NumericFirstAggregator<BaseDoubleColumnValueSelector>
 
 Review comment:
   Well, I guess that is going to happen anyway in making the pair.. so I guess maybe the on heap version of get could be shared, but not really possible for the buffer 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