Aggregate Functions¶
Aggregate functions operate on a set of values to compute a single result.
Except for count(), count_if(), max_by(), min_by() and
approx_distinct(), all of these aggregate functions ignore null values
and return null for no input rows or when all values are null. For example,
sum() returns null rather than zero and avg() does not include null
values in the count. The coalesce function can be used to convert null into
zero.
Some aggregate functions such as array_agg() produce different results
depending on the order of input values.
General Aggregate Functions¶
- arbitrary(x) [same as x]¶
Returns an arbitrary non-null value of
x, if one exists.
- any_value(x) [same as x]¶
This is an alias for
arbitrary().
- array_agg(x) array<[same as x]>¶
Returns an array created from the input
xelements. Ignores null inputs if presto.array_agg.ignore_nulls is set to false.
- avg(x) double | real | decimal¶
Returns the average (arithmetic mean) of all non-null input values. When x is of type REAL, the result type is REAL. When x is an integer or a DOUBLE, the result is DOUBLE. When x is of type DECIMAL(p, s), the result type is DECIMAL(p, s). Note: For the overflow cases, Velox returns a result when Presto throws “Decimal overflow”.
SELECT AVG(col) FROM ( VALUES (CAST(9999999999999999999999999999999.9999999 AS DECIMAL(38,7))), (CAST(9999999999999999999999999999999.9999999 AS DECIMAL(38,7))) ) AS t(col); -- Velox: 9999999999999999999999999999999.9999999 -- Presto: Decimal overflow
- bool_and(boolean) boolean¶
Returns
TRUEif every input value isTRUE, otherwiseFALSE.
- bool_or(boolean) boolean¶
Returns
TRUEif any input value isTRUE, otherwiseFALSE.
- checksum(x) varbinary¶
Returns an order-insensitive checksum of the given values.
- count(*) bigint¶
Returns the number of input rows.
- count(x) bigint
Returns the number of non-null input values.
- count_if(x) bigint¶
Returns the number of
TRUEinput values. This function is equivalent tocount(CASE WHEN x THEN 1 END).
- entropy(c) double¶
Returns the log-2 entropy of count input-values.
\[\mathrm{entropy}(c) = \sum_i \left[ {c_i \over \sum_j [c_j]} \log_2\left({\sum_j [c_j] \over c_i}\right) \right].\]cmust be aintegercolumn of non-negative values.The function ignores any
NULLcount. If the sum of non-NULLcounts is 0, it returns 0.
- every(boolean) boolean¶
This is an alias for
bool_and().
- histogram(x)¶
Returns a map containing the count of the number of times each input value occurs. Supports integral, floating-point, boolean, timestamp, and date input types.
- geometric_mean(bigint) double¶
- geometric_mean(double) double
- geometric_mean(real) real
Returns the geometric mean of all input values.
- max_by(x, y) [same as x]¶
Returns the value of
xassociated with the maximum value ofyover all input values.ymust be an orderable type.
- max_by(x, y, n) -> array([same as x])
Returns n values of
xassociated with the n largest values ofyin descending order ofy.
- min_by(x, y) [same as x]¶
Returns the value of
xassociated with the minimum value ofyover all input values.ymust be an orderable type.
- min_by(x, y, n) -> array([same as x])
Returns n values of
xassociated with the n smallest values ofyin ascending order ofy.
- max(x) [same as x]¶
Returns the maximum value of all input values.
xmust not contain nulls when it is complex type.xmust be an orderable type. Nulls are ignored if there are any non-null inputs. For REAL and DOUBLE types, NaN is considered greater than Infinity.
- max(x, n) array<[same as x]>
Returns
nlargest values of all input values ofx.nmust be a positive integer and not exceed 10’000. Currently not supported for ARRAY, MAP, and ROW input types. Nulls are not included in the output array. For REAL and DOUBLE types, NaN is considered greater than Infinity.
- min(x) [same as x]¶
Returns the minimum value of all input values.
xmust not contain nulls when it is complex type.xmust be an orderable type. Nulls are ignored if there are any non-null inputs. For REAL and DOUBLE types, NaN is considered greater than Infinity.
- min(x, n) array<[same as x]>
Returns
nsmallest values of all input values ofx.nmust be a positive integer and not exceed 10’000. Currently not supported for ARRAY, MAP, and ROW input types. Nulls are not included in output array. For REAL and DOUBLE types, NaN is considered greater than Infinity.
- multimap_agg(K key, V value) -> map(K, array(V))¶
Returns a multimap created from the input
key/valuepairs. Each key can be associated with multiple values.
- reduce_agg(inputValue T, initialState S, inputFunction(S, T, S), combineFunction(S, S, S)) S¶
Reduces all non-NULL input values into a single value.
inputFunctionwill be invoked for each non-NULL input value. If all inputs are NULL, the result is NULL. In addition to taking the input value,inputFunctiontakes the current state, initiallyinitialState, and returns the new state.combineFunctionwill be invoked to combine two states into a new state. The final state is returned. Throws an error ifinitialStateis NULL orinputFunctionorcombineFunctionreturns a NULL.Take care when designing
initialState,inputFunctionandcombineFunction. These need to support evaluating aggregation in a distributed manner using partial aggregation on many nodes, followed by shuffle over group-by keys, followed by final aggregation. Given a set of all possible values of state, make sure that combineFunction is commutative and associative operation with initialState as the identity value.combineFunction(s, initialState) = s for any s
combineFunction(s1, s2) = combineFunction(s2, s1) for any s1 and s2
combineFunction(s1, combineFunction(s2, s3)) = combineFunction(combineFunction(s1, s2), s3) for any s1, s2, s3
In addition, make sure that the following holds for the inputFunction:
inputFunction(inputFunction(initialState, x), y) = combineFunction(inputFunction(initialState, x), inputFunction(initialState, y)) for any x and y
Check out blog post about reduce_agg for more context.
Note that reduce_agg doesn’t support evaluation over sorted inputs.:
-- Compute sum (for illustration purposes only; use SUM aggregate function in production queries). SELECT id, reduce_agg(value, 0, (a, b) -> a + b, (a, b) -> a + b) FROM ( VALUES (1, 2), (1, 3), (1, 4), (2, 20), (2, 30), (2, 40) ) AS t(id, value) GROUP BY id; -- (1, 9) -- (2, 90) -- Compute product. SELECT id, reduce_agg(value, 1, (a, b) -> a * b, (a, b) -> a * b) FROM ( VALUES (1, 2), (1, 3), (1, 4), (2, 20), (2, 30), (2, 40) ) AS t(id, value) GROUP BY id; -- (1, 24) -- (2, 24000) -- Compute avg (for illustration purposes only; use AVG aggregate function in production queries). SELECT id, sum_and_count.sum / sum_and_count.count FROM ( SELECT id, reduce_agg(value, CAST(row(0, 0) AS row(sum double, count bigint)), (s, x) -> CAST(row(s.sum + x, s.count + 1) AS row(sum double, count bigint)), (s, s2) -> CAST(row(s.sum + s2.sum, s.count + s2.count) AS row(sum double, count bigint))) AS sum_and_count FROM ( VALUES (1, 2), (1, 3), (1, 4), (2, 20), (2, 30), (2, 40) ) AS t(id, value) GROUP BY id ); -- (1, 3.0) -- (2, 30.0)
- set_agg(x) array<[same as x]>¶
Returns an array created from the distinct input
xelements.xmust not contain nulls when it is complex type.
- set_union(array(T)) -> array(T)¶
Returns an array of all the distinct values contained in each array of the input.
Returns an empty array if all input arrays are NULL.
Example:
SELECT set_union(elements) FROM ( VALUES ARRAY[1, 2, 3], ARRAY[2, 3, 4] ) AS t(elements);
Returns ARRAY[1, 2, 3, 4]
- sum(x) [same as x]¶
Returns the sum of all input values.
Bitwise Aggregate Functions¶
- bitwise_and_agg(x) [same as x]¶
Returns the bitwise AND of all input values in 2’s complement representation.
Supported types are TINYINT, SMALLINT, INTEGER and BIGINT.
- bitwise_or_agg(x) [same as x]¶
Returns the bitwise OR of all input values in 2’s complement representation.
Supported types are TINYINT, SMALLINT, INTEGER and BIGINT.
- bitwise_xor_agg(x) [same as x]¶
Returns the bitwise XOR of all input values in 2’s complement representation.
Supported types are TINYINT, SMALLINT, INTEGER and BIGINT.
Map Aggregate Functions¶
- map_agg(K key, V value) -> map(K, V)¶
Returns a map created from the input
key/valuepairs. Inputs with NULL or duplicate keys are ignored.
- map_union(map(K, V)) -> map(K, V)¶
Returns the union of all the input
maps. If akeyis found in multiple inputmaps, thatkey’svaluein the resultingmapcomes from an arbitrary inputmap.
- map_union_sum(map(K, V)) -> map(K, V)¶
Returns the union of all the input maps summing the values of matching keys in all the maps. All null values in the original maps are coalesced to 0.
Approximate Aggregate Functions¶
- approx_distinct(x) bigint¶
Returns the approximate number of distinct input values. This function provides an approximation of
count(DISTINCT x). Zero is returned if all input values are null.This function should produce a standard error of 2.3%, which is the standard deviation of the (approximately normal) error distribution over all possible sets. It does not guarantee an upper bound on the error for any specific input set.
- approx_distinct(x, e) bigint
Returns the approximate number of distinct input values. This function provides an approximation of
count(DISTINCT x). Zero is returned if all input values are null.This function should produce a standard error of no more than
e, which is the standard deviation of the (approximately normal) error distribution over all possible sets. It does not guarantee an upper bound on the error for any specific input set. The current implementation of this function requires thatebe in the range of[0.0040625, 0.26000].
- approx_most_frequent(buckets, value, capacity) map<[same as value], bigint>¶
Computes the top frequent values up to
bucketselements approximately. Approximate estimation of the function enables us to pick up the frequent values with less memory. Largercapacityimproves the accuracy of underlying algorithm with sacrificing the memory capacity. The returned value is a map containing the top elements with corresponding estimated frequency.For BOOLEAN ‘value’, this function always returns ‘perfect’ result. ‘bucket’ and ‘capacity’ arguments are ignored in this case.
The error of the function depends on the permutation of the values and its cardinality. We can set the capacity same as the cardinality of the underlying data to achieve the least error.
bucketsandcapacitymust bebigint.valuecan be numeric or string type.The function uses the stream summary data structure proposed in the paper Efficient computation of frequent and top-k elements in data streams by A. Metwally, D. Agrawal and A. Abbadi.
- approx_percentile(x, percentage) [same as x]¶
Returns the approximate percentile for all input values of
xat the givenpercentage. The value ofpercentagemust be between zero and one and must be constant for all input rows.
- approx_percentile(x, percentage, accuracy) [same as x]
As
approx_percentile(x, percentage), but with a maximum rank error ofaccuracy. The value ofaccuracymust be between zero and one (exclusive) and must be constant for all input rows. Note that a lower “accuracy” is really a lower error threshold, and thus more accurate. The default accuracy is 0.0133. The underlying implementation is KLL sketch thus has a stronger guarantee for accuracy than T-Digest.
- approx_percentile(x, percentages) array<[same as x]>
Returns the approximate percentile for all input values of
xat each of the specified percentages. Each element of thepercentagesarray must be between zero and one, and the array must be constant for all input rows.
- approx_percentile(x, percentages, accuracy) array<[same as x]>
As
approx_percentile(x, percentages), but with a maximum rank error ofaccuracy.
- approx_percentile(x, w, percentage) [same as x]
Returns the approximate weighed percentile for all input values of
xusing the per-item weightwat the percentagep. The weight must be an integer value of at least one. It is effectively a replication count for the valuexin the percentile set. The value ofpmust be between zero and one and must be constant for all input rows.
- approx_percentile(x, w, percentage, accuracy) [same as x]
As
approx_percentile(x, w, percentage), but with a maximum rank error ofaccuracy.
- approx_percentile(x, w, percentages) array<[same as x]>
Returns the approximate weighed percentile for all input values of
xusing the per-item weightwat each of the given percentages specified in the array. The weight must be an integer value of at least one. It is effectively a replication count for the valuexin the percentile set. Each element of the array must be between zero and one, and the array must be constant for all input rows.
- approx_percentile(x, w, percentages, accuracy) array<[same as x]>
As
approx_percentile(x, w, percentages), but with a maximum rank error ofaccuracy.
Classification Metrics Aggregate Functions¶
The following functions each measure how some metric of a binary confusion matrix changes as a function of classification thresholds. They are meant to be used in conjunction.
For example, to find the precision-recall curve, use
WITH recall_precision AS ( SELECT CLASSIFICATION_RECALL(10000, correct, pred) AS recalls, CLASSIFICATION_PRECISION(10000, correct, pred) AS precisions FROM classification_dataset ) SELECT recall, precision FROM recall_precision CROSS JOIN UNNEST(recalls, precisions) AS t(recall, precision)
To get the corresponding thresholds for these values, use
WITH recall_precision AS ( SELECT CLASSIFICATION_THRESHOLDS(10000, correct, pred) AS thresholds, CLASSIFICATION_RECALL(10000, correct, pred) AS recalls, CLASSIFICATION_PRECISION(10000, correct, pred) AS precisions FROM classification_dataset ) SELECT threshold, recall, precision FROM recall_precision CROSS JOIN UNNEST(thresholds, recalls, precisions) AS t(threshold, recall, precision)
To find the ROC curve, use
WITH fallout_recall AS ( SELECT CLASSIFICATION_FALLOUT(10000, correct, pred) AS fallouts, CLASSIFICATION_RECALL(10000, correct, pred) AS recalls FROM classification_dataset ) SELECT fallout recall, FROM recall_fallout CROSS JOIN UNNEST(fallouts, recalls) AS t(fallout, recall)
- classification_miss_rate(buckets, y, x, weight) array<double>¶
Computes the miss-rate with up to
bucketsnumber of buckets. Returns an array of miss-rate values.yshould be a boolean outcome value;xshould be predictions, each between 0 and 1;weightshould be non-negative values, indicating the weight of the instance.The miss-rate is defined as a sequence whose \(j\)-th entry is
\[{ \sum_{i \;|\; x_i \leq t_j \bigwedge y_i = 1} \left[ w_i \right] \over \sum_{i \;|\; x_i \leq t_j \bigwedge y_i = 1} \left[ w_i \right] + \sum_{i \;|\; x_i > t_j \bigwedge y_i = 1} \left[ w_i \right] },\]where \(t_j\) is the \(j\)-th smallest threshold, and \(y_i\), \(x_i\), and \(w_i\) are the \(i\)-th entries of
y,x, andweight, respectively.
- classification_miss_rate(buckets, y, x) array<double>¶
This function is equivalent to the variant of
classification_miss_rate()that takes aweight, with a per-item weight of1.
- classification_fall_out(buckets, y, x, weight) array<double>¶
Computes the fall-out with up to
bucketsnumber of buckets. Returns an array of fall-out values.yshould be a boolean outcome value;xshould be predictions, each between 0 and 1;weightshould be non-negative values, indicating the weight of the instance.The fall-out is defined as a sequence whose \(j\)-th entry is
\[{ \sum_{i \;|\; x_i > t_j \bigwedge y_i = 0} \left[ w_i \right] \over \sum_{i \;|\; y_i = 0} \left[ w_i \right] },\]where \(t_j\) is the \(j\)-th smallest threshold, and \(y_i\), \(x_i\), and \(w_i\) are the \(i\)-th entries of
y,x, andweight, respectively.
- classification_fall_out(buckets, y, x) array<double>¶
This function is equivalent to the variant of
classification_fall_out()that takes aweight, with a per-item weight of1.
- classification_precision(buckets, y, x, weight) array<double>¶
Computes the precision with up to
bucketsnumber of buckets. Returns an array of precision values.yshould be a boolean outcome value;xshould be predictions, each between 0 and 1;weightshould be non-negative values, indicating the weight of the instance.The precision is defined as a sequence whose \(j\)-th entry is
\[{ \sum_{i \;|\; x_i > t_j \bigwedge y_i = 1} \left[ w_i \right] \over \sum_{i \;|\; x_i > t_j} \left[ w_i \right] },\]where \(t_j\) is the \(j\)-th smallest threshold, and \(y_i\), \(x_i\), and \(w_i\) are the \(i\)-th entries of
y,x, andweight, respectively.
- classification_precision(buckets, y, x) array<double>¶
This function is equivalent to the variant of
classification_precision()that takes aweight, with a per-item weight of1.
- classification_recall(buckets, y, x, weight) array<double>¶
Computes the recall with up to
bucketsnumber of buckets. Returns an array of recall values.yshould be a boolean outcome value;xshould be predictions, each between 0 and 1;weightshould be non-negative values, indicating the weight of the instance.The recall is defined as a sequence whose \(j\)-th entry is
\[{ \sum_{i \;|\; x_i > t_j \bigwedge y_i = 1} \left[ w_i \right] \over \sum_{i \;|\; y_i = 1} \left[ w_i \right] },\]where \(t_j\) is the \(j\)-th smallest threshold, and \(y_i\), \(x_i\), and \(w_i\) are the \(i\)-th entries of
y,x, andweight, respectively.
- classification_recall(buckets, y, x) array<double>¶
This function is equivalent to the variant of
classification_recall()that takes aweight, with a per-item weight of1.
- classification_thresholds(buckets, y, x) array<double>¶
Computes the thresholds with up to
bucketsnumber of buckets. Returns an array of threshold values.yshould be a boolean outcome value;xshould be predictions, each between 0 and 1.The thresholds are defined as a sequence whose \(j\)-th entry is the \(j\)-th smallest threshold.
Statistical Aggregate Functions¶
- corr(y, x) double¶
Returns correlation coefficient of input values.
- covar_pop(y, x) double¶
Returns the population covariance of input values.
- covar_samp(y, x) double¶
Returns the sample covariance of input values.
- kurtosis(x) double¶
Returns the excess kurtosis of all input values. Unbiased estimate using the following expression:
\[\mathrm{kurtosis}(x) = {n(n+1) \over (n-1)(n-2)(n-3)} { \sum[(x_i-\mu)^4] \over \sigma^4} -3{ (n-1)^2 \over (n-2)(n-3) },\]where \(\mu\) is the mean, and \(\sigma\) is the standard deviation.
- regr_avgx(y, x) double¶
Returns the average of the independent value in a group.
yis the dependent value.xis the independent value.
- regr_avgy(y, x) double¶
Returns the average of the dependent value in a group.
yis the dependent value.xis the independent value.
- regr_count(y, x) double¶
Returns the number of non-null pairs of input values.
yis the dependent value.xis the independent value.
- regr_intercept(y, x) double¶
Returns linear regression intercept of input values.
yis the dependent value.xis the independent value.
- regr_r2(y, x) double¶
Returns the coefficient of determination of the linear regression.
yis the dependent value.xis the independent value. If regr_sxx(y, x) is 0, result is null. If regr_syy(y, x) is 0 and regr_sxx(y, x) isn’t 0, result is 1.
- regr_slope(y, x) double¶
Returns linear regression slope of input values.
yis the dependent value.xis the independent value.
- regr_sxx(y, x) double¶
Returns the sum of the squares of the independent values in a group.
yis the dependent value.xis the independent value.
- regr_sxy(y, x) double¶
Returns the sum of the product of the dependent and independent values in a group.
yis the dependent value.xis the independent value.
- regr_syy(y, x) double¶
Returns the sum of the squares of the dependent values in a group.
yis the dependent value.xis the independent value.
- skewness(x) double¶
Returns the skewness of all input values.
- stddev(x) double¶
This is an alias for
stddev_samp().
- stddev_pop(x) double¶
Returns the population standard deviation of all input values.
- stddev_samp(x) double¶
Returns the sample standard deviation of all input values.
- variance(x) double¶
This is an alias for
var_samp().
- var_pop(x) double¶
Returns the population variance of all input values.
- var_samp(x) double¶
Returns the sample variance of all input values.
Noisy Aggregate Functions¶
Overview¶
Noisy aggregate functions provide random, noisy approximations of common
aggregations like sum(), count(), and approx_distinct() as well as sketches like
approx_set(). By injecting random noise into results, noisy aggregation functions make it
more difficult to determine or confirm the exact data that was aggregated.
While many of these functions resemble differential privacy mechanisms, neither the values returned by these functions nor the query results that incorporate these functions are differentially private in general. See Limitations below for more details. Users who wish to support a strong privacy guarantee should discuss with a suitable technical expert first.
Counts, Sums, and Averages¶
- noisy_count_if_gaussian(col, noise_scale[, random_seed]) bigint¶
Counts the
TRUEvalues incoland then adds a normally distributed random double value with 0 mean and standard deviation ofnoise_scaleto the true count. The noisy count is post-processed to be non-negative and rounded to bigint.If provided,
random_seedis used to seed the random number generator. Otherwise, noise is drawn from a secure random. (Note: ``random_seed`` is a constant and shared across all groups in a query. It is kept in each accmulator to ensure the ``random_seed`` is accessible in the final aggregation step.)SELECT noisy_count_if_gaussian(orderkey > 10000, 20.0) FROM lineitem; -- 50180 (1 row) SELECT noisy_count_if_gaussian(orderkey > 10000, 20.0) FROM lineitem WHERE false; -- NULL (1 row)
Note
Unlike
count_if(), this function returnsNULLwhen the (true) count is 0.
- noisy_count_gaussian(col, noise_scale[, random_seed]) bigint¶
Counts the non-null values in
coland then adds a normally distributed random double value with 0 mean and standard deviation ofnoise_scaleto the true count. The noisy count is post-processed to be non-negative and rounded to bigint.If provided,
random_seedis used to seed the random number generator. Otherwise, noise is drawn from a secure random.SELECT noisy_count_gaussian(orderkey, 20.0) FROM lineitem; -- 60181 (1 row) SELECT noisy_count_gaussian(orderkey, 20.0) FROM lineitem WHERE false; -- NULL (1 row)
Note
Unlike
count(), this function returnsNULLwhen the (true) count is 0.Distinct counting can be performed using
noisy_count_gaussian()(DISTINCT col, ...), or withnoisy_approx_distinct_sfm(). Generally speaking,noisy_count_gaussian()returns more accurate results but at a larger computational cost.
- noisy_sum_gaussian(col, noise_scale[, random_seed]) double¶
Calculates the sum over the input values in
coland then adds a normally distributed random double value with 0 mean and standard deviation ofnoise_scale.If provided,
random_seedis used to seed the random number generator. Otherwise, noise is drawn from a secure random.
- noisy_sum_gaussian(col, noise_scale, lower, upper[, random_seed]) double¶
Calculates the sum over the input values in
coland then adds a normally distributed random double value with 0 mean and standard deviation ofnoise_scale. Each value is clipped to the range of [lower,upper] before adding to the sum.If provided,
random_seedis used to seed the random number generator. Otherwise, noise is drawn from a secure random.
- noisy_avg_gaussian(col, noise_scale[, random_seed]) double¶
Calculates the average (arithmetic mean) of all the input values in col and then adds a normally distributed random double value with 0 mean and standard deviation of noise_scale.
If provided,
random_seedis used to seed the random number generator. Otherwise, noise is drawn from a secure random.
- noisy_avg_gaussian(col, noise_scale, lower, upper[, random_seed]) double¶
Calculates the average (arithmetic mean) of all the input values in
coland then adds a normally distributed random double value with 0 mean and standard deviation ofnoise_scale. Each value is clipped to the range of [lower,upper] before averaging.If provided,
random_seedis used to seed the random number generator. Otherwise, noise is drawn from a secure random.
- noisy_approx_set_sfm(col, epsilon[, buckets[, precision]]) SfmSketch¶
Returns an SFM sketch of the input values in
col. This is analogous to theapprox_set()function, which returns a (deterministic) HyperLogLog sketch.colcurrently supports types: “bigint”, “double”, “string”, “varbinary”.epsilon(double) is a positive number that controls the level of noise in the sketch, as described in [Hehir2023]. Smaller values of epsilon correspond to noisier sketches.buckets(int) defaults to 4096.precision(int) defaults to 24.
Note
Unlike
approx_set(), this function returnsNULLwhencolis empty. If this behavior is undesirable, usecoalesce()withnoisy_empty_approx_set_sfm().
- noisy_approx_distinct_sfm(col, epsilon[, buckets[, precision]]) bigint¶
Equivalent to
cardinality(noisy_approx_set_sfm(col, epsilon, buckets, precision)), this returns the approximate cardinality (distinct count) of the column col. This is analogous to the (deterministic)approx_distinct()function.Note
Unlike
approx_distinct(), this function returnsNULLwhencolis empty.
- noisy_empty_approx_set_sfm(epsilon[, buckets[, precision]]) SfmSketch¶
Returns an SFM sketch with no items in it. This is analogous to the
empty_approx_set()function, which returns an empty (deterministic)HyperLogLogsketch.epsilon(double) is a positive number that controls the level of noise in the sketch, as described in [Hehir2023]. Smaller values of epsilon correspond to noisier sketches.buckets(int) defaults to 4096.precision(int) defaults to 24.
- noisy_approx_set_sfm_from_index_and_zeros(col_index, col_zeros, epsilon, buckets[, precision]) SfmSketch¶
Returns an SFM sketch of the input values in
col_indexandcol_zeros.This is similar to
noisy_approx_set_sfm()except that function calculates axxhash64()ofcol, and calculates the SFM PCSA bucket index and number of trailing zeros as described in [FlajoletMartin1985]. In this function, the caller must explicitly calculate the hash bucket index and zeros themselves and pass them as argumentscol_indexandcol_zeros.col_index(bigint) must be in the range0..buckets-1.col_zeros(bigint) must be in the range0..64. If it exceedsprecision, it is cropped toprecision-1.epsilon(double) is a positive number that controls the level of noise in the sketch, as described in [Hehir2023]. Smaller values of epsilon correspond to noisier sketches.buckets(int) is the number of buckets in the SFM PCSA sketch as described in [Hehir2023].precision(int) defaults to 24.
Note
Like
noisy_approx_set_sfm(), this function returnsNULLwhencol_indexorcol_zerosisNULL. If this behavior is undesirable, usecoalesce()withnoisy_empty_approx_set_sfm().
- cardinality(SfmSketch) bigint¶
Returns the estimated cardinality (distinct count) of an
SfmSketchobject.
- merge(SfmSketch) SfmSketch¶
An aggregator function that returns a merged
SfmSketchof the set union of individualSfmSketchobjects, similar tomerge(HyperLogLog).SELECT year, cardinality(merge(sketch)) AS annual_distinct_count FROM monthly_sketches GROUP BY 1
- merge_sfm(ARRAY[SfmSketch, ...]) SfmSketch¶
A scalar function that returns a merged
SfmSketchof the set union of an array ofSfmSketchobjects, similar tomerge_hll().SELECT cardinality(merge_sfm(ARRAY[ noisy_approx_set_sfm(col_1, 5.0), noisy_approx_set_sfm(col_2, 5.0), noisy_approx_set_sfm(col_3, 5.0) ])) AS distinct_count_over_3_cols FROM my_table
Limitations¶
While these functions resemble differential privacy mechanisms, the values returned by these functions are not differentially private in general. There are several important limitations to keep in mind if using these functions for privacy-preserving purposes, including:
All noisy aggregate functions return
NULLwhen aggregating empty sets. This means aNULLreturn value noiselessly indicates the absence of data.GROUP BYclauses used in combination with noisy aggregation functions reveal non-noisy information: the presence or absence of a group noiselessly indicates the presence or absence of data. See, e.g., [Wilkins2024].Functions relying on floating-point noise may be susceptible to inference attacks such as those identified in [Mironov2012] and [Casacuberta2022].
References¶
Casacuberta, S., Shoemate, M., Vadhan, S., & Wagaman, C. (2022). Widespread Underestimation of Sensitivity in Differentially Private Libraries and How to Fix It. In Proceedings of the 2022 ACM SIGSAC Conference on Computer and Communications Security (pp. 471-484).
Hehir, J., Ting, D., & Cormode, G. (2023). Sketch-Flip-Merge: Mergeable Sketches for Private Distinct Counting. In Proceedings of the 40th International Conference on Machine Learning (Vol. 202).
Mironov, I. (2012). On significance of the least significant bits for differential privacy. In Proceedings of the 2012 ACM Conference on Computer and Communications Security (pp. 650-661).
Wilkins, A., Kifer, D., Zhang, D., & Karrer, B. (2024). Exact Privacy Analysis of the Gaussian Sparse Histogram Mechanism. Journal of Privacy and Confidentiality, 14 (1).
Flajolet, P, Martin, G. N. (1985). Probabilistic Counting Algorithms for Data Base Applications. In Journal of Computer and System Sciences, 31:182-209, 1985
Miscellaneous¶
- max_data_size_for_stats(x) bigint¶
Returns an estimate of the the maximum in-memory size in bytes of
x.
- sum_data_size_for_stats(x) bigint¶
Returns an estimate of the sum of in-memory size in bytes of
x.