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

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

                Author: ASF GitHub Bot
            Created on: 15/Aug/19 17:21
            Start Date: 15/Aug/19 17:21
    Worklog Time Spent: 10m 
      Work Description: robinyqiu commented on pull request #9144: [BEAM-7013] 
Integrating ZetaSketch's HLL++ algorithm with Beam
URL: https://github.com/apache/beam/pull/9144#discussion_r314412303
 
 

 ##########
 File path: 
sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/HllCountMergePartialFn.java
 ##########
 @@ -44,109 +37,55 @@ private HllCountMergePartialFn() {}
     return new HllCountMergePartialFn();
   }
 
-  /**
-   * Accumulator for the {@link HllCount.MergePartial} combiner. Cannot use 
{@link
-   * HyperLogLogPlusPlus} directly because we need an "identity element" for 
the {@code merge}
-   * operation, and we are not able to create such "identity element" of type 
{@link
-   * HyperLogLogPlusPlus} without knowing the sketch precision and value type.
-   */
-  static final class HyperLogLogPlusPlusWrapper<T> {
-    @Nullable private HyperLogLogPlusPlus<T> hll;
-
-    private HyperLogLogPlusPlusWrapper(@Nullable HyperLogLogPlusPlus<T> hll) {
-      this.hll = hll;
-    }
-  }
-
-  /**
-   * Coder for the {@link HyperLogLogPlusPlusWrapper} class with generic type 
parameter {@code T}.
-   */
-  private static final class HyperLogLogPlusPlusWrapperCoder<T>
-      extends AtomicCoder<HyperLogLogPlusPlusWrapper<T>> {
-
-    private static final HyperLogLogPlusPlusWrapperCoder<?> INSTANCE =
-        new HyperLogLogPlusPlusWrapperCoder<>();
-
-    private final Coder<HyperLogLogPlusPlus<T>> nullableHllCoder =
-        NullableCoder.of(HyperLogLogPlusPlusCoder.of());
-
-    // Generic singleton factory pattern; the coder works for all 
HyperLogLogPlusPlusWrapper objects
-    // at runtime regardless of type T
-    @SuppressWarnings("unchecked")
-    static <T> HyperLogLogPlusPlusWrapperCoder<T> of() {
-      return (HyperLogLogPlusPlusWrapperCoder<T>) INSTANCE;
-    }
-
-    @Override
-    public void encode(HyperLogLogPlusPlusWrapper<T> value, OutputStream 
outStream)
-        throws IOException {
-      nullableHllCoder.encode(value.hll, outStream);
-    }
-
-    @Override
-    public HyperLogLogPlusPlusWrapper<T> decode(InputStream inStream) throws 
IOException {
-      return new 
HyperLogLogPlusPlusWrapper<T>(nullableHllCoder.decode(inStream));
-    }
-
-    @Override
-    public void verifyDeterministic() throws NonDeterministicException {
-      nullableHllCoder.verifyDeterministic();
-    }
-  }
-
   @Override
-  public Coder<HyperLogLogPlusPlusWrapper<T>> getAccumulatorCoder(
+  public Coder<HyperLogLogPlusPlus<T>> getAccumulatorCoder(
       CoderRegistry registry, Coder<byte[]> inputCoder) {
-    return HyperLogLogPlusPlusWrapperCoder.of();
+    return NullableCoder.of(HyperLogLogPlusPlusCoder.of());
   }
 
   @Override
-  public HyperLogLogPlusPlusWrapper<T> createAccumulator() {
-    return new HyperLogLogPlusPlusWrapper<>(null);
+  public HyperLogLogPlusPlus<T> createAccumulator() {
+    return null;
   }
 
   @Override
-  public HyperLogLogPlusPlusWrapper<T> addInput(
-      HyperLogLogPlusPlusWrapper<T> accumulator, byte[] input) {
-    if (accumulator.hll == null) {
+  public HyperLogLogPlusPlus<T> addInput(HyperLogLogPlusPlus<T> accumulator, 
byte[] input) {
+    if (accumulator == null) {
       @SuppressWarnings("unchecked")
-      HyperLogLogPlusPlus<T> hll = (HyperLogLogPlusPlus<T>) 
HyperLogLogPlusPlus.forProto(input);
-      accumulator.hll = hll;
+      HyperLogLogPlusPlus<T> result = (HyperLogLogPlusPlus<T>) 
HyperLogLogPlusPlus.forProto(input);
+      return result;
     } else {
-      accumulator.hll.merge(input);
+      accumulator.merge(input);
+      return accumulator;
     }
-    return accumulator;
   }
 
   @Override
-  public HyperLogLogPlusPlusWrapper<T> mergeAccumulators(
-      Iterable<HyperLogLogPlusPlusWrapper<T>> accumulators) {
-    HyperLogLogPlusPlusWrapper<T> merged = createAccumulator();
-    for (HyperLogLogPlusPlusWrapper<T> accumulator : accumulators) {
-      if (accumulator.hll == null) {
+  public HyperLogLogPlusPlus<T> 
mergeAccumulators(Iterable<HyperLogLogPlusPlus<T>> accumulators) {
+    HyperLogLogPlusPlus<T> merged = createAccumulator();
 
 Review comment:
   They are functionally equivalent. Calls `createAccumulator()` because 
basically every implementation of `mergeAccumulators()` will start with that 
(so it's like a paradigm?). Would leave it as it is.
 
----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 295598)
    Time Spent: 19h 10m  (was: 19h)

> A new count distinct transform based on BigQuery compatible HyperLogLog++ 
> implementation
> ----------------------------------------------------------------------------------------
>
>                 Key: BEAM-7013
>                 URL: https://issues.apache.org/jira/browse/BEAM-7013
>             Project: Beam
>          Issue Type: New Feature
>          Components: extensions-java-sketching, sdk-java-core
>            Reporter: Yueyang Qiu
>            Assignee: Yueyang Qiu
>            Priority: Major
>             Fix For: 2.16.0
>
>          Time Spent: 19h 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to