lsyldliu commented on code in PR #20243:
URL: https://github.com/apache/flink/pull/20243#discussion_r921864800


##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/functions/aggregate/hyperloglog/HyperLogLogPlusPlusTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.flink.table.runtime.functions.aggregate.hyperloglog;
+
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.function.Function;
+
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** The test of HyperLogLogPlusPlus is inspired from Apache Spark. */
+public class HyperLogLogPlusPlusTest {
+
+    @Test
+    public void testInvalidRelativeSD() {
+        assertThatThrownBy(() -> new HyperLogLogPlusPlus(0.4))

Review Comment:
   What about using `CommonTestUtils.assertThrows()`



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/BatchApproxCountDistinctAggFunctions.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import 
org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.HllBuffer;
+import 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.HyperLogLogPlusPlus;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.DEFAULT_SEED;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashInt;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashLong;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashUnsafeBytes;
+import static 
org.apache.flink.table.types.utils.DataTypeUtils.toInternalDataType;
+
+/** Built-in APPROX_COUNT_DISTINCT aggregate function for Batch sql. */
+public class BatchApproxCountDistinctAggFunctions {

Review Comment:
   This class must be placed in runtime mode, the planner package is not 
available in runtime after scala free.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/BatchApproxCountDistinctAggFunctions.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import 
org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.HllBuffer;
+import 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.HyperLogLogPlusPlus;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.DEFAULT_SEED;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashInt;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashLong;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashUnsafeBytes;
+import static 
org.apache.flink.table.types.utils.DataTypeUtils.toInternalDataType;
+
+/** Built-in APPROX_COUNT_DISTINCT aggregate function for Batch sql. */
+public class BatchApproxCountDistinctAggFunctions {

Review Comment:
   It would be better add an e2e test.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/hyperloglog/XxHash64Function.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.flink.table.runtime.functions.aggregate.hyperloglog;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+/** Utility class to get hash code based on {@link XXH64}. */
+public class XxHash64Function {

Review Comment:
   As offline discuss, this class is not needed.



##########
flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/functions/aggregate/hyperloglog/HyperLogLogPlusPlusTest.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.flink.table.runtime.functions.aggregate.hyperloglog;
+
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.function.Function;
+
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** The test of HyperLogLogPlusPlus is inspired from Apache Spark. */
+public class HyperLogLogPlusPlusTest {
+
+    @Test

Review Comment:
   It would be better use junit5.



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/BatchApproxCountDistinctAggFunctions.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.flink.table.planner.functions.aggfunctions;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import 
org.apache.flink.table.runtime.functions.aggregate.BuiltInAggregateFunction;
+import 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.HllBuffer;
+import 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.HyperLogLogPlusPlus;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import java.util.Collections;
+import java.util.List;
+
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.DEFAULT_SEED;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashInt;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashLong;
+import static 
org.apache.flink.table.runtime.functions.aggregate.hyperloglog.XxHash64Function.hashUnsafeBytes;
+import static 
org.apache.flink.table.types.utils.DataTypeUtils.toInternalDataType;
+
+/** Built-in APPROX_COUNT_DISTINCT aggregate function for Batch sql. */
+public class BatchApproxCountDistinctAggFunctions {
+
+    /** Base function for APPROX_COUNT_DISTINCT aggregate. */
+    public abstract static class ApproxCountDistinctAggFunction<T>
+            extends BuiltInAggregateFunction<Long, HllBuffer> {
+
+        private static final Double RELATIVE_SD = 0.01;
+        private transient HyperLogLogPlusPlus hll;
+
+        private final transient DataType valueDataType;
+
+        public ApproxCountDistinctAggFunction(LogicalType valueType) {
+            this.valueDataType = toInternalDataType(valueType);
+        }
+
+        @Override
+        public HllBuffer createAccumulator() {
+            hll = new HyperLogLogPlusPlus(RELATIVE_SD);
+            HllBuffer buffer = new HllBuffer();
+            buffer.array = new long[hll.getNumWords()];
+            resetAccumulator(buffer);
+            return buffer;
+        }
+
+        public void accumulate(HllBuffer buffer, Object input) throws 
Exception {
+            if (input != null) {
+                hll.updateByHashcode(buffer, getHashcode((T) input));
+            }
+        }
+
+        abstract long getHashcode(T t);

Review Comment:
   Why here provide an abstract method and implement different subclass?



-- 
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: issues-unsubscr...@flink.apache.org

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

Reply via email to