[ 
https://issues.apache.org/jira/browse/BEAM-6693?focusedWorklogId=241996&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-241996
 ]

ASF GitHub Bot logged work on BEAM-6693:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 14/May/19 19:54
            Start Date: 14/May/19 19:54
    Worklog Time Spent: 10m 
      Work Description: pabloem commented on pull request #8535: [BEAM-6693] 
ApproximateUnique transform for Python SDK
URL: https://github.com/apache/beam/pull/8535#discussion_r283547926
 
 

 ##########
 File path: sdks/python/apache_beam/transforms/stats_test.py
 ##########
 @@ -0,0 +1,384 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from __future__ import absolute_import
+from __future__ import division
+
+import math
+import random
+import unittest
+from collections import defaultdict
+
+import numpy as np
+
+import apache_beam as beam
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+
+
+class ApproximateUniqueTest(unittest.TestCase):
+  """Unit tests for ApproximateUniqueGlobally and ApproximateUniquePerKey."""
+
+  def test_approximate_unique_global_by_invalid_size(self):
+    # test if the transformation throws an error as expected with an invalid
+    # small input size (< 16).
+    sample_size = 10
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+
+    with self.assertRaises(ValueError) as e:
+      pipeline = TestPipeline()
+      _ = (pipeline
+           | 'create'
+           >> beam.Create(test_input)
+           | 'get_estimate'
+           >> beam.ApproximateUniqueGlobally(size=sample_size))
+      pipeline.run()
+
+    expected_msg = beam.ApproximateUniqueGlobally._INPUT_SIZE_ERR_MSG % (
+        sample_size)
+
+    assert e.exception.args[0] == expected_msg
+
+  def test_approximate_unique_global_by_invalid_type_size(self):
+    # test if the transformation throws an error as expected with an invalid
+    # type of input size (not int).
+    sample_size = 100.0
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+
+    with self.assertRaises(ValueError) as e:
+      pipeline = TestPipeline()
+      _ = (pipeline
+           | 'create' >> beam.Create(test_input)
+           | 'get_estimate'
+           >> beam.ApproximateUniqueGlobally(size=sample_size))
+      pipeline.run()
+
+    expected_msg = beam.ApproximateUniqueGlobally._INPUT_SIZE_ERR_MSG % (
+        sample_size)
+
+    assert e.exception.args[0] == expected_msg
+
+  def test_approximate_unique_global_by_invalid_small_error(self):
+    # test if the transformation throws an error as expected with an invalid
+    # small input error (< 0.01).
+    est_err = 0.0
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+
+    with self.assertRaises(ValueError) as e:
+      pipeline = TestPipeline()
+      _ = (pipeline
+           | 'create' >> beam.Create(test_input)
+           | 'get_estimate'
+           >> beam.ApproximateUniqueGlobally(error=est_err))
+      pipeline.run()
+
+    expected_msg = beam.ApproximateUniqueGlobally._INPUT_ERROR_ERR_MSG % (
+        est_err)
+
+    assert e.exception.args[0] == expected_msg
+
+  def test_approximate_unique_global_by_invalid_big_error(self):
+    # test if the transformation throws an error as expected with an invalid
+    # big input error (> 0.50).
+    est_err = 0.6
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+
+    with self.assertRaises(ValueError) as e:
+      pipeline = TestPipeline()
+      _ = (pipeline
+           | 'create' >> beam.Create(test_input)
+           | 'get_estimate'
+           >> beam.ApproximateUniqueGlobally(error=est_err))
+      pipeline.run()
+
+    expected_msg = beam.ApproximateUniqueGlobally._INPUT_ERROR_ERR_MSG % (
+        est_err)
+
+    assert e.exception.args[0] == expected_msg
+
+  def test_approximate_unique_global_by_invalid_no_input(self):
+    # test if the transformation throws an error as expected with no input.
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+
+    with self.assertRaises(ValueError) as e:
+      pipeline = TestPipeline()
+      _ = (pipeline
+           | 'create' >> beam.Create(test_input)
+           | 'get_estimate'
+           >> beam.ApproximateUniqueGlobally())
+      pipeline.run()
+
+    expected_msg = beam.ApproximateUniqueGlobally._NO_VALUE_ERR_MSG
+    assert e.exception.args[0] == expected_msg
+
+  def test_approximate_unique_global_by_invalid_both_input(self):
+    # test if the transformation throws an error as expected with multi input.
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+    est_err = 0.2
+    sample_size = 30
+
+    with self.assertRaises(ValueError) as e:
+      pipeline = TestPipeline()
+      _ = (pipeline
+           | 'create' >> beam.Create(test_input)
+           | 'get_estimate'
+           >> beam.ApproximateUniqueGlobally(size=sample_size, error=est_err))
+      pipeline.run()
+
+    expected_msg = beam.ApproximateUniqueGlobally._MULTI_VALUE_ERR_MSG % (
+        sample_size, est_err)
+
+    assert e.exception.args[0] == expected_msg
+
+  def test_get_sample_size_from_est_error(self):
+    # test if get correct sample size from input error.
+    assert beam.ApproximateUniqueGlobally.\
+      _get_sample_size_from_est_error(0.5) == 16
+    assert beam.ApproximateUniqueGlobally.\
+      _get_sample_size_from_est_error(0.4) == 25
+    assert beam.ApproximateUniqueGlobally.\
+      _get_sample_size_from_est_error(0.2) == 100
+    assert beam.ApproximateUniqueGlobally.\
+      _get_sample_size_from_est_error(0.1) == 400
+    assert beam.ApproximateUniqueGlobally.\
+      _get_sample_size_from_est_error(0.05) == 1600
+    assert beam.ApproximateUniqueGlobally.\
+      _get_sample_size_from_est_error(0.01) == 40000
+
+  def test_approximate_unique_global_by_sample_size(self):
+    # test if estimation error with a given sample size is not greater than
+    # expected max error (sample size = 50% of population).
+    sample_size = 50
+    max_err = 2 / math.sqrt(sample_size)
+    test_input = [random.randint(0, 1000) for _ in range(100)]
+    actual_count = len(set(test_input))
+
+    pipeline = TestPipeline()
+    result = (pipeline
+              | 'create' >> beam.Create(test_input)
+              | 'get_estimate'
+              >> beam.ApproximateUniqueGlobally(size=sample_size)
+              | 'compare'
+              >> beam.ParDo(lambda x: [abs(x - actual_count) * 1.0
 
 Review comment:
   Please use `beam.FlatMap` instead of `beam.ParDo`.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 241996)
    Time Spent: 4h 10m  (was: 4h)

> ApproximateUnique transform for Python SDK
> ------------------------------------------
>
>                 Key: BEAM-6693
>                 URL: https://issues.apache.org/jira/browse/BEAM-6693
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-py-core
>            Reporter: Ahmet Altay
>            Assignee: Hannah Jiang
>            Priority: Minor
>          Time Spent: 4h 10m
>  Remaining Estimate: 0h
>
> Add a PTransform for estimating the number of distinct elements in a 
> PCollection and the number of distinct values associated with each key in a 
> PCollection KVs.
> it should offer the same API as its Java counterpart: 
> https://github.com/apache/beam/blob/11a977b8b26eff2274d706541127c19dc93131a2/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateUnique.java



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to