diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md
index ce703213ba5..aad31264914 100644
--- a/docs/additional-functionality/advanced_configs.md
+++ b/docs/additional-functionality/advanced_configs.md
@@ -397,6 +397,7 @@ Name | SQL Function(s) | Description | Default Value | Notes
spark.rapids.sql.expression.CollectSet|`collect_set`|Collect a set of unique elements, not supported in reduction|true|None|
spark.rapids.sql.expression.Count|`count`|Count aggregate operator|true|None|
spark.rapids.sql.expression.First|`first_value`, `first`|first aggregate operator|true|None|
+spark.rapids.sql.expression.HyperLogLogPlusPlus|`approx_count_distinct`|Aggregation approximate count distinct|true|None|
spark.rapids.sql.expression.Last|`last_value`, `last`|last aggregate operator|true|None|
spark.rapids.sql.expression.Max|`max`|Max aggregate operator|true|None|
spark.rapids.sql.expression.MaxBy|`max_by`|MaxBy aggregate operator. It may produce different results than CPU when multiple rows in a group have same minimum value in the ordering column and different associated values in the value column.|true|None|
diff --git a/docs/supported_ops.md b/docs/supported_ops.md
index 2bc819b4a41..556b43ba439 100644
--- a/docs/supported_ops.md
+++ b/docs/supported_ops.md
@@ -19820,6 +19820,132 @@ are limited.
NS |
+HyperLogLogPlusPlus |
+`approx_count_distinct` |
+Aggregation approximate count distinct |
+None |
+aggregation |
+input |
+S |
+S |
+S |
+S |
+S |
+S |
+S |
+S |
+PS UTC is only supported TZ for TIMESTAMP |
+S |
+S |
+S |
+NS |
+NS |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+NS |
+NS |
+NS |
+
+
+result |
+ |
+ |
+ |
+ |
+S |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
+reduction |
+input |
+S |
+S |
+S |
+S |
+S |
+S |
+S |
+S |
+PS UTC is only supported TZ for TIMESTAMP |
+S |
+S |
+S |
+NS |
+NS |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+PS UTC is only supported TZ for child TIMESTAMP; unsupported child types BINARY, CALENDAR, UDT, DAYTIME, YEARMONTH |
+NS |
+NS |
+NS |
+
+
+result |
+ |
+ |
+ |
+ |
+S |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+ |
+
+
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Last |
`last_value`, `last` |
last aggregate operator |
@@ -19965,34 +20091,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
Max |
`max` |
Max aggregate operator |
@@ -20282,6 +20380,34 @@ are limited.
NS |
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Min |
`min` |
Min aggregate operator |
@@ -20427,34 +20553,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
MinBy |
`min_by` |
MinBy aggregate operator. It may produce different results than CPU when multiple rows in a group have same minimum value in the ordering column and different associated values in the value column. |
@@ -20789,6 +20887,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
PivotFirst |
|
PivotFirst operator |
@@ -20933,34 +21059,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
StddevPop |
`stddev_pop` |
Aggregation computing population standard deviation |
@@ -21251,6 +21349,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
Sum |
`sum` |
Sum aggregate operator |
@@ -21396,34 +21522,6 @@ are limited.
|
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
VariancePop |
`var_pop` |
Aggregation computing population variance |
@@ -21714,6 +21812,34 @@ are limited.
|
+Expression |
+SQL Functions(s) |
+Description |
+Notes |
+Context |
+Param/Output |
+BOOLEAN |
+BYTE |
+SHORT |
+INT |
+LONG |
+FLOAT |
+DOUBLE |
+DATE |
+TIMESTAMP |
+STRING |
+DECIMAL |
+NULL |
+BINARY |
+CALENDAR |
+ARRAY |
+MAP |
+STRUCT |
+UDT |
+DAYTIME |
+YEARMONTH |
+
+
NormalizeNaNAndZero |
|
Normalize NaN and zero |
@@ -21793,34 +21919,6 @@ are limited.
NS |
-Expression |
-SQL Functions(s) |
-Description |
-Notes |
-Context |
-Param/Output |
-BOOLEAN |
-BYTE |
-SHORT |
-INT |
-LONG |
-FLOAT |
-DOUBLE |
-DATE |
-TIMESTAMP |
-STRING |
-DECIMAL |
-NULL |
-BINARY |
-CALENDAR |
-ARRAY |
-MAP |
-STRUCT |
-UDT |
-DAYTIME |
-YEARMONTH |
-
-
HiveGenericUDF |
|
Hive Generic UDF, the UDF can choose to implement a RAPIDS accelerated interface to get better performance |
diff --git a/integration_tests/src/main/python/hashing_test.py b/integration_tests/src/main/python/hashing_test.py
index e3ef67bc9f9..15e1bdd8ab8 100644
--- a/integration_tests/src/main/python/hashing_test.py
+++ b/integration_tests/src/main/python/hashing_test.py
@@ -1,4 +1,4 @@
-# Copyright (c) 2023-2024, NVIDIA CORPORATION.
+# Copyright (c) 2023-2025, NVIDIA CORPORATION.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@@ -36,7 +36,7 @@
_struct_of_xxhash_gens = StructGen([(f"c{i}", g) for i, g in enumerate(_atomic_gens)])
-# will be used by HyperLogLogPlusPLus(approx_count_distinct)
+# This is also used by HyperLogLogPlusPLus(approx_count_distinct)
xxhash_gens = (_atomic_gens + [_struct_of_xxhash_gens] + single_level_array_gens
+ nested_array_gens_sample + [
all_basic_struct_gen,
diff --git a/integration_tests/src/main/python/hyper_log_log_plus_plus_test.py b/integration_tests/src/main/python/hyper_log_log_plus_plus_test.py
new file mode 100644
index 00000000000..5a6f8f621de
--- /dev/null
+++ b/integration_tests/src/main/python/hyper_log_log_plus_plus_test.py
@@ -0,0 +1,38 @@
+# Copyright (c) 2021-2025, NVIDIA CORPORATION.
+#
+# Licensed 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.
+
+import pytest
+
+from asserts import assert_gpu_and_cpu_are_equal_sql
+from data_gen import *
+from hashing_test import xxhash_gens
+from marks import ignore_order
+
+
+@ignore_order(local=True)
+@pytest.mark.parametrize('data_gen', xxhash_gens, ids=idfn)
+def test_hllpp_groupby(data_gen):
+ assert_gpu_and_cpu_are_equal_sql(
+ lambda spark: gen_df(spark, [("c1", int_gen), ("c2", data_gen)]),
+ "tab",
+ "select c1, APPROX_COUNT_DISTINCT(c2) from tab group by c1")
+
+
+@ignore_order(local=True)
+@pytest.mark.parametrize('data_gen', xxhash_gens, ids=idfn)
+def test_hllpp_reduction(data_gen):
+ assert_gpu_and_cpu_are_equal_sql(
+ lambda spark: unary_op_df(spark, data_gen),
+ "tab",
+ "select APPROX_COUNT_DISTINCT(a) from tab")
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
index c23728ba43a..465d97e81eb 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
@@ -1,5 +1,5 @@
/*
- * Copyright (c) 2019-2024, NVIDIA CORPORATION.
+ * Copyright (c) 2019-2025, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -3989,6 +3989,36 @@ object GpuOverrides extends Logging {
GpuDynamicPruningExpression(child)
}
}),
+ expr[HyperLogLogPlusPlus](
+ "Aggregation approximate count distinct",
+ ExprChecks.reductionAndGroupByAgg(TypeSig.LONG, TypeSig.LONG,
+ // HyperLogLogPlusPlus depends on Xxhash64
+ // HyperLogLogPlusPlus supports all the types that Xxhash 64 supports
+ Seq(ParamCheck("input",XxHash64Shims.supportedTypes, TypeSig.all))),
+ (a, conf, p, r) => new UnaryExprMeta[HyperLogLogPlusPlus](a, conf, p, r) {
+
+ // It's the same as Xxhash64
+ override def tagExprForGpu(): Unit = {
+ val maxDepth = a.children.map(
+ c => XxHash64Utils.computeMaxStackSize(c.dataType)).max
+ if (maxDepth > Hash.MAX_STACK_DEPTH) {
+ willNotWorkOnGpu(s"The data type requires a stack depth of $maxDepth, " +
+ s"which exceeds the GPU limit of ${Hash.MAX_STACK_DEPTH}. " +
+ "The algorithm to calculate stack depth: " +
+ "1: Primitive type counts 1 depth; " +
+ "2: Array of Structure counts: 1 + depthOf(Structure); " +
+ "3: Array of Other counts: depthOf(Other); " +
+ "4: Structure counts: 1 + max of depthOf(child); " +
+ "5: Map counts: 2 + max(depthOf(key), depthOf(value)); "
+ )
+ }
+ }
+
+ override def convertToGpu(child: Expression): GpuExpression = {
+ GpuHyperLogLogPlusPlus(child, a.relativeSD)
+ }
+ }
+ ),
SparkShimImpl.ansiCastRule
).collect { case r if r != null => (r.getClassFor.asSubclass(classOf[Expression]), r)}.toMap
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/aggregate/GpuHyperLogLogPlusPlus.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/aggregate/GpuHyperLogLogPlusPlus.scala
new file mode 100644
index 00000000000..63508700fef
--- /dev/null
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/aggregate/GpuHyperLogLogPlusPlus.scala
@@ -0,0 +1,192 @@
+/*
+ * Copyright (c) 2024-2025, NVIDIA CORPORATION.
+ *
+ * Licensed 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.spark.sql.rapids.aggregate
+
+import scala.collection.immutable.Seq
+
+import ai.rapids.cudf
+import ai.rapids.cudf.{DType, GroupByAggregation, ReductionAggregation}
+import com.nvidia.spark.rapids._
+import com.nvidia.spark.rapids.Arm.{withResource, withResourceIfAllowed}
+import com.nvidia.spark.rapids.RapidsPluginImplicits.ReallyAGpuExpression
+import com.nvidia.spark.rapids.jni.HyperLogLogPlusPlusHostUDF
+import com.nvidia.spark.rapids.jni.HyperLogLogPlusPlusHostUDF.AggregationType
+import com.nvidia.spark.rapids.shims.ShimExpression
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression}
+import org.apache.spark.sql.rapids.{GpuCreateNamedStruct, GpuGetStructField}
+import org.apache.spark.sql.rapids.shims.DataTypeUtilsShim
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+case class CudfHLLPP(override val dataType: DataType,
+ precision: Int) extends CudfAggregate {
+ override lazy val reductionAggregate: cudf.ColumnVector => cudf.Scalar =
+ (input: cudf.ColumnVector) => {
+ if (input.getNullCount == input.getRowCount) {
+ // For NullType column or all values are null,
+ // return a struct scalar: struct(0L, 0L, ..., 0L)
+ val numCols = (1 << precision) / 10 + 1
+ withResource(cudf.ColumnVector.fromLongs(0L)) { cv =>
+ // Underlying uses deep-copy, so we can reuse this `cv` and fill multiple times.
+ val cvs: Array[cudf.ColumnView] = Array.fill(numCols)(cv)
+ cudf.Scalar.structFromColumnViews(cvs: _*)
+ }
+ } else {
+ val hll = new HyperLogLogPlusPlusHostUDF(AggregationType.Reduction, precision)
+ input.reduce(ReductionAggregation.hostUDF(hll), DType.STRUCT)
+ }
+ }
+ override lazy val groupByAggregate: GroupByAggregation = {
+ val hll =new HyperLogLogPlusPlusHostUDF(AggregationType.GroupBy, precision)
+ GroupByAggregation.hostUDF(hll)
+ }
+ override val name: String = "CudfHyperLogLogPlusPlus"
+}
+
+case class CudfMergeHLLPP(override val dataType: DataType,
+ precision: Int)
+ extends CudfAggregate {
+ override lazy val reductionAggregate: cudf.ColumnVector => cudf.Scalar =
+ (input: cudf.ColumnVector) => {
+ val hll = new HyperLogLogPlusPlusHostUDF(AggregationType.ReductionMerge, precision)
+ input.reduce(ReductionAggregation.hostUDF(hll), DType.STRUCT)
+ }
+
+ override lazy val groupByAggregate: GroupByAggregation = {
+ val hll = new HyperLogLogPlusPlusHostUDF(AggregationType.GroupByMerge, precision)
+ GroupByAggregation.hostUDF(hll)
+ }
+
+ override val name: String = "CudfMergeHyperLogLogPlusPlus"
+}
+
+/**
+ * Perform the final evaluation step to compute approximate count distinct from sketches.
+ * Input is long columns, first construct struct of long then feed to cuDF
+ */
+case class GpuHyperLogLogPlusPlusEvaluation(childExpr: Expression,
+ precision: Int)
+ extends GpuExpression with ShimExpression {
+ override def dataType: DataType = LongType
+
+ override def nullable: Boolean = false
+
+ override def prettyName: String = "HyperLogLogPlusPlus_evaluation"
+
+ override def children: scala.Seq[Expression] = Seq(childExpr)
+
+ override def columnarEval(batch: ColumnarBatch): GpuColumnVector = {
+ withResourceIfAllowed(childExpr.columnarEval(batch)) { sketches =>
+ val distinctValues = HyperLogLogPlusPlusHostUDF.estimateDistinctValueFromSketches(
+ sketches.getBase, precision)
+ GpuColumnVector.from(distinctValues, LongType)
+ }
+ }
+}
+
+/**
+ * Gpu version of HyperLogLogPlusPlus
+ * Spark APPROX_COUNT_DISTINCT on NULLs returns zero
+ */
+case class GpuHyperLogLogPlusPlus(childExpr: Expression, relativeSD: Double)
+ extends GpuAggregateFunction with Serializable {
+
+ // Consistent with Spark
+ private lazy val precision: Int =
+ Math.ceil(2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)).toInt;
+
+ private lazy val numRegistersPerSketch: Int = 1 << precision;
+
+ // Each long contains 10 register(max 6 bits)
+ private lazy val numWords = numRegistersPerSketch / 10 + 1
+
+ // Spark agg buffer type: long array
+ private lazy val sparkAggBufferAttributes: Seq[AttributeReference] = {
+ Seq.tabulate(numWords) { i =>
+ AttributeReference(s"MS[$i]", LongType)()
+ }
+ }
+
+ /**
+ * Spark uses long columns to save agg buffer, e.g.: long[52]
+ * Each long compacts multiple registers to save memory
+ */
+ override val aggBufferAttributes: Seq[AttributeReference] = sparkAggBufferAttributes
+
+ /**
+ * init long array with all zero
+ */
+ override lazy val initialValues: Seq[Expression] = Seq.tabulate(numWords) { _ =>
+ GpuLiteral(0L, LongType)
+ }
+
+ override lazy val inputProjection: Seq[Expression] = Seq(childExpr)
+
+ /**
+ * cuDF HLLPP sketch type: struct
+ */
+ private lazy val cuDFBufferType: DataType = DataTypeUtilsShim.fromAttributes(aggBufferAttributes)
+
+ /**
+ * cuDF uses Struct column to do aggregate
+ */
+ override lazy val updateAggregates: Seq[CudfAggregate] =
+ Seq(CudfHLLPP(cuDFBufferType, precision))
+
+ /**
+ * Convert long columns to Struct column
+ */
+ private def genStruct: Seq[Expression] = {
+ val names = Seq.tabulate(numWords) { i => GpuLiteral(s"MS[$i]", StringType) }
+ Seq(GpuCreateNamedStruct(names.zip(aggBufferAttributes).flatten { case (a, b) => List(a, b) }))
+ }
+
+ /**
+ * Convert Struct column to long columns
+ */
+ override lazy val postUpdate: Seq[Expression] = Seq.tabulate(numWords) {
+ i => GpuGetStructField(postUpdateAttr.head, i)
+ }
+
+ /**
+ * convert to Struct
+ */
+ override lazy val preMerge: Seq[Expression] = genStruct
+
+ override lazy val mergeAggregates: Seq[CudfAggregate] =
+ Seq(CudfMergeHLLPP(cuDFBufferType, precision))
+
+ /**
+ * Convert Struct column to long columns
+ */
+ override lazy val postMerge: Seq[Expression] = Seq.tabulate(numWords) {
+ i => GpuGetStructField(postMergeAttr.head, i)
+ }
+
+ override lazy val evaluateExpression: Expression =
+ GpuHyperLogLogPlusPlusEvaluation(genStruct.head, precision)
+
+ override def dataType: DataType = LongType
+
+ // Spark APPROX_COUNT_DISTINCT on NULLs returns zero
+ override def nullable: Boolean = false
+
+ override def prettyName: String = "approx_count_distinct"
+
+ override def children: Seq[Expression] = Seq(childExpr)
+}
diff --git a/tools/generated_files/320/operatorsScore.csv b/tools/generated_files/320/operatorsScore.csv
index d8c4ca63adc..24f3c88f8c4 100644
--- a/tools/generated_files/320/operatorsScore.csv
+++ b/tools/generated_files/320/operatorsScore.csv
@@ -134,6 +134,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/320/supportedExprs.csv b/tools/generated_files/320/supportedExprs.csv
index a9b6720be13..2aeca916a1a 100644
--- a/tools/generated_files/320/supportedExprs.csv
+++ b/tools/generated_files/320/supportedExprs.csv
@@ -688,6 +688,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/321/operatorsScore.csv b/tools/generated_files/321/operatorsScore.csv
index d8c4ca63adc..24f3c88f8c4 100644
--- a/tools/generated_files/321/operatorsScore.csv
+++ b/tools/generated_files/321/operatorsScore.csv
@@ -134,6 +134,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/321/supportedExprs.csv b/tools/generated_files/321/supportedExprs.csv
index a9b6720be13..2aeca916a1a 100644
--- a/tools/generated_files/321/supportedExprs.csv
+++ b/tools/generated_files/321/supportedExprs.csv
@@ -688,6 +688,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/322/operatorsScore.csv b/tools/generated_files/322/operatorsScore.csv
index d8c4ca63adc..24f3c88f8c4 100644
--- a/tools/generated_files/322/operatorsScore.csv
+++ b/tools/generated_files/322/operatorsScore.csv
@@ -134,6 +134,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/322/supportedExprs.csv b/tools/generated_files/322/supportedExprs.csv
index a9b6720be13..2aeca916a1a 100644
--- a/tools/generated_files/322/supportedExprs.csv
+++ b/tools/generated_files/322/supportedExprs.csv
@@ -688,6 +688,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/323/operatorsScore.csv b/tools/generated_files/323/operatorsScore.csv
index d8c4ca63adc..24f3c88f8c4 100644
--- a/tools/generated_files/323/operatorsScore.csv
+++ b/tools/generated_files/323/operatorsScore.csv
@@ -134,6 +134,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/323/supportedExprs.csv b/tools/generated_files/323/supportedExprs.csv
index a9b6720be13..2aeca916a1a 100644
--- a/tools/generated_files/323/supportedExprs.csv
+++ b/tools/generated_files/323/supportedExprs.csv
@@ -688,6 +688,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/324/operatorsScore.csv b/tools/generated_files/324/operatorsScore.csv
index d8c4ca63adc..24f3c88f8c4 100644
--- a/tools/generated_files/324/operatorsScore.csv
+++ b/tools/generated_files/324/operatorsScore.csv
@@ -134,6 +134,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/324/supportedExprs.csv b/tools/generated_files/324/supportedExprs.csv
index a9b6720be13..2aeca916a1a 100644
--- a/tools/generated_files/324/supportedExprs.csv
+++ b/tools/generated_files/324/supportedExprs.csv
@@ -688,6 +688,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/330/operatorsScore.csv b/tools/generated_files/330/operatorsScore.csv
index e86e30e606c..6707e7a0c00 100644
--- a/tools/generated_files/330/operatorsScore.csv
+++ b/tools/generated_files/330/operatorsScore.csv
@@ -139,6 +139,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/330/supportedExprs.csv b/tools/generated_files/330/supportedExprs.csv
index aa744638220..30211146127 100644
--- a/tools/generated_files/330/supportedExprs.csv
+++ b/tools/generated_files/330/supportedExprs.csv
@@ -713,6 +713,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/331/operatorsScore.csv b/tools/generated_files/331/operatorsScore.csv
index 229201ba885..d617fa10a0c 100644
--- a/tools/generated_files/331/operatorsScore.csv
+++ b/tools/generated_files/331/operatorsScore.csv
@@ -140,6 +140,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/331/supportedExprs.csv b/tools/generated_files/331/supportedExprs.csv
index 82aad5f070b..50d0e6bcabe 100644
--- a/tools/generated_files/331/supportedExprs.csv
+++ b/tools/generated_files/331/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/332/operatorsScore.csv b/tools/generated_files/332/operatorsScore.csv
index 229201ba885..d617fa10a0c 100644
--- a/tools/generated_files/332/operatorsScore.csv
+++ b/tools/generated_files/332/operatorsScore.csv
@@ -140,6 +140,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/332/supportedExprs.csv b/tools/generated_files/332/supportedExprs.csv
index 82aad5f070b..50d0e6bcabe 100644
--- a/tools/generated_files/332/supportedExprs.csv
+++ b/tools/generated_files/332/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/333/operatorsScore.csv b/tools/generated_files/333/operatorsScore.csv
index 229201ba885..d617fa10a0c 100644
--- a/tools/generated_files/333/operatorsScore.csv
+++ b/tools/generated_files/333/operatorsScore.csv
@@ -140,6 +140,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/333/supportedExprs.csv b/tools/generated_files/333/supportedExprs.csv
index 82aad5f070b..50d0e6bcabe 100644
--- a/tools/generated_files/333/supportedExprs.csv
+++ b/tools/generated_files/333/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/334/operatorsScore.csv b/tools/generated_files/334/operatorsScore.csv
index 229201ba885..d617fa10a0c 100644
--- a/tools/generated_files/334/operatorsScore.csv
+++ b/tools/generated_files/334/operatorsScore.csv
@@ -140,6 +140,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/334/supportedExprs.csv b/tools/generated_files/334/supportedExprs.csv
index 82aad5f070b..50d0e6bcabe 100644
--- a/tools/generated_files/334/supportedExprs.csv
+++ b/tools/generated_files/334/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/340/operatorsScore.csv b/tools/generated_files/340/operatorsScore.csv
index 16ac93a02ba..53fc6296144 100644
--- a/tools/generated_files/340/operatorsScore.csv
+++ b/tools/generated_files/340/operatorsScore.csv
@@ -141,6 +141,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/340/supportedExprs.csv b/tools/generated_files/340/supportedExprs.csv
index 9f9a3c37ed9..78a67e9ecf9 100644
--- a/tools/generated_files/340/supportedExprs.csv
+++ b/tools/generated_files/340/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/341/operatorsScore.csv b/tools/generated_files/341/operatorsScore.csv
index 16ac93a02ba..53fc6296144 100644
--- a/tools/generated_files/341/operatorsScore.csv
+++ b/tools/generated_files/341/operatorsScore.csv
@@ -141,6 +141,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/341/supportedExprs.csv b/tools/generated_files/341/supportedExprs.csv
index 9f9a3c37ed9..78a67e9ecf9 100644
--- a/tools/generated_files/341/supportedExprs.csv
+++ b/tools/generated_files/341/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/342/operatorsScore.csv b/tools/generated_files/342/operatorsScore.csv
index 16ac93a02ba..53fc6296144 100644
--- a/tools/generated_files/342/operatorsScore.csv
+++ b/tools/generated_files/342/operatorsScore.csv
@@ -141,6 +141,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/342/supportedExprs.csv b/tools/generated_files/342/supportedExprs.csv
index 9f9a3c37ed9..78a67e9ecf9 100644
--- a/tools/generated_files/342/supportedExprs.csv
+++ b/tools/generated_files/342/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/343/operatorsScore.csv b/tools/generated_files/343/operatorsScore.csv
index 16ac93a02ba..53fc6296144 100644
--- a/tools/generated_files/343/operatorsScore.csv
+++ b/tools/generated_files/343/operatorsScore.csv
@@ -141,6 +141,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/343/supportedExprs.csv b/tools/generated_files/343/supportedExprs.csv
index 9f9a3c37ed9..78a67e9ecf9 100644
--- a/tools/generated_files/343/supportedExprs.csv
+++ b/tools/generated_files/343/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/344/operatorsScore.csv b/tools/generated_files/344/operatorsScore.csv
index 16ac93a02ba..53fc6296144 100644
--- a/tools/generated_files/344/operatorsScore.csv
+++ b/tools/generated_files/344/operatorsScore.csv
@@ -141,6 +141,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/344/supportedExprs.csv b/tools/generated_files/344/supportedExprs.csv
index 9f9a3c37ed9..78a67e9ecf9 100644
--- a/tools/generated_files/344/supportedExprs.csv
+++ b/tools/generated_files/344/supportedExprs.csv
@@ -715,6 +715,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/350/operatorsScore.csv b/tools/generated_files/350/operatorsScore.csv
index cfb5b486942..1beb9c3dfcf 100644
--- a/tools/generated_files/350/operatorsScore.csv
+++ b/tools/generated_files/350/operatorsScore.csv
@@ -142,6 +142,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/350/supportedExprs.csv b/tools/generated_files/350/supportedExprs.csv
index 9e3939f4566..9cea2c67448 100644
--- a/tools/generated_files/350/supportedExprs.csv
+++ b/tools/generated_files/350/supportedExprs.csv
@@ -723,6 +723,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/351/operatorsScore.csv b/tools/generated_files/351/operatorsScore.csv
index cfb5b486942..1beb9c3dfcf 100644
--- a/tools/generated_files/351/operatorsScore.csv
+++ b/tools/generated_files/351/operatorsScore.csv
@@ -142,6 +142,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/351/supportedExprs.csv b/tools/generated_files/351/supportedExprs.csv
index 9e3939f4566..9cea2c67448 100644
--- a/tools/generated_files/351/supportedExprs.csv
+++ b/tools/generated_files/351/supportedExprs.csv
@@ -723,6 +723,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/352/operatorsScore.csv b/tools/generated_files/352/operatorsScore.csv
index cfb5b486942..1beb9c3dfcf 100644
--- a/tools/generated_files/352/operatorsScore.csv
+++ b/tools/generated_files/352/operatorsScore.csv
@@ -142,6 +142,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/352/supportedExprs.csv b/tools/generated_files/352/supportedExprs.csv
index 9e3939f4566..9cea2c67448 100644
--- a/tools/generated_files/352/supportedExprs.csv
+++ b/tools/generated_files/352/supportedExprs.csv
@@ -723,6 +723,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/353/operatorsScore.csv b/tools/generated_files/353/operatorsScore.csv
index cfb5b486942..1beb9c3dfcf 100644
--- a/tools/generated_files/353/operatorsScore.csv
+++ b/tools/generated_files/353/operatorsScore.csv
@@ -142,6 +142,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/353/supportedExprs.csv b/tools/generated_files/353/supportedExprs.csv
index 9e3939f4566..9cea2c67448 100644
--- a/tools/generated_files/353/supportedExprs.csv
+++ b/tools/generated_files/353/supportedExprs.csv
@@ -723,6 +723,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/354/operatorsScore.csv b/tools/generated_files/354/operatorsScore.csv
index cfb5b486942..1beb9c3dfcf 100644
--- a/tools/generated_files/354/operatorsScore.csv
+++ b/tools/generated_files/354/operatorsScore.csv
@@ -142,6 +142,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/354/supportedExprs.csv b/tools/generated_files/354/supportedExprs.csv
index 9e3939f4566..9cea2c67448 100644
--- a/tools/generated_files/354/supportedExprs.csv
+++ b/tools/generated_files/354/supportedExprs.csv
@@ -723,6 +723,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv
index d8c4ca63adc..24f3c88f8c4 100644
--- a/tools/generated_files/operatorsScore.csv
+++ b/tools/generated_files/operatorsScore.csv
@@ -134,6 +134,7 @@ HiveGenericUDF,4
HiveHash,4
HiveSimpleUDF,4
Hour,4
+HyperLogLogPlusPlus,4
Hypot,4
If,4
In,4
diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv
index a9b6720be13..2aeca916a1a 100644
--- a/tools/generated_files/supportedExprs.csv
+++ b/tools/generated_files/supportedExprs.csv
@@ -688,6 +688,10 @@ First,S,`first_value`; `first`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,N
First,S,`first_value`; `first`,None,reduction,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
First,S,`first_value`; `first`,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,aggregation,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS
+HyperLogLogPlusPlus,S,`approx_count_distinct`,None,reduction,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA
Last,S,`last_value`; `last`,None,aggregation,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,aggregation,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS
Last,S,`last_value`; `last`,None,reduction,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS