Skip to content

numeric quantiles sketch aggregator#5002

Merged
jihoonson merged 17 commits intoapache:masterfrom
AlexanderSaydakov:quantiles-sketch-aggregator
Dec 5, 2017
Merged

numeric quantiles sketch aggregator#5002
jihoonson merged 17 commits intoapache:masterfrom
AlexanderSaydakov:quantiles-sketch-aggregator

Conversation

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor

@AlexanderSaydakov AlexanderSaydakov commented Oct 24, 2017

This is to support numeric quantiles sketch (DoublesSketch) to estimate distributions: obtain quantiles, ranks, probability mass functions (PMFs and CDFs) or histograms.

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

Is anything wrong with the build configuration?

@drcrallen
Copy link
Copy Markdown
Contributor

[INFO] Compiling 36 source files to /home/travis/build/druid-io/druid/extensions-core/datasketches/target/classes
/home/travis/build/druid-io/druid/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SynchronizedUnion.java:71: error: [ParameterPackage] Method parameter has wrong package
 public synchronized void update(byte[] data)
 ^
 (see http://errorprone.info/bugpattern/ParameterPackage)
 Did you mean 'public synchronized void update(Array data)'?
/home/travis/build/druid-io/druid/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SynchronizedUnion.java:77: error: [ParameterPackage] Method parameter has wrong package
 public synchronized void update(int[] data)
 ^
 (see http://errorprone.info/bugpattern/ParameterPackage)
 Did you mean 'public synchronized void update(Array data)'?
/home/travis/build/druid-io/druid/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SynchronizedUnion.java:83: error: [ParameterPackage] Method parameter has wrong package
 public synchronized void update(char[] chars)
 ^
 (see http://errorprone.info/bugpattern/ParameterPackage)
 Did you mean 'public synchronized void update(Array chars)'?
Note: /home/travis/build/druid-io/druid/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java uses or overrides a deprecated API.
Note: Recompile with -Xlint:deprecation for details.
Note: Some input files use unchecked or unsafe operations.
Note: Recompile with -Xlint:unchecked for details.

@leventov do you know why this one is failing?

@leventov
Copy link
Copy Markdown
Member

@AlexanderSaydakov could you please try to update plexus-compiler-javac-errorprone dependency in pom.xml to 2.8.2 and compile without @SuppressWarnings?

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

no, plexus-compiler-javac-errorprone-2.8.2 doesn't help

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

I see that error_prone_core is a few versions behind, but all versions newer than currently used 2.0.19 are not happy about something in druid/java-util

@himanshug
Copy link
Copy Markdown
Contributor

@AlexanderSaydakov currently teamcity appears to be failing due to foreach related rules , see
screen shot 2017-10-30 at 2 33 49 pm

you can find above in the "Code Inspection" tab

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

I have a couple of questions:

  1. Is the synchronization in aggregators still necessary? I looked at the Theta sketch aggregator as an example.There a SynchronizedUnion was used for many years. I was shown this: https://github.com/druid-io/druid/pull/1027/files, which seems to suggest that Druid takes care of synchronization. If so, I can remove synchronization in this aggregator.

  2. I hard-coded cache IDs in factories and post-aggregators. Perhaps they should be kept in a central registry somewhere instead.

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Nov 1, 2017

@AlexanderSaydakov

  1. Druid synchronizes calls to "aggregate", but it can call "aggregate" and "get" simultaneously. So you do need to make sure that this combination is thread-safe. I'm not too familiar with the theta sketch code, but that could be the reason that it uses synchronization.

  2. For core extensions you can put the cache key ID in AggregatorUtil (for aggregator factories) and PostAggregatorIds (for post aggregators). Theta sketches already do that.

return sketch;
}
catch (NumberFormatException e) {
// Log.info("Expected Double. Got string with value " +
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider adding Log.debug saying expected was Double but found base64Encoded string.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why debug? should we use log.error? this should never happen, but it is not fatal either

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

after a discussion with @akashdw we believe that it is better to throw an exception here

@Override
public void deserializeColumn(final ByteBuffer buffer, final ColumnBuilder builder)
{
final GenericIndexed<DoublesSketch> column = GenericIndexed.read(buffer, strategy);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use GenericIndexed.read(buffer, strategy, builder.getFileMapper()) similar to thetaSketch to enable largeColumns. Also Override getSerializer method as

  @Override
  public GenericColumnSerializer getSerializer(IOPeon peon, String column)
  {
    return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
  }

import io.druid.query.aggregation.Aggregator;
import io.druid.segment.ColumnValueSelector;

public class DoublesSketchDoubleAggregator implements Aggregator
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider renaming this class to DoublesQuantileSketchAggregator

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure about adding Quantiles into the class names. This is implied by the package name, and class names are long already. Regarding the second "Double", I believe the intention was to say that this aggregator works on double values as input (as opposed to sketches). I propose to rename it to BuildAggregator, and rename the other one to MergeAggregator (or leave Combining, but switch the words around for consistency)

import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;

public class DoublesSketchDoubleBufferAggregator implements BufferAggregator
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider renaming this class to DoublesQuantileSketchBufferAggregator

private final IdentityHashMap<ByteBuffer, WritableMemory> memCache = new IdentityHashMap<>();
private final IdentityHashMap<ByteBuffer, Int2ObjectMap<UpdateDoublesSketch>> sketches = new IdentityHashMap<>();

public DoublesSketchDoubleBufferAggregator(final ColumnValueSelector<Double> valueSelector, final int size,
Copy link
Copy Markdown
Contributor

@akashdw akashdw Nov 2, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please add a comment explaining sketch can grow on-heap also, also explain what happens in relocation when sketch grows on-heap.

import io.druid.initialization.DruidModule;
import io.druid.segment.serde.ComplexMetrics;

public class DoublesSketchModule implements DruidModule
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider renaming it to DoublesQuantileSketchModule or QuantilesDoublesSketch

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

I believe that I addressed all reviewers' suggestions so far. How can we proceed? Thank you.

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AlexanderSaydakov nice work! I left some comments. Please consider them.

private final String name;
private final String fieldName;
private final int k;
private final byte cacheTypeId;
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks this is always AggregatorUtil.QUANTILES_DOUBLES_SKETCH_BUILD_CACHE_TYPE_ID and don't have to be a member variable.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

no, DoublesSketchMergeAggregatorFactory overrides it

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah ok.

if (metricFactory.getColumnCapabilities(fieldName) != null
&& ValueType.isNumeric(metricFactory.getColumnCapabilities(fieldName).getType())) {
final ColumnValueSelector<Double> valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector == null) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm curious when this can be null. I couldn't find it.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I it seems to me that this can happen if a non-existent field is mentioned as the input. I am not convinced myself that we really need this special no-op aggregator. This is how it was done in the Theta sketch aggregator. I think that it is not worth optimizing an erroneous query.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In Druid, NilColumnValueSelector is returned for non-existent input fields. I think it's worthwhile to optimize, but we already have NoopAggregator and NoopBufferAggregator, and you can use them instead of adding new ones.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe I tested this case and got a null for selector. Regarding the dummy aggregators, I believe the idea was to always return a sketch, even an empty one. The custom dummy aggregators do just that.

Copy link
Copy Markdown
Contributor

@akashdw akashdw Nov 21, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jihoonson sketches post aggs expect a sketch object, not sure how a sketches post agg will behave if we use NoopAggregator and NoopBufferAggregator. This check was added considering you can have a sketch field in some segments(say I added a new sketch column from today onwards) but past data does not have that field.

Copy link
Copy Markdown
Contributor

@jihoonson jihoonson Nov 22, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If it's nullable, every other aggregatorFactory should consider it too, but they don't. As I said, if a segment doesn't have a specified column, NilColumnValueSelector is returned instead of null.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we should check for NilColumnValueSelector but will continue to return DoublesSketchNoOpAggregator as quantile postAggregators expect sketch values.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You mean returning DoublesSketchNoOpAggregator for NilColumnValueSelector? It makes sense to me.

if (selector == null) {
return new DoublesSketchNoOpAggregator();
}
return new DoublesSketchMergeAggregator(selector, k);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder why DoublesSketchAggregatorFactory is able to return the DoublesSketchMergeAggregator. I guess DoublesSketchMergeAggregatorFactory is to get mergeAggregator and DoublesSketchAggregatorFactory is for a plain aggregator.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a common entry point associated with the type name. If the input field is a numeric field, then so-called "build" aggregator is used to build sketches. Otherwise, we assume that the input contains sketches to merge.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Druid calls combine() or gets combiningFactories by calling getCombiningFactory() for merging aggregates. Calling DoublesSketchAggregatorFactory.factorize() should not happen for merging aggregates.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps, I did not make myself clear. By merging sketches I meant such an aggregation in which the input field contains sketches as opposed to the raw values. I used to think about this as having two modes: building sketches (from raw input) and merging sketches. The closest thing to this is Theta sketch aggregator, with a twist that it cannot autodetect the input type (here the input is numeric, but there it can be of almost any type).

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, sorry I misunderstood. Sounds good.

if (selector == null) {
return new DoublesSketchNoOpBufferAggregator();
}
return new DoublesSketchMergeBufferAggregator(selector, k, getMaxIntermediateSize());
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similar comment here. Better to consider only a plain buffer aggregator.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Plain? Here is the same selection of an aggregator based on the input type, just for the buffered case.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, I used 'plain' for non-mergingAggregatorFactory. Similar to the above comment, Calling DoublesSketchAggregatorFactory.factorizeBuffered() should not happen for merging aggregates.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

same as above

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good.

new DoublesSketchAggregatorFactory(
fieldName,
fieldName,
k));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please break this line like

    k
  )
);

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am using the Druid formatter for Eclipse, which is a part of this repo (eclipse_formatting.xml)

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, sorry but it can't catch everything.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok, will change

} else if (serializedSketch instanceof DoublesSketch) {
return (DoublesSketch) serializedSketch;
}
throw new IllegalStateException(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We usually use ISE() because it supports string format.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

return splitPoints;
}

// comparing histograms doesn't make much sense, so this comparator pretends that everything is equal
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's better to throw an exception.
BTW, ApproximateHistogramPostAggregator returns a comparator comparing histogram's count. I'm not sure the same approach is possible for this class.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I was told that throwing here would be a bad idea. And comparing by total count doesn't make much sense. I think we can just do nothing.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, would you tell me why it's a bad idea? I think it's better because getComparator() is used for limitting the number of results like in TopN query, and users can get completely different results even though they run the same query multiple times.
Well, a better way is to check LimitSpec is specified with DoublesSketchToHistogramPostAggregator together before query execution, but I think it's beyond the scope of this PR.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I heard that there are some systems like dashboard frontends and such, which generate queries automatically, and always use some ordering. We don't want to break compatibility with such systems. And providing some ordering, which doesn't make much sense, also sounds like a bad idea.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree on that providing some ordering doesn't make sense.
But, I think there is no compatibility issue because this is a new feature and the systems generating Druid queries can change their logic to not include ordering for this new postAggregator.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this particular aggregator is a new feature, but we don't want it to have some properties, which would be obstacles to integration with other systems like Hive or Pivot. These systems just assume some ordering by default. I don't think we are in a position to dictate the rules.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jihoonson not comparable does not means its an error, IMO it means we can expect random ordered set instead of sorted ordered set.
Some clients (including pivot) add a default limit spec with order by on the selected metric, not sure if we want to fail the request or return a random ordered set ?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What I'm concerned with is, when people get a result of a query with ordering and a limit, they expect an ordered top-n result. But we cannot guarantee that the result is ordered in some order with this post aggregator, so I think this is an error of unsupported feature.

I understand what you guys are concerned with, but I think we can help the clients to do the right thing by like explicitly specifying that ordering by this post aggregator is not supported in the release note.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this particular aggregator is a new feature, but we don't want it to have some properties, which would be obstacles to integration with other systems like Hive or Pivot.

I don't think this is an obstacle to other ecosystems of Druid. This post aggregator is newly added in this patch, and they can add some special logic to handle this post aggregator when they decide to support it.

These systems just assume some ordering by default. I don't think we are in a position to dictate the rules.

I'm not sure why you think so. Every system evolves as time goes by, and their ecosystems should be evolved together. Assuming every result can be ordered may be true so far, but it becomes wrong with this post aggregator. There is a recent example similar to this case. We recently added support for numeric dimensions. Druid's ecosystems can assume that every dimension has the string type before, but now it's a wrong assumption.

return sketch.getQuantiles(fractions);
}

// comparing arrays of quantiles doesn't make much sense, so this comparator
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same comment here.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see how comparing these arrays would make any sense at all.

return sketch.toString();
}

// comparing sketch summaries doesn't make much sense, so this comparator
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same comment here.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Again, I don't see how comparing sketch summaries can be helpful.


import com.yahoo.sketches.quantiles.UpdateDoublesSketch;

public class GenerateTestData
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, do you think we may need to modify test data someday? It should be rare because we should also fix the expected results in unit tests accordingly.
If you think it's needed, please add some comments on this class to let others know this is used for generating test data for DoublesSketchAggregatorTest.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

@drcrallen
Copy link
Copy Markdown
Contributor

@AlexanderSaydakov please comment when this is ready for another round of review. This looks like a very useful feature.

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

I believe it is ready. I addressed the last two points: NilColumnValueSelector instead of null and throwing exceptions instead of providing dummy comparators

@jihoonson
Copy link
Copy Markdown
Contributor

@AlexanderSaydakov thanks for the update. I'll finish my review soon.

@jihoonson
Copy link
Copy Markdown
Contributor

@AlexanderSaydakov the latest change looks good to me. Would you please add some documents for these new aggregators? Please see https://github.com/druid-io/druid/blob/master/docs/content/development/extensions-core/datasketches-aggregators.md as an example. Also please add this new extension to the core extensions list here.

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

This is not a new extension, but a part of the existing datasketches extension. We need to rewrite the document so that it would no longer assume datasketches means Theta sketch aggregator for approximate count-distinct. We are going to have more soon: new HLL sketch aggregator and Tuple sketch aggregator (ArrayOfDoubles). I have them almost ready, just need to brush up based on this review of Quantiles sketch aggregator.

@jihoonson
Copy link
Copy Markdown
Contributor

You are right. Do you want to rewrite the document at once after merging all your patches? It sounds good to me.

@jihoonson
Copy link
Copy Markdown
Contributor

@drcrallen @himanshug @leventov @gianm @akashdw do you have further comments? If not, I'm going to merge this PR.

@jihoonson jihoonson merged commit 45f91a2 into apache:master Dec 5, 2017
@jon-wei
Copy link
Copy Markdown
Contributor

jon-wei commented Dec 6, 2017

looks like there's a conflict with this PR that removed the IOPeon class: #4762

@jihoonson
Copy link
Copy Markdown
Contributor

@jon-wei thanks. Raised a PR.

@himanshug
Copy link
Copy Markdown
Contributor

@jihoonson thanks for reviewing it ... LGTM . We actually have had this code internally reviewed and used a bit beforehand.

@jihoonson
Copy link
Copy Markdown
Contributor

@himanshug good to know. This patch is awesome. I'm looking forward to the follow-up patches.

@jon-wei jon-wei added this to the 0.12.0 milestone Jan 5, 2018
@jon-wei
Copy link
Copy Markdown
Contributor

jon-wei commented Jan 5, 2018

@AlexanderSaydakov Can you provide a doc update for this patch?

{
if (metricFactory.getColumnCapabilities(fieldName) != null
&& ValueType.isNumeric(metricFactory.getColumnCapabilities(fieldName).getType())) {
final ColumnValueSelector<Double> selector = metricFactory.makeColumnValueSelector(fieldName);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This variable should have type BaseDoubleColumnValueSelector, as well as all the way down.

}
return new DoublesSketchBuildAggregator(selector, k);
}
final ColumnValueSelector<DoublesSketch> selector = metricFactory.makeColumnValueSelector(fieldName);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should use BaseObjectColumnValueSelector<DoublesSketch>, as well as all the way down

return (DoublesSketch) serializedSketch;
}
throw new ISE(
"Object is not of a type that can be deserialized to a quantiles DoublsSketch: "
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use String.format format instead of concat

@kyleboyle
Copy link
Copy Markdown

Would someone be able to compare/contrast using this doubles sketch aggregator versus the existing approxHistogramFold aggregator? thanks

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

AlexanderSaydakov commented Jan 30, 2018 via email

@jon-wei
Copy link
Copy Markdown
Contributor

jon-wei commented Jan 30, 2018

@AlexanderSaydakov can you provide docs for this feature? I'd like to provide a link to them in the 0.12.0 release notes

@AlexanderSaydakov
Copy link
Copy Markdown
Contributor Author

AlexanderSaydakov commented Jan 30, 2018 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants