You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/10/17 00:31:00 UTC

[jira] [Work logged] (BEAM-12393) Beam SQL support for HLL count

     [ https://issues.apache.org/jira/browse/BEAM-12393?focusedWorklogId=665762&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-665762 ]

ASF GitHub Bot logged work on BEAM-12393:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 17/Oct/21 00:30
            Start Date: 17/Oct/21 00:30
    Worklog Time Spent: 10m 
      Work Description: brachi-wernick commented on a change in pull request #15728:
URL: https://github.com/apache/beam/pull/15728#discussion_r730304342



##########
File path: sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/HllCount.java
##########
@@ -279,6 +279,10 @@ private Builder(HllCountInitFn<InputT, ?> initFn) {
       public <K> Combine.PerKey<K, InputT, byte[]> perKey() {
         return Combine.perKey(initFn);
       }
+
+      public HllCountInitFn<InputT, ?> asUdaf() {

Review comment:
       Why is it better to return `Combine.CombineFn`? this `builder` is specific for `HllCount`?
   Also it is problematic to return `Combine.CombineFn<InputT, HyperLogLogPlusPlus<?>, byte[]>` because the `?` makes issues to compile:
   ```
    error: incompatible types: HllCountInitFn<InputT,CAP#1> cannot be converted to CombineFn<InputT,HyperLogLogPlusPlus<?>,byte[]>
           return initFn;
                  ^
     where InputT is a type-variable:
       InputT extends @org.checkerframework.checker.nullness.qual.Nullable Object declared in class Builder
     where CAP#1 is a fresh type-variable:
       CAP#1 extends Object from capture of ?
   1 error
   ```
   
   if I remove the `?` it also failed with
   ```
     missing type arguments for generic class HyperLogLogPlusPlus<V>
   ```

##########
File path: sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/HllCount.java
##########
@@ -279,6 +279,10 @@ private Builder(HllCountInitFn<InputT, ?> initFn) {
       public <K> Combine.PerKey<K, InputT, byte[]> perKey() {
         return Combine.perKey(initFn);
       }
+
+      public HllCountInitFn<InputT, ?> asUdaf() {

Review comment:
       Why is it better to return `Combine.CombineFn`? this `builder` is specific for `HllCount`.
   
   Also it is problematic to return `Combine.CombineFn<InputT, HyperLogLogPlusPlus<?>, byte[]>` because the `?` makes issues to compile:
   ```
    error: incompatible types: HllCountInitFn<InputT,CAP#1> cannot be converted to CombineFn<InputT,HyperLogLogPlusPlus<?>,byte[]>
           return initFn;
                  ^
     where InputT is a type-variable:
       InputT extends @org.checkerframework.checker.nullness.qual.Nullable Object declared in class Builder
     where CAP#1 is a fresh type-variable:
       CAP#1 extends Object from capture of ?
   1 error
   ```
   
   if I remove the `?` it also failed with
   ```
     missing type arguments for generic class HyperLogLogPlusPlus<V>
   ```

##########
File path: sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/ApproximateCountDistinct.java
##########
@@ -99,6 +99,10 @@
         .build();
   }
 
+  public static <T> HllCountInitFn<T, ?> getUdaf(TypeDescriptor<T> input) {

Review comment:
       yes works when putting `?` instead of `HyperLogLogPlusPlus`, less strict but fine enough :)
   commit 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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

            Worklog Id:     (was: 665762)
    Remaining Estimate: 0h
            Time Spent: 10m

> Beam SQL support for HLL count
> ------------------------------
>
>                 Key: BEAM-12393
>                 URL: https://issues.apache.org/jira/browse/BEAM-12393
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql, extensions-java-sketching
>            Reporter: Brachi Packter
>            Priority: P3
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> There is support for HLL sketch in Ptransform:
>  
> {code:java}
> p.apply("Int", Create.of(ints)).apply("IntHLL", ApproximateCountDistinct.globally()
> .withPercision(PRECISION));{code}
>  
> or 
>  
> {code:java}
> PCollection<KV<Integer, Long>> result =
> p.apply("Long", Create.of(longs)).apply("LongHLL", ApproximateCountDistinct.perKey());
>  
> {code}
> But, no support for beam sql. 
> We can't initiate it to be used  in SqlTransform (even the combiner: HllCountMergePartialFn exists)
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)