You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by gi...@apache.org on 2019/06/27 22:10:04 UTC

[incubator-druid] branch master updated: theta sketch to string post agg (#7937)

This is an automated email from the ASF dual-hosted git repository.

gian pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new f38a62e  theta sketch to string post agg (#7937)
f38a62e is described below

commit f38a62e9492b158431e1c1a110f6ced6aadb17ac
Author: Alexander Saydakov <13...@users.noreply.github.com>
AuthorDate: Thu Jun 27 15:09:57 2019 -0700

    theta sketch to string post agg (#7937)
---
 .../extensions-core/datasketches-theta.md          |  12 ++
 .../datasketches/theta/SketchModule.java           |   5 +-
 .../theta/SketchToStringPostAggregator.java        | 136 +++++++++++++++++++++
 .../theta/SketchToStringPostAggregatorTest.java    |  55 +++++++++
 .../query/aggregation/post/PostAggregatorIds.java  |   3 +-
 5 files changed, 207 insertions(+), 4 deletions(-)

diff --git a/docs/content/development/extensions-core/datasketches-theta.md b/docs/content/development/extensions-core/datasketches-theta.md
index 5a2d1af..4112a71 100644
--- a/docs/content/development/extensions-core/datasketches-theta.md
+++ b/docs/content/development/extensions-core/datasketches-theta.md
@@ -78,6 +78,18 @@ druid.extensions.loadList=["druid-datasketches"]
 }
 ```
 
+#### Sketch Summary
+
+This returns a summary of the sketch that can be used for debugging. This is the result of calling toString() method.
+
+```json
+{
+  "type"  : "thetaSketchToString",
+  "name": <output name>,
+  "field"  : <post aggregator that refers to a Theta sketch (fieldAccess or another post aggregator)>
+}
+```
+
 ### Examples
 
 Assuming, you have a dataset containing (timestamp, product, user_id). You want to answer questions like
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java
index 4af6e94..0a6b732 100644
--- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java
@@ -41,8 +41,8 @@ public class SketchModule implements DruidModule
 
   public static final String THETA_SKETCH_ESTIMATE_POST_AGG = "thetaSketchEstimate";
   public static final String THETA_SKETCH_SET_OP_POST_AGG = "thetaSketchSetOp";
-  
   public static final String THETA_SKETCH_CONSTANT_POST_AGG = "thetaSketchConstant";
+  public static final String THETA_SKETCH_TO_STRING_POST_AGG = "thetaSketchToString";
 
   @Override
   public void configure(Binder binder)
@@ -60,7 +60,8 @@ public class SketchModule implements DruidModule
                 new NamedType(SketchMergeAggregatorFactory.class, THETA_SKETCH),
                 new NamedType(SketchEstimatePostAggregator.class, THETA_SKETCH_ESTIMATE_POST_AGG),
                 new NamedType(SketchSetPostAggregator.class, THETA_SKETCH_SET_OP_POST_AGG),
-                new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG)
+                new NamedType(SketchConstantPostAggregator.class, THETA_SKETCH_CONSTANT_POST_AGG),
+                new NamedType(SketchToStringPostAggregator.class, THETA_SKETCH_TO_STRING_POST_AGG)
             )
             .addSerializer(SketchHolder.class, new SketchHolderJsonSerializer())
     );
diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.java
new file mode 100644
index 0000000..b34b239
--- /dev/null
+++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.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.datasketches.theta;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.aggregation.post.PostAggregatorIds;
+import org.apache.druid.query.cache.CacheKeyBuilder;
+
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Returns a human-readable summary of a given Theta sketch.
+ * This is a string returned by toString() method of the sketch.
+ * This can be useful for debugging.
+ */
+public class SketchToStringPostAggregator implements PostAggregator
+{
+
+  private final String name;
+  private final PostAggregator field;
+
+  @JsonCreator
+  public SketchToStringPostAggregator(
+      @JsonProperty("name") final String name,
+      @JsonProperty("field") final PostAggregator field
+  )
+  {
+    this.name = name;
+    this.field = field;
+  }
+
+  @Override
+  public Set<String> getDependentFields()
+  {
+    return field.getDependentFields();
+  }
+
+  @Override
+  public Comparator<String> getComparator()
+  {
+    return Comparator.nullsFirst(Comparator.naturalOrder());
+  }
+
+  @Override
+  public Object compute(final Map<String, Object> combinedAggregators)
+  {
+    final SketchHolder holder = (SketchHolder) field.compute(combinedAggregators);
+    return holder.getSketch().toString();
+  }
+
+  @Override
+  @JsonProperty
+  public String getName()
+  {
+    return name;
+  }
+
+  @Override
+  public PostAggregator decorate(final Map<String, AggregatorFactory> aggregators)
+  {
+    return this;
+  }
+
+  @JsonProperty
+  public PostAggregator getField()
+  {
+    return field;
+  }
+
+  @Override
+  public String toString()
+  {
+    return getClass().getSimpleName() + "{" +
+        "name='" + name + '\'' +
+        ", field=" + field +
+        "}";
+  }
+
+  @Override
+  public boolean equals(final Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof SketchToStringPostAggregator)) {
+      return false;
+    }
+
+    final SketchToStringPostAggregator that = (SketchToStringPostAggregator) o;
+
+    if (!name.equals(that.name)) {
+      return false;
+    }
+
+    return field.equals(that.field);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(name, field);
+  }
+
+  @Override
+  public byte[] getCacheKey()
+  {
+    return new CacheKeyBuilder(PostAggregatorIds.THETA_SKETCH_TO_STRING)
+        .appendString(name)
+        .appendCacheable(field)
+        .build();
+  }
+
+}
diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregatorTest.java
new file mode 100644
index 0000000..46309e3
--- /dev/null
+++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregatorTest.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.aggregation.datasketches.theta;
+
+import org.apache.druid.query.aggregation.Aggregator;
+import org.apache.druid.query.aggregation.PostAggregator;
+import org.apache.druid.query.aggregation.TestObjectColumnSelector;
+import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SketchToStringPostAggregatorTest
+{
+
+  @Test
+  public void test()
+  {
+    // not going to iterate over the selector since getting a summary of an empty sketch is sufficient
+    final TestObjectColumnSelector selector = new TestObjectColumnSelector(new Object[0]);
+    final Aggregator agg = new SketchAggregator(selector, 4096);
+
+    final Map<String, Object> fields = new HashMap<>();
+    fields.put("sketch", agg.get());
+
+    final PostAggregator postAgg = new SketchToStringPostAggregator(
+        "summary",
+        new FieldAccessPostAggregator("field", "sketch")
+    );
+
+    final String summary = (String) postAgg.compute(fields);
+    Assert.assertNotNull(summary);
+    Assert.assertTrue(summary.contains("SUMMARY"));
+  }
+
+}
diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java
index 537a180c..52d693d 100644
--- a/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java
+++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/PostAggregatorIds.java
@@ -48,8 +48,7 @@ public class PostAggregatorIds
   public static final byte MOMENTS_SKETCH_TO_QUANTILES_CACHE_TYPE_ID = 24;
   public static final byte MOMENTS_SKETCH_TO_MIN_CACHE_TYPE_ID = 25;
   public static final byte MOMENTS_SKETCH_TO_MAX_CACHE_TYPE_ID = 26;
-
-  // Datasketches Quantiles sketch aggregator (part 2)
   public static final byte QUANTILES_DOUBLES_SKETCH_TO_RANK_CACHE_TYPE_ID = 27;
   public static final byte QUANTILES_DOUBLES_SKETCH_TO_CDF_CACHE_TYPE_ID = 28;
+  public static final byte THETA_SKETCH_TO_STRING = 29;
 }


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