Skip to content

Commit

Permalink
Histogram error estimator (#458)
Browse files Browse the repository at this point in the history
  • Loading branch information
dvadym authored Jul 13, 2023
1 parent 0b2cdd9 commit 45c6acc
Show file tree
Hide file tree
Showing 4 changed files with 309 additions and 4 deletions.
158 changes: 158 additions & 0 deletions pipeline_dp/dataset_histograms/histogram_error_estimator.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
# Copyright 2023 OpenMined.
#
# 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.
"""Estimation of errors from DatasetHistograms."""
import pipeline_dp
from pipeline_dp.dataset_histograms import histograms as hist
from typing import Optional, Sequence, Tuple
import math
import bisect


class CountErrorEstimator:
"""Estimator of the error from DP pipeline from DatasetHistograms.
The recommended way to create this object is to use create_error_estimator.
It works only for COUNT and PRIVACY_ID_COUNT.
Partition selection error is not implemented yet. Now only contribution
bounding and noise error are taken into consideration.
"""

def __init__(self, base_std: float, metric: pipeline_dp.Metrics,
noise: pipeline_dp.NoiseKind,
l0_ratios_dropped: Sequence[Tuple[int, float]],
linf_ratios_dropped: Sequence[Tuple[int, float]],
partition_histogram: hist.Histogram):
self._base_std = base_std
self._metric = metric
self._noise = noise
self._l0_ratios_dropped = l0_ratios_dropped
self._linf_ratios_dropped = linf_ratios_dropped
self._partition_histogram = partition_histogram

def estimate_rmse(self,
l0_bound: int,
linf_bound: Optional[int] = None) -> float:
"""Estimates RMSE error for given l0 and linf bounds.
Estimation algorithm is the following:
1. From l0_bound and l0_contributions_histogram the ratio data dropped
from l0 contribution bounding is computed.
2. From linf_bound and linf_contributions_histogram the
ratio data dropped from linf contribution bounding is computed.
3. The total 'ratio_data_dropped' for contribution bounding is estimated
from l0 and linf ratios data dropped.
4. Then under the assumption that contribution bounding drops data
uniformly on all partitions, for a partition of the size n, it is
assumed that n*ratio_data_dropped data points are dropped with
contribution bounding. And RMSE for this partition is computed as
sqrt((n*ratio_data_dropped)**2 + noise_std**2)
5. RMSE are averaged across all partitions.
Args:
l0_bound: l0 contribution bound, AKA max_partition_contributed.
linf_bound: linf contribution bound, AKA for COUNT as
max_contributions_per_partition. This parameter is ignored for
PRIVACY_ID_COUNT
Returns:
the estimated error.
"""
if self._metric == pipeline_dp.Metrics.COUNT:
if linf_bound is None:
raise ValueError("linf must be given for COUNT")
ratio_dropped_l0 = self.get_ratio_dropped_l0(l0_bound)
ratio_dropped_linf = 0
if self._metric == pipeline_dp.Metrics.COUNT:
ratio_dropped_linf = self.get_ratio_dropped_linf(linf_bound)
ratio_dropped = 1 - (1 - ratio_dropped_l0) * (1 - ratio_dropped_linf)
stddev = self._get_stddev(l0_bound, linf_bound)
return _estimate_rmse_impl(ratio_dropped, stddev,
self._partition_histogram)

def get_ratio_dropped_l0(self, l0_bound: int) -> float:
"""Computes ratio"""
return self._get_ratio_dropped(self._l0_ratios_dropped, l0_bound)

def get_ratio_dropped_linf(self, linf_bound: int) -> float:
return self._get_ratio_dropped(self._linf_ratios_dropped, linf_bound)

def _get_ratio_dropped(self, ratios_dropped: Sequence[Tuple[int, float]],
bound: int) -> float:
if bound <= 0:
return 1
if bound > ratios_dropped[-1][0]:
return 0
index = bisect.bisect_left(ratios_dropped, (bound, 0))
if ratios_dropped[index][0] == bound:
return ratios_dropped[index][1]

# index > 0, because ratio_dropped starts from 0, and bound > 0.
x1, y1 = ratios_dropped[index - 1]
x2, y2 = ratios_dropped[index]
# Linearly interpolate between (x1, y1) and (x2, y2) for x=bound.
return (y1 * (x2 - bound) + y2 * (bound - x1)) / (x2 - x1)

def _get_stddev(self,
l0_bound: int,
linf_bound: Optional[int] = None) -> float:
if self._metric == pipeline_dp.Metrics.PRIVACY_ID_COUNT:
linf_bound = 1
if self._noise == pipeline_dp.NoiseKind.LAPLACE:
return self._base_std * l0_bound * linf_bound
# Gaussian noise case.
return self._base_std * math.sqrt(l0_bound) * linf_bound


def create_error_estimator(histograms: hist.DatasetHistograms, base_std: float,
metric: pipeline_dp.Metric,
noise: pipeline_dp.NoiseKind) -> CountErrorEstimator:
"""Creates histogram based error estimator for COUNT or PRIVACY_ID_COUNT.
Args:
histograms: dataset histograms.
base_std: what's standard deviation of the noise, when l0 and linf
bounds equal to 1.
metric: DP aggregation, COUNT or PRIVACY_ID_COUNT.
noise: type of DP noise.
Returns:
Error estimator.
"""
if metric not in [
pipeline_dp.Metrics.COUNT, pipeline_dp.Metrics.PRIVACY_ID_COUNT
]:
raise ValueError(
f"Only COUNT and PRIVACY_ID_COUNT are supported, but metric={metric}"
)
l0_ratios_dropped = hist.compute_ratio_dropped(
histograms.l0_contributions_histogram)
linf_ratios_dropped = hist.compute_ratio_dropped(
histograms.linf_contributions_histogram)
if metric == pipeline_dp.Metrics.COUNT:
partition_histogram = histograms.count_per_partition_histogram
else:
partition_histogram = histograms.count_privacy_id_per_partition
return CountErrorEstimator(base_std, metric, noise, l0_ratios_dropped,
linf_ratios_dropped, partition_histogram)


def _estimate_rmse_impl(ratio_dropped: float, std: float,
partition_histogram: hist.Histogram) -> float:
sum_rmse = 0
num_partitions = partition_histogram.total_count()
for bin in partition_histogram.bins:
average_partition_size_in_bin = bin.sum / bin.count
rmse = math.sqrt((ratio_dropped * average_partition_size_in_bin)**2 +
std**2)
sum_rmse += bin.count * rmse
return sum_rmse / num_partitions
13 changes: 12 additions & 1 deletion pipeline_dp/dataset_histograms/histograms.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,17 @@ def __eq__(self, other):


class HistogramType(enum.Enum):
# For L0 contribution histogram, for each bin:
# 'count' is the number of privacy units which contribute to
# [lower, next_lower) partitions.
# 'sum' is the total number (privacy_unit, partition) for these privacy
# units.
L0_CONTRIBUTIONS = 'l0_contributions'
L1_CONTRIBUTIONS = 'l1_contributions'
# For Linf contribution histogram, for each bin:
# 'count' is the number of pairs (privacy_unit, partition) which contribute
# with [lower, next_lower) contributions.
# 'sum' is the total number of contributions for these pairs.
LINF_CONTRIBUTIONS = 'linf_contributions'
COUNT_PER_PARTITION = 'count_per_partition'
COUNT_PRIVACY_ID_PER_PARTITION = 'privacy_id_per_partition_count'
Expand Down Expand Up @@ -112,7 +121,8 @@ def compute_ratio_dropped(
For each FrequencyBin.lower in contribution_histogram it computes what would
the ratio of data dropped because of contribution bounding when it is taken
as bounding threshold (e.g. in case of L0 histogram bounding_threshold it is
max_partition_contribution).
max_partition_contribution). For convenience the (0, 1) is added as 1st
element.
Args:
contribution_histogram: histogram of contributions. It can be L0, L1,
Expand Down Expand Up @@ -142,6 +152,7 @@ def compute_ratio_dropped(
ratio_dropped.append((current_value, dropped / total_sum))
previous_value = current_value
elements_larger += bin.count
ratio_dropped.append((0, 1))
return ratio_dropped[::-1]


Expand Down
136 changes: 136 additions & 0 deletions tests/dataset_histograms/histogram_error_estimator_test.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
# Copyright 2023 OpenMined.
#
# 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.
"""Tests for histogram error estimator."""

from absl.testing import absltest
from absl.testing import parameterized

import pipeline_dp
from pipeline_dp.dataset_histograms import histograms as hist
from pipeline_dp.dataset_histograms import computing_histograms
from pipeline_dp.dataset_histograms import histogram_error_estimator


class HistogramErrorEstimatorTest(parameterized.TestCase):

def _get_histograms(self) -> hist.DatasetHistograms:
# Generate dataset
dataset = []
# 1st privacy unit contributes to 10 partitions once
dataset.extend([(1, i) for i in range(10)])
# 2nd privacy unit contributes to 1 partition 20 times.
dataset.extend([(2, 0) for i in range(20)])

data_extractors = pipeline_dp.DataExtractors(
privacy_id_extractor=lambda x: x[0],
partition_extractor=lambda x: x[1])
return list(
computing_histograms.compute_dataset_histograms(
dataset, data_extractors, pipeline_dp.LocalBackend()))[0]

def _get_estimator(
self,
metric: pipeline_dp.Metric,
noise_kind: pipeline_dp.NoiseKind = pipeline_dp.NoiseKind.LAPLACE,
base_std: float = 2.0):
return histogram_error_estimator.create_error_estimator(
self._get_histograms(), base_std, metric, noise_kind)

@parameterized.named_parameters(
dict(testcase_name='count_gaussian',
metric=pipeline_dp.Metrics.COUNT,
noise_kind=pipeline_dp.NoiseKind.GAUSSIAN,
base_std=2.0,
l0=9,
linf=5,
expected=30),
dict(testcase_name='count_laplace',
metric=pipeline_dp.Metrics.COUNT,
noise_kind=pipeline_dp.NoiseKind.LAPLACE,
base_std=2.0,
l0=9,
linf=5,
expected=90),
dict(testcase_name='privacy_id_count_gaussian',
metric=pipeline_dp.Metrics.PRIVACY_ID_COUNT,
noise_kind=pipeline_dp.NoiseKind.GAUSSIAN,
base_std=1.5,
l0=9,
linf=5,
expected=4.5),
dict(testcase_name='privacy_id_count_laplace',
metric=pipeline_dp.Metrics.PRIVACY_ID_COUNT,
noise_kind=pipeline_dp.NoiseKind.LAPLACE,
base_std=1.5,
l0=9,
linf=5,
expected=13.5),
)
def test_count_get_sigma(self, metric: pipeline_dp.Metric, base_std: float,
noise_kind: pipeline_dp.NoiseKind, l0: float,
linf: float, expected: float):
estimator = self._get_estimator(metric=metric,
base_std=base_std,
noise_kind=noise_kind)
self.assertAlmostEqual(estimator._get_stddev(l0, linf),
expected,
delta=1e-10)

def test_sum_not_supported(self):
with self.assertRaisesRegex(
ValueError, "Only COUNT and PRIVACY_ID_COUNT are supported"):
self._get_estimator(pipeline_dp.Metrics.SUM)

@parameterized.parameters((0, 1), (1, 9 / 11), (2, 8 / 11), (3, 7 / 11),
(9, 1 / 11), (10, 0), (20, 0))
# there are 11 (privacy_id, partition) pairs (from 2 privacy units), when
# l0_bound=1, 9 are dropped (from 1 privacy unit).
def test_get_ratio_dropped_l0(self, l0_bound, expected):
estimator = self._get_estimator(pipeline_dp.Metrics.COUNT)
self.assertAlmostEqual(estimator.get_ratio_dropped_l0(l0_bound),
expected)

@parameterized.parameters((0, 1), (1, 19 / 30), (2, 18 / 30), (10, 10 / 30),
(20, 0), (21, 0))
# there are 30 rows (from 2 privacy units), when linf_bound=1, 19 are
# dropped (from 1 privacy unit, which contributes 20 to 1 partition).
def test_get_ratio_dropped_linf(self, linf_bound, expected):
estimator = self._get_estimator(pipeline_dp.Metrics.COUNT)
self.assertAlmostEqual(estimator.get_ratio_dropped_linf(linf_bound),
expected)

@parameterized.parameters((1, 1, 3.9565310998335823),
(1, 2, 5.683396971098993),
(10, 10, 200.01249625055996))
# This is explanation how estimation is computed. See _get_histograms
# for dataset description.
# l0_bound = linf_bound = 1
# ratio_dropped_l0 = 9/11, ratio_dropped_linf = 19/30.
# total_ratio_dropped is estimated as 1 - (1 - 9/11)*(1 - 19/30) ~= 0.933333
# noise_stddev = 2
# RMSE is estimated separately on partitions with 1 row and on the partition
# with 21 rows.
# On a partition with 1 row (9 such partitions):
# rmse1 = sqrt(1*total_ratio_dropped + noise_stddev**2) ~= 2.20706
# On a partition with 21 row:
# rmse2 = sqrt(21*total_ratio_dropped + noise_stddev**2) ~= 19.70177
# rmse = (9*rmse1+rmse2)/10.
def test_estimate_rmse_count(self, l0_bound, linf_bound, expected):
estimator = self._get_estimator(pipeline_dp.Metrics.COUNT)
self.assertAlmostEqual(estimator.estimate_rmse(l0_bound, linf_bound),
expected)


if __name__ == '__main__':
absltest.main()
6 changes: 3 additions & 3 deletions tests/dataset_histograms/histograms_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ def test_quantile_contributions(self, bins, q, expected_quantiles):
bins=[
hist.FrequencyBin(lower=1000, count=10, sum=10100, max=1020),
],
expected_ratios=[(1000, 100 / 10100), (1020, 0.0)]),
expected_ratios=[(0, 1), (1000, 100 / 10100), (1020, 0.0)]),
dict(testcase_name='7 bins histogram',
bins=[
hist.FrequencyBin(lower=1, count=8, sum=8, max=1),
Expand All @@ -61,8 +61,8 @@ def test_quantile_contributions(self, bins, q, expected_quantiles):
hist.FrequencyBin(lower=6, count=1, sum=6, max=6),
hist.FrequencyBin(lower=11, count=1, sum=11, max=11),
],
expected_ratios=[(1, 0.66), (2, 0.48), (3, 0.34), (4, 0.22),
(5, 0.14), (6, 0.1), (11, 0.0)]))
expected_ratios=[(0, 1), (1, 0.66), (2, 0.48), (3, 0.34),
(4, 0.22), (5, 0.14), (6, 0.1), (11, 0.0)]))
def test_ratio_dropped(self, bins, expected_ratios):
histogram = hist.Histogram("name", bins)
output = hist.compute_ratio_dropped(histogram)
Expand Down

0 comments on commit 45c6acc

Please sign in to comment.