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/09/02 11:56:36 UTC

[GitHub] [druid] clintropolis commented on a change in pull request #10304: Add vectorization for druid-histogram extension

clintropolis commented on a change in pull request #10304:
URL: https://github.com/apache/druid/pull/10304#discussion_r481953935



##########
File path: extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramBufferAggregatorInternal.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.histogram;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A helper class used by {@link ApproximateHistogramBufferAggregator} and {@link ApproximateHistogramVectorAggregator}
+ * for aggregation operations on byte buffers. Getting the object from value selectors is outside this class.
+ */
+final class ApproximateHistogramBufferAggregatorInternal

Review comment:
       super nitpick, feel free to ignore, but maybe consider naming this (and similar classes) to something like `ApproximateHistogramBufferAggregatorHelper` instead of `ApproximateHistogramBufferAggregatorInternal` to be more consistent with the naming of this style of class with the rest of the codebase. I looked around and this PR has the only classes with an `Internal` suffix but there are many with the `Helper` suffix, and is consistent with the javadoc for this class. 

##########
File path: extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramVectorAggregator.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.histogram;
+
+import org.apache.druid.query.aggregation.VectorAggregator;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+public class ApproximateHistogramVectorAggregator implements VectorAggregator
+{
+
+  private final VectorValueSelector selector;
+  private final ApproximateHistogramBufferAggregatorInternal innerAggregator;
+
+  public ApproximateHistogramVectorAggregator(
+      VectorValueSelector selector,
+      int resolution
+  )
+  {
+    this.selector = selector;
+    this.innerAggregator = new ApproximateHistogramBufferAggregatorInternal(resolution);
+  }
+
+  @Override
+  public void init(final ByteBuffer buf, final int position)
+  {
+    innerAggregator.init(buf, position);
+  }
+
+  @Override
+  public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
+  {
+    final boolean[] isValueNull = selector.getNullVector();
+    final float[] vector = selector.getFloatVector();
+    ApproximateHistogram histogram = innerAggregator.get(buf, position);
+
+    for (int i = startRow; i < endRow; i++) {
+      if (isValueNull != null && isValueNull[i]) {

Review comment:
       Ah yeah i wasn't imagining checking all the conditions in the loop, the `checkNulls` value I was thinking of would be in the loop, similar to `hasNulls` in your example. Thinking further about it though, there is no real need/advantage to checking `NullHandling.sqlCompatible()`.

##########
File path: extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramVectorAggregatorTest.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.histogram;
+
+import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.query.aggregation.VectorAggregator;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
+import org.apache.druid.segment.column.ValueType;
+import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
+import org.apache.druid.segment.vector.VectorValueSelector;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+
+public class ApproximateHistogramVectorAggregatorTest

Review comment:
       it isn't obvious from this PR, but out of curiosity are there any tests which confirm that the vectorized aggregator results match the non-vectorized output?

##########
File path: extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java
##########
@@ -102,6 +105,21 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
     );
   }
 
+  @Override
+  public VectorAggregator factorizeVector(VectorColumnSelectorFactory metricVectorFactory)
+  {
+    return new ApproximateHistogramVectorAggregator(
+        metricVectorFactory.makeValueSelector(fieldName),
+        resolution
+    );
+  }
+
+  @Override
+  public boolean canVectorize(ColumnInspector columnInspector)
+  {
+    return true;

Review comment:
       We need to handle it _somehow_ because if not it will fail when making the value selector (because there is no string value selector) ```org.apache.druid.query.QueryInterruptedException: Cannot make VectorValueSelector for column with class[org.apache.druid.segment.column.StringDictionaryEncodedColumn]```. This is inconsistent with the non-vectorized behavior, which treats the input as `0` from the dimension selectors.
   
   The ways it can be handled are with either the `canVectorize` method checking explicitly for numeric types, or special handling in `factorizeVector` to use a nil vector selector instead of trying to make a value selector. You probably want similar checks for other agg factories, as is appropriate for the types they handle.




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



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