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

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

                Author: ASF GitHub Bot
            Created on: 02/Aug/19 17:59
            Start Date: 02/Aug/19 17:59
    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_r310238533
 
 

 ##########
 File path: 
sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/HllCountInitFn.java
 ##########
 @@ -0,0 +1,152 @@
+/*
+ * 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.beam.sdk.extensions.zetasketch;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import com.google.zetasketch.HyperLogLogPlusPlus;
+import com.google.zetasketch.shaded.com.google.protobuf.ByteString;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderRegistry;
+import org.apache.beam.sdk.transforms.Combine;
+
+/**
+ * {@link Combine.CombineFn} for the {@link HllCount.Init} combiner.
+ *
+ * @param <InputT> type of input values to the function
+ * @param <HllT> type of the HLL++ sketch to compute
+ */
+abstract class HllCountInitFn<InputT, HllT>
+    extends Combine.CombineFn<InputT, HyperLogLogPlusPlus<HllT>, byte[]> {
+
+  private int precision;
+
+  private HllCountInitFn() {
+    setPrecision(HllCount.DEFAULT_PRECISION);
+  }
+
+  int getPrecision() {
+    return precision;
+  }
+
+  void setPrecision(int precision) {
+    checkArgument(
+        precision >= HllCount.MINIMUM_PRECISION && precision <= 
HllCount.MAXIMUM_PRECISION,
+        "Invalid precision: %s. Valid range is [%s, %s].",
+        precision,
+        HllCount.MINIMUM_PRECISION,
+        HllCount.MAXIMUM_PRECISION);
+    this.precision = precision;
+  }
+
+  @Override
+  public Coder<HyperLogLogPlusPlus<HllT>> getAccumulatorCoder(
+      CoderRegistry registry, Coder<InputT> inputCoder) {
+    return HyperLogLogPlusPlusCoder.of();
+  }
+
+  @Override
+  public HyperLogLogPlusPlus<HllT> mergeAccumulators(
+      Iterable<HyperLogLogPlusPlus<HllT>> accumulators) {
+    HyperLogLogPlusPlus<HllT> merged = createAccumulator();
+    for (HyperLogLogPlusPlus<HllT> accumulator : accumulators) {
+      merged.merge(accumulator);
+    }
+    return merged;
+  }
+
+  @Override
+  public byte[] extractOutput(HyperLogLogPlusPlus<HllT> accumulator) {
+    return accumulator.serializeToByteArray();
+  }
+
+  static HllCountInitFn<Integer, Integer> forInteger() {
+    return new ForInteger();
+  }
+
+  static HllCountInitFn<Long, Long> forLong() {
+    return new ForLong();
+  }
+
+  static HllCountInitFn<String, String> forString() {
+    return new ForString();
+  }
+
+  static HllCountInitFn<byte[], ByteString> forBytes() {
+    return new ForBytes();
+  }
+
+  private static class ForInteger extends HllCountInitFn<Integer, Integer> {
+
+    @Override
+    public HyperLogLogPlusPlus<Integer> createAccumulator() {
+      return new 
HyperLogLogPlusPlus.Builder().normalPrecision(getPrecision()).buildForIntegers();
+    }
+
+    @Override
+    public HyperLogLogPlusPlus<Integer> addInput(
+        HyperLogLogPlusPlus<Integer> accumulator, Integer input) {
+      accumulator.add(input.intValue());
 
 Review comment:
   Yeah, that's something I could do. But let's keep the current status for the 
efficiency benefit. :)
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 288127)
    Time Spent: 9.5h  (was: 9h 20m)

> 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: 9.5h
>  Remaining Estimate: 0h
>




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

Reply via email to