[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-09 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282706910
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  private final int compression;
+
+  @GuardedBy("this")
+  private final Map> sketches = new 
IdentityHashMap<>();
+
+  public TDigestBuildSketchBufferAggregator(
+  final ColumnValueSelector valueSelector,
+  @Nullable final Integer compression
+  )
+  {
+Preconditions.checkNotNull(valueSelector);
+this.selector = valueSelector;
+if (compression != null) {
+  this.compression = compression;
+} else {
+  this.compression = 
TDigestBuildSketchAggregatorFactory.DEFAULT_COMPRESSION;
+}
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
+  {
+MergingDigest emptyDigest = new MergingDigest(compression);
+putSketch(buffer, position, emptyDigest);
+  }
+
+  @Override
+  public synchronized void aggregate(ByteBuffer buffer, int position)
+  {
+MergingDigest sketch = sketches.get(buffer).get(position);
+Object x = selector.getObject();
+if (x instanceof Number) {
+  sketch.add(((Number) x).doubleValue());
+} else {
+  throw new IAE("Unexpected value of type " + x.getClass().getName() + " 
encountered");
+}
+  }
+
+  @Override
+  public synchronized Object get(final ByteBuffer buffer, final int position)
+  {
+return sketches.get(buffer).get(position);
 
 Review comment:
   Maybe we need to use an off-heap implementation in the future. Seems like 
there's no off-heap implementation currently.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-09 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282705978
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  private final int compression;
+
+  @GuardedBy("this")
+  private final Map> sketches = new 
IdentityHashMap<>();
+
+  public TDigestBuildSketchBufferAggregator(
+  final ColumnValueSelector valueSelector,
+  @Nullable final Integer compression
+  )
+  {
+Preconditions.checkNotNull(valueSelector);
+this.selector = valueSelector;
+if (compression != null) {
+  this.compression = compression;
+} else {
+  this.compression = 
TDigestBuildSketchAggregatorFactory.DEFAULT_COMPRESSION;
+}
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
+  {
+MergingDigest emptyDigest = new MergingDigest(compression);
+putSketch(buffer, position, emptyDigest);
+  }
+
+  @Override
+  public synchronized void aggregate(ByteBuffer buffer, int position)
+  {
+MergingDigest sketch = sketches.get(buffer).get(position);
+Object x = selector.getObject();
+if (x instanceof Number) {
+  sketch.add(((Number) x).doubleValue());
+} else {
+  throw new IAE("Unexpected value of type " + x.getClass().getName() + " 
encountered");
+}
+  }
+
+  @Override
+  public synchronized Object get(final ByteBuffer buffer, final int position)
+  {
+return sketches.get(buffer).get(position);
 
 Review comment:
   Hmm yeah, it makes sense since `MergingDigest` is stored on heap.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-08 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282295708
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  @Nonnull
+  private final int compression;
+
+  @GuardedBy("this")
+  private Map> sketches = new 
IdentityHashMap<>();
+
+  public TDigestBuildSketchBufferAggregator(
+  final ColumnValueSelector valueSelector,
+  final Integer compression
+  )
+  {
+Preconditions.checkNotNull(valueSelector);
+this.selector = valueSelector;
+if (compression != null) {
+  this.compression = compression;
+} else {
+  this.compression = TDigestBuildSketchAggregator.DEFAULT_COMPRESSION;
+}
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
 
 Review comment:
   If a query is issued while a stream ingestion task is running, then the 
query would be routed to that task. This is when concurrent reads and writes 
can happen. Since only `OnHeapIncrementalIndex` is used at ingestion time which 
uses `Aggregator`, we need to consider if there's any concurrency issue between 
`get()` and `aggregate()`. Check out these comments: 
https://github.com/apache/incubator-druid/pull/5002#issuecomment-341179982, 
https://github.com/apache/incubator-druid/pull/5148#discussion_r170906998
   
   I'm not sure why `HistogramAggregator` is not synchronized even though it 
looks to have to. 


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-08 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282295729
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  private final int compression;
+
+  @GuardedBy("this")
+  private final Map> sketches = new 
IdentityHashMap<>();
+
+  public TDigestBuildSketchBufferAggregator(
+  final ColumnValueSelector valueSelector,
+  @Nullable final Integer compression
+  )
+  {
+Preconditions.checkNotNull(valueSelector);
+this.selector = valueSelector;
+if (compression != null) {
+  this.compression = compression;
+} else {
+  this.compression = 
TDigestBuildSketchAggregatorFactory.DEFAULT_COMPRESSION;
+}
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
+  {
+MergingDigest emptyDigest = new MergingDigest(compression);
+putSketch(buffer, position, emptyDigest);
+  }
+
+  @Override
+  public synchronized void aggregate(ByteBuffer buffer, int position)
+  {
+MergingDigest sketch = sketches.get(buffer).get(position);
+Object x = selector.getObject();
+if (x instanceof Number) {
+  sketch.add(((Number) x).doubleValue());
+} else {
+  throw new IAE("Unexpected value of type " + x.getClass().getName() + " 
encountered");
+}
+  }
+
+  @Override
+  public synchronized Object get(final ByteBuffer buffer, final int position)
+  {
+return sketches.get(buffer).get(position);
 
 Review comment:
   Thank you for calling out 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-08 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282269647
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  @Nonnull
+  private final int compression;
+
+  @GuardedBy("this")
+  private Map> sketches = new 
IdentityHashMap<>();
+
+  public TDigestBuildSketchBufferAggregator(
+  final ColumnValueSelector valueSelector,
+  final Integer compression
+  )
+  {
+Preconditions.checkNotNull(valueSelector);
+this.selector = valueSelector;
+if (compression != null) {
+  this.compression = compression;
+} else {
+  this.compression = TDigestBuildSketchAggregator.DEFAULT_COMPRESSION;
+}
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
 
 Review comment:
   Yeah, it's lame that the doc is missing about what should be synchronized. I 
think DataSketches implementations are wrong. It doesn't have to be 
synchronized because concurrent reads and writes can happen only in incremental 
index. You would see other BufferAggregator implementations of druid-core or 
druid-extensions-core don't do it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-08 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282269391
 
 

 ##
 File path: 
docs/content/development/extensions-contrib/tdigestsketch-quantiles.md
 ##
 @@ -0,0 +1,159 @@
+---
+layout: doc_page
+title: "T-Digest Quantiles Sketch module"
+---
+
+
+
+# T-Digest Quantiles Sketch module
+
+This module provides Apache Druid (incubating) approximate sketch aggregators 
based on T-Digest.
+T-Digest (https://github.com/tdunning/t-digest) is a popular datastructure for 
accurate on-line accumulation of
+rank-based statistics such as quantiles and trimmed means.
+The datastructure is also designed for parallel programming use cases like 
distributed aggregations or map reduce jobs by making combining two 
intermediate t-digests easy and efficient.
+
+There are three flavors of T-Digest sketch aggregator available in Apache 
Druid (incubating):
+
+1. buildTDigestSketch - used for building T-Digest sketches from raw numeric 
values. It generally makes sense to
+use this aggregator when ingesting raw data into Druid. One can also use this 
aggregator during query time too to
+generate sketches, just that one would be building these sketches on every 
query execution instead of building them
+once during ingestion.
+2. mergeTDigestSketch - used for merging pre-built T-Digest sketches. This 
aggregator is generally used during
+query time to combine sketches generated by buildTDigestSketch aggregator.
+3. quantilesFromTDigestSketch - used for generating quantiles from T-Digest 
sketches. This aggregator is generally used
+during query time to generate quantiles from sketches built using the above 
two sketch generating aggregators.
+
+To use this aggregator, make sure you 
[include](../../operations/including-extensions.html) the extension in your 
config file:
+
+```
+druid.extensions.loadList=["druid-tdigestsketch"]
+```
+
+### Aggregator
+
+The result of the aggregation is a T-Digest sketch that is built ingesting 
numeric values from the raw data.
+
+```json
+{
+  "type" : "buildTDigestSketch",
+  "name" : ,
+  "fieldName" : ,
+  "compression": 
+ }
+```
+Example:
+```json
+{
+   "type": "buildTDigestSketch",
+   "name": "sketch",
+   "fieldName": "session_duration",
+   "compression": 200
+}
+```
+
+|property|description|required?|
+||---|-|
+|type|This String should always be "buildTDigestSketch"|yes|
+|name|A String for the output (result) name of the calculation.|yes|
+|fieldName|A String for the name of the input field containing raw numeric 
values.|yes|
+|compression|Parameter that determines the accuracy and size of the sketch. 
Higher compression means higher accuracy but more space to store sketches.|no, 
defaults to 100|
+
+
+The result of the aggregation is a T-Digest sketch that is built by merging 
pre-built T-Digest sketches.
+
+```json
+{
+  "type" : "mergeTDigestSketch",
+  "name" : ,
+  "fieldName" : ,
+  "compression": 
+ }
+```
+
+|property|description|required?|
+||---|-|
+|type|This String should always be "buildTDigestSketch"|yes|
+|name|A String for the output (result) name of the calculation.|yes|
+|fieldName|A String for the name of the input field containing raw numeric 
values.|yes|
+|compression|Parameter that determines the accuracy and size of the sketch. 
Higher compression means higher accuracy but more space to store sketches.|no, 
defaults to 100|
+
+Example:
+```json
+{
+   "queryType": "groupBy",
+   "dataSource": "test_datasource",
+   "granularity": "ALL",
+   "dimensions": [],
+   "aggregations": [{
+   "type": "mergeTDigestSketch",
+   "name": "merged_sketch",
+   "fieldName": "ingested_sketch",
+   "compression": 200
+   }],
+   "intervals": ["2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z"]
+}
+```
+### Post Aggregators
+
+ Quantiles
+
+This returns an array of quantiles corresponding to a given array of fractions.
+
+```json
+{
+  "type"  : "quantilesFromTDigestSketch",
+  "name": ,
+  "field"  : ,
+  "fractions" : 
+}
+```
+
+|property|description|required?|
+||---|-|
+|type|This String should always be "buildTDigestSketch"|yes|
 
 Review comment:
   Should be `quantilesFromTDigestSketch`


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-05-08 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r282269622
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
+
+  @Nonnull
+  private final String name;
+  @Nonnull
+  private final String fieldName;
+  @Nonnull
+  final Integer compression;
+  @Nonnull
+  private final byte cacheTypeId;
+
+  public static final String TYPE_NAME = "buildTDigestSketch";
+
+  @JsonCreator
+  public TDigestBuildSketchAggregatorFactory(
+  @JsonProperty("name") final String name,
+  @JsonProperty("fieldName") final String fieldName,
+  @Nullable @JsonProperty("compression") final Integer compression
+  )
+  {
+this(name, fieldName, compression, 
AggregatorUtil.TDIGEST_BUILD_SKETCH_CACHE_TYPE_ID);
+  }
+
+  TDigestBuildSketchAggregatorFactory(
+  final String name,
+  final String fieldName,
+  @Nullable final Integer compression,
+  final byte cacheTypeId
+  )
+  {
+Objects.requireNonNull(name, "Must have a valid, non-null aggregator 
name");
+this.name = name;
+Objects.requireNonNull(fieldName, "Parameter fieldName must be specified");
+this.fieldName = fieldName;
+this.compression = compression == null ? DEFAULT_COMRESSION : compression;
+this.cacheTypeId = cacheTypeId;
+  }
+
+
+  @Override
+  public byte[] getCacheKey()
+  {
+return new CacheKeyBuilder(
+cacheTypeId
+).appendString(fieldName).appendInt(compression).build();
+  }
+
+
+  @Override
+  public Aggregator factorize(ColumnSelectorFactory metricFactory)
+  {
+ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
+if (cap == null || ValueType.isNumeric(cap.getType())) {
+  final ColumnValueSelector selector = 
metricFactory.makeColumnValueSelector(fieldName);
+  return new TDigestBuildSketchAggregator(selector, compression);
+} else {
+  final ColumnValueSelector selector = 
metricFactory.makeColumnValueSelector(fieldName);
+  return new TDigestMergeSketchAggregator(selector, compression);
+}
+  }
+
+  @Override

[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279134453
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
+
+  @Nonnull
+  private final String name;
+  @Nonnull
+  private final String fieldName;
+  @Nonnull
+  final Integer compression;
+  @Nonnull
+  private final byte cacheTypeId;
+
+  public static final String TYPE_NAME = "buildTDigestSketch";
+
+  @JsonCreator
+  public TDigestBuildSketchAggregatorFactory(
+  @JsonProperty("name") final String name,
+  @JsonProperty("fieldName") final String fieldName,
+  @Nullable @JsonProperty("compression") final Integer compression
+  )
+  {
+this(name, fieldName, compression, 
AggregatorUtil.TDIGEST_BUILD_SKETCH_CACHE_TYPE_ID);
+  }
+
+  TDigestBuildSketchAggregatorFactory(
+  final String name,
+  final String fieldName,
+  @Nullable final Integer compression,
+  final byte cacheTypeId
+  )
+  {
+Objects.requireNonNull(name, "Must have a valid, non-null aggregator 
name");
+this.name = name;
+Objects.requireNonNull(fieldName, "Parameter fieldName must be specified");
+this.fieldName = fieldName;
+this.compression = compression == null ? DEFAULT_COMRESSION : compression;
+this.cacheTypeId = cacheTypeId;
+  }
+
+
+  @Override
+  public byte[] getCacheKey()
+  {
+return new CacheKeyBuilder(
+cacheTypeId
+).appendString(fieldName).appendInt(compression).build();
+  }
+
+
+  @Override
+  public Aggregator factorize(ColumnSelectorFactory metricFactory)
+  {
+ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
+if (cap == null || ValueType.isNumeric(cap.getType())) {
+  final ColumnValueSelector selector = 
metricFactory.makeColumnValueSelector(fieldName);
+  return new TDigestBuildSketchAggregator(selector, compression);
+} else {
+  final ColumnValueSelector selector = 
metricFactory.makeColumnValueSelector(fieldName);
+  return new TDigestMergeSketchAggregator(selector, compression);
+}
+  }
+
+  @Override

[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279934071
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorTest.java
 ##
 @@ -0,0 +1,284 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.data.input.Row;
+import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.aggregation.AggregationTestHelper;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class TDigestSketchAggregatorTest
+{
+
+  private final AggregationTestHelper helper;
+  private final AggregationTestHelper timeSeriesHelper;
 
 Review comment:
   Please remove unused 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279135638
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  @Nonnull
+  private final int compression;
+
+  @GuardedBy("this")
+  private Map> sketches = new 
IdentityHashMap<>();
+
+  public TDigestBuildSketchBufferAggregator(
+  final ColumnValueSelector valueSelector,
+  final Integer compression
+  )
+  {
+Preconditions.checkNotNull(valueSelector);
+this.selector = valueSelector;
+if (compression != null) {
+  this.compression = compression;
+} else {
+  this.compression = TDigestBuildSketchAggregator.DEFAULT_COMPRESSION;
+}
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
 
 Review comment:
   `BufferAggregator` doesn't have to be synchronized because it's not used in 
incremental index.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279901373
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
+
+  @Nonnull
+  private final String name;
+  @Nonnull
+  private final String fieldName;
+  @Nonnull
+  final Integer compression;
 
 Review comment:
   Can be int without `@Nonnull` annotation.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279932319
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchObjectStrategy.java
 ##
 @@ -0,0 +1,62 @@
+/*
+ * 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.tdigestsketch;
+
+import com.tdunning.math.stats.MergingDigest;
+import org.apache.druid.segment.data.ObjectStrategy;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+public class TDigestSketchObjectStrategy implements 
ObjectStrategy
+{
+  private static final byte[] EMPTY_BYTES = new byte[0];
+
+  @Override
+  public Class getClazz()
+  {
+return MergingDigest.class;
+  }
+
+  @Override
+  public MergingDigest fromByteBuffer(ByteBuffer buffer, int numBytes)
+  {
+if (numBytes == 0) {
+  return null;
+}
+buffer.limit(buffer.position() + numBytes);
 
 Review comment:
   Is it safe to set limit for `buffer` here? I'm not sure. Suggest to get a 
read-only 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r278355948
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregator.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.tdigestsketch;
+
+import com.tdunning.math.stats.MergingDigest;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+/**
+ * Aggregator to build tDigest sketches on numeric values.
+ */
+public class TDigestBuildSketchAggregator implements Aggregator
+{
+
+  private final ColumnValueSelector selector;
+
+  // Default compression
+  static final int DEFAULT_COMPRESSION = 50;
+
+  @GuardedBy("this")
+  private MergingDigest histogram;
+
+
+  public TDigestBuildSketchAggregator(ColumnValueSelector selector, Integer 
compression)
+  {
+this.selector = selector;
+if (compression != null) {
+  this.histogram = new MergingDigest(compression);
+} else {
+  this.histogram = new MergingDigest(DEFAULT_COMPRESSION);
+}
+  }
+
+  @Override
+  public synchronized void aggregate()
+  {
+if (selector.getObject() instanceof Number) {
+  histogram.add(((Number) selector.getObject()).doubleValue());
+} else {
+  throw new IAE("Expected a number, received "
++ selector.getObject()
++ " of type "
++ selector.getObject());
+}
+  }
+
+  @Nullable
+  @Override
+  public synchronized Object get()
+  {
+return histogram;
+  }
+
+  @Override
+  public float getFloat()
+  {
+throw new UnsupportedOperationException("not implemented");
 
 Review comment:
   Maybe it should say "Casting to float type is not supported". Similar for 
`getLong()`.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279906236
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestMergeSketchBufferAggregator.java
 ##
 @@ -0,0 +1,111 @@
+/*
+ * 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.tdigestsketch;
+
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+
+/**
+ * Aggregator that is capable of combining t-digest sketches serialized as 
{@link ByteBuffer}
+ */
+public class TDigestMergeSketchBufferAggregator implements BufferAggregator
+{
+  private final ColumnValueSelector selector;
+  private final int compression;
+  @GuardedBy("this")
+  private final IdentityHashMap> 
sketches = new IdentityHashMap<>();
+
+  public TDigestMergeSketchBufferAggregator(
+  ColumnValueSelector selector,
+  int compression
+  )
+  {
+this.selector = selector;
+this.compression = compression;
+  }
+
+  @Override
+  public synchronized void init(ByteBuffer buffer, int position)
 
 Review comment:
   Same here. You don't have to synchronize these 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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279898696
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
+
+  @Nonnull
+  private final String name;
+  @Nonnull
+  private final String fieldName;
+  @Nonnull
+  final Integer compression;
+  @Nonnull
+  private final byte cacheTypeId;
+
+  public static final String TYPE_NAME = "buildTDigestSketch";
+
+  @JsonCreator
+  public TDigestBuildSketchAggregatorFactory(
+  @JsonProperty("name") final String name,
+  @JsonProperty("fieldName") final String fieldName,
+  @Nullable @JsonProperty("compression") final Integer compression
+  )
+  {
+this(name, fieldName, compression, 
AggregatorUtil.TDIGEST_BUILD_SKETCH_CACHE_TYPE_ID);
+  }
+
+  TDigestBuildSketchAggregatorFactory(
+  final String name,
+  final String fieldName,
+  @Nullable final Integer compression,
+  final byte cacheTypeId
+  )
+  {
+Objects.requireNonNull(name, "Must have a valid, non-null aggregator 
name");
+this.name = name;
+Objects.requireNonNull(fieldName, "Parameter fieldName must be specified");
+this.fieldName = fieldName;
 
 Review comment:
   nit: `Objects.requireNonNull()` returns the given parameter itself if it's 
not null. So, you can simplify the above lines.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r278356437
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
 
 Review comment:
   How is it different from `TDigestBuildSketchAggregator.DEFAULT_COMPRESSION`?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279902411
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  @Nonnull
 
 Review comment:
   It seems not necessary.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279902582
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchBufferAggregator.java
 ##
 @@ -0,0 +1,125 @@
+/*
+ * 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.tdigestsketch;
+
+import com.google.common.base.Preconditions;
+import com.tdunning.math.stats.MergingDigest;
+import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
+import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+import java.nio.ByteBuffer;
+import java.util.IdentityHashMap;
+import java.util.Map;
+
+/**
+ * Aggregator that builds t-digest backed sketches using numeric values read 
from {@link ByteBuffer}
+ */
+public class TDigestBuildSketchBufferAggregator implements BufferAggregator
+{
+
+  @Nonnull
+  private final ColumnValueSelector selector;
+  @Nonnull
+  private final int compression;
+
+  @GuardedBy("this")
+  private Map> sketches = new 
IdentityHashMap<>();
 
 Review comment:
   nit: can be final.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279903582
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
+
+  @Nonnull
+  private final String name;
+  @Nonnull
+  private final String fieldName;
+  @Nonnull
+  final Integer compression;
 
 Review comment:
   Btw, you can add a `package-info` file which claims everything is non null 
by default as in 
https://github.com/apache/incubator-druid/blob/master/processing/src/main/java/org/apache/druid/segment/package-info.java.
   You don't have to add this file in this PR, but may help you in the future.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r278356040
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregator.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.tdigestsketch;
+
+import com.tdunning.math.stats.MergingDigest;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+/**
+ * Aggregator to build tDigest sketches on numeric values.
+ */
+public class TDigestBuildSketchAggregator implements Aggregator
+{
+
+  private final ColumnValueSelector selector;
+
+  // Default compression
+  static final int DEFAULT_COMPRESSION = 50;
+
+  @GuardedBy("this")
+  private MergingDigest histogram;
+
+
+  public TDigestBuildSketchAggregator(ColumnValueSelector selector, Integer 
compression)
 
 Review comment:
   Please add `@Nullable` for `compression`.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r279897774
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregator.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.tdigestsketch;
+
+import com.tdunning.math.stats.MergingDigest;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.segment.ColumnValueSelector;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+/**
+ * Aggregator to build tDigest sketches on numeric values.
+ */
+public class TDigestBuildSketchAggregator implements Aggregator
+{
+
+  private final ColumnValueSelector selector;
+
+  // Default compression
+  static final int DEFAULT_COMPRESSION = 50;
+
+  @GuardedBy("this")
+  private MergingDigest histogram;
+
+
+  public TDigestBuildSketchAggregator(ColumnValueSelector selector, Integer 
compression)
+  {
+this.selector = selector;
+if (compression != null) {
+  this.histogram = new MergingDigest(compression);
+} else {
+  this.histogram = new MergingDigest(DEFAULT_COMPRESSION);
+}
+  }
+
+  @Override
+  public synchronized void aggregate()
+  {
+if (selector.getObject() instanceof Number) {
+  histogram.add(((Number) selector.getObject()).doubleValue());
+} else {
+  throw new IAE("Expected a number, received "
++ selector.getObject()
++ " of type "
++ selector.getObject());
 
 Review comment:
   Probably this should be 
   
   ```java
   } else {
 final String msg = selector.getObject() == null
? StringUtils.format("Expected a number, but 
received null")
: StringUtils.format(
"Expected a number, but received [%s] of type 
[%s]",
selector.getObject(),
selector.getObject().getClass()
);
 throw new IAE(msg);
   }
   ```


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-druid] jihoonson commented on a change in pull request #7331: TDigest backed sketch aggregators

2019-04-30 Thread GitBox
jihoonson commented on a change in pull request #7331: TDigest backed sketch 
aggregators
URL: https://github.com/apache/incubator-druid/pull/7331#discussion_r278356375
 
 

 ##
 File path: 
extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestBuildSketchAggregatorFactory.java
 ##
 @@ -0,0 +1,269 @@
+/*
+ * 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.tdigestsketch;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.tdunning.math.stats.MergingDigest;
+import com.tdunning.math.stats.TDigest;
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import 
org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
+import org.apache.druid.query.aggregation.AggregatorUtil;
+import org.apache.druid.query.aggregation.BufferAggregator;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+import org.apache.druid.segment.ColumnSelectorFactory;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.column.ColumnCapabilities;
+import org.apache.druid.segment.column.ValueType;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Aggregation operations over the tdigest-based quantile sketch
+ * available on https://github.com/tdunning/t-digest";>github and 
described
+ * in the paper
+ * https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf";>
+ * Computing extremely accurate quantiles using t-digests.
+ * 
+ * 
+ * At the time of writing this implementation, there are two flavors of {@link 
TDigest}
+ * available - {@link MergingDigest} and {@link 
com.tdunning.math.stats.AVLTreeDigest}.
+ * This implementation uses {@link MergingDigest} since it is more suited for 
the cases
+ * when we have to merge intermediate aggregations which Druid needs to do as
+ * part of query processing.
+ */
+public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
+{
+
+  // Default compression
+  public static final int DEFAULT_COMRESSION = 100;
 
 Review comment:
   COMRESSION -> COMPRESSION


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:
us...@infra.apache.org


With regards,
Apache Git Services

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