[ https://issues.apache.org/jira/browse/BEAM-7013?focusedWorklogId=293262&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-293262 ]
ASF GitHub Bot logged work on BEAM-7013: ---------------------------------------- Author: ASF GitHub Bot Created on: 12/Aug/19 17:06 Start Date: 12/Aug/19 17:06 Worklog Time Spent: 10m Work Description: zfraa commented on pull request #9144: [BEAM-7013] Integrating ZetaSketch's HLL++ algorithm with Beam URL: https://github.com/apache/beam/pull/9144#discussion_r310683876 ########## File path: sdks/java/extensions/zetasketch/src/main/java/org/apache/beam/sdk/extensions/zetasketch/HllCountInitFn.java ########## @@ -0,0 +1,153 @@ +/* + * 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; + } + + // Should not be called at pipeline execution time. Review comment: Would expand on this (e.g., what happens if the advice is ignored?), and also turn it into Javadoc: "this will only have an effect if called before ...; any updates after that will be discarded; only intended to be used from within a builder" or similar. Since you have a builder in the caller class, I was wondering whether there is a good way to only allow setting the precision at construction time, which would be cleaner. But I think it would require a enum for the allowed types in the builder... (I'll think about it some more). ---------------------------------------------------------------- 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: 293262) Time Spent: 12h (was: 11h 50m) > 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: 12h > Remaining Estimate: 0h > -- This message was sent by Atlassian JIRA (v7.6.14#76016)