On Wed, Nov 13, 2019 at 9:58 AM Ahmet Altay <[email protected]> wrote:

> Thank you for writing this summary.
>
> On Tue, Nov 12, 2019 at 6:35 PM Reza Rokni <[email protected]> wrote:
>
>> Hi everyone;
>>
>> TL/DR : Discussion on Beam's various Approximate Distinct Count
>> algorithms.
>>
>> Today there are several options for Approximate Algorithms in Apache Beam
>> 2.16 with HLLCount being the most recently added. Would like to canvas
>> opinions here on the possibility of rationalizing these API's by removing
>> obsolete / less efficient implementations.
>> The current situation:
>>
>> There are three options available to users: ApproximateUnique.java
>> <https://github.com/apache/beam/blob/058f5bc1d03e207ce5c8b9d221a231f390b088f3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java>,
>> ApproximateDistinct.java
>> <https://github.com/apache/beam/blob/058f5bc1d03e207ce5c8b9d221a231f390b088f3/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java>
>> and HllCount.java
>> <https://beam.apache.org/releases/javadoc/2.16.0/org/apache/beam/sdk/extensions/zetasketch/HllCount.html>.
>> A quick summary of these API's as I understand them:
>>
>> HllCount.java
>> <https://beam.apache.org/releases/javadoc/2.16.0/org/apache/beam/sdk/extensions/zetasketch/HllCount.html>:
>> Marked as @Experimental
>>
>> PTransforms to compute HyperLogLogPlusPlus (HLL++) sketches on data
>> streams based on the ZetaSketch <https://github.com/google/zetasketch>
>> implementation.Detailed design of this class, see
>> https://s.apache.org/hll-in-beam.
>>
>> ApproximateUnique.java
>> <https://github.com/apache/beam/blob/058f5bc1d03e207ce5c8b9d221a231f390b088f3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java>:
>> Not Marked with experimental
>>
>> This API does not expose the ability to create sketches so it's not
>> suitable for the OLAP use case that HLL++ is geared towards (do
>> pre-aggregation into sketches to allow interactive query speeds). It's also
>> less precise for the same amount of memory used: the error bounds in the
>> doc comments give :
>>
>> /* The error is about
>>
>> {@code 2 * / sqrt(sampleSize)},) */
>>
>> Compared to the default HLLCount sketch size, its error is 10X larger
>> than the HLL++ error.
>>
>
> FWIW, There is a python implementation only for this version:
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/stats.py#L38
>
>
>
>> ApproximateDistinct.java
>> <https://github.com/apache/beam/blob/058f5bc1d03e207ce5c8b9d221a231f390b088f3/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java>
>> Marked with @Experimental
>>
>> This is a re-implementation of the HLL++ algorithm, based on the paper
>> published in 2013. It is exposing sketches via a HyperLogLogPlusCoder. We
>> have not run any benchmarks to compare this implementation compared to the
>> HLLCount and we need to be careful to ensure that if we were to change any
>> of these API's that the binary format of the sketches should never change,
>> there could be users who have stored previous sketches using
>> ApproximateDistinct and it will be important to try and ensure they do not
>> have a bad experience.
>>
>>
>> Proposals:
>>
>> There are two classes of users expected for these algorithms:
>>
>> 1) Users who simply use the transform to estimate the size of their data
>> set in Beam
>>
>> 2) Users who want to create sketches and store them, either for
>> interoperability with other systems, or as features to be used in further
>> data processing.
>>
>>
>>
>> For use case 1, it is possible to make use of naming which does not
>> expose the implementation, however for use case 2 it is important for the
>> implementation to be explicit as sketches produced with one implementation
>> will not work with other implementations.
>>
>> ApproximateUnique.java
>> <https://github.com/apache/beam/blob/058f5bc1d03e207ce5c8b9d221a231f390b088f3/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java>
>> :
>>
>> This one does not provide sketches and based on the notes above, is not
>> as efficient as HLLCount. However it does have a very searchable name and
>> is likely to be the one users will gravitate to when searching for
>> Approximate unique algorithms but do not need the capabilities of sketches.
>>
>> Ideally we should think about switching the implementation of this
>> transform to wrap HLLCount. However this could mean changing things in a
>> way which is not transparent to the end developer.  Although as a result of
>> the change they would get a better implementation for free on an upgrade :-)
>>
>> Another option would be to mark this transform as @Deprecated and create
>> a new transform ApproximateCountDistinct which would wrap HLLCount. The
>> name is also much clearer.
>>
>
> Marking it deprecated instead of changing its implementation will probably
> create a less surprising experience for the users.
>
>
>>
>> ApproximateDistinct.java
>> <https://github.com/apache/beam/blob/058f5bc1d03e207ce5c8b9d221a231f390b088f3/sdks/java/extensions/sketching/src/main/java/org/apache/beam/sdk/extensions/sketching/ApproximateDistinct.java>
>>
>> This transform does generate sketches as output and given its marked as
>> @Experimental, one option we would have is to create a name which includes
>> the algorithm implementation details, for example
>> ApproximateCountDistinctClearSpring.
>>
>
> Can we remove this, if it is both experimental and providing the same
> utility as HllCount? Is the concern that users might have stored sketches?
>

I think this is the sort of thing that has been experimental forever, and
therefore not experimental (e.g. the entire triggering API is experimental
as are all our file-based sinks). I think that many users use this, and
probably store the state implicitly in streaming pipelines.


>
>>
>>
>> HllCount.java
>> <https://beam.apache.org/releases/javadoc/2.16.0/org/apache/beam/sdk/extensions/zetasketch/HllCount.html>
>> .
>>
>> Again we have a few options here, as the name does not include search
>> words like Approximate, we can create a wrapper which has a name similar to
>> ApproximateCountDistinctZetaSketch.
>>
>
>>
>> Thoughts?
>>
>>
>>
>> Cheers
>>
>>
>>
>> Reza
>>
>>
>>
>>
>>
>> --
>>
>> This email may be confidential and privileged. If you received this
>> communication by mistake, please don't forward it to anyone else, please
>> erase all copies and attachments, and please let me know that it has gone
>> to the wrong person.
>>
>> The above terms reflect a potential business arrangement, are provided
>> solely as a basis for further discussion, and are not intended to be and do
>> not constitute a legally binding obligation. No legally binding obligations
>> will be created, implied, or inferred until an agreement in final form is
>> executed in writing by all parties involved.
>>
>

Reply via email to