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

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

                Author: ASF GitHub Bot
            Created on: 10/Jun/20 17:32
            Start Date: 10/Jun/20 17:32
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on a change in pull request #11943:
URL: https://github.com/apache/beam/pull/11943#discussion_r438291777



##########
File path: sdks/python/apache_beam/transforms/combiners_test.py
##########
@@ -105,6 +109,16 @@ def test_builtin_combines(self):
       assert_that(result_mean, equal_to([mean]), label='assert:mean')
       assert_that(result_count, equal_to([size]), label='assert:size')
 
+      # Now for global combines without default
+      timestamped = pcoll | Map(lambda x: TimestampedValue(timestamp))

Review comment:
       TimestampedValue requires the timestamp and the value in the 
constructor. 

##########
File path: sdks/python/apache_beam/transforms/combiners_test.py
##########
@@ -97,6 +100,7 @@ def test_builtin_combines(self):
       vals = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]
       mean = sum(vals) / float(len(vals))
       size = len(vals)
+      timestamp = 1591485720

Review comment:
       Just use a timestamp of 0 or similar. 

##########
File path: sdks/python/apache_beam/transforms/combiners.py
##########
@@ -66,8 +66,21 @@ class Mean(object):
   """Combiners for computing arithmetic means of elements."""
   class Globally(ptransform.PTransform):
     """combiners.Mean.Globally computes the arithmetic mean of the elements."""
+    def __init__(self, has_defaults=True, *args, **kwargs):
+      super(Mean.Globally, self).__init__()
+      self.has_defaults = has_defaults
+      self.args = args
+      self.kwargs = kwargs
+
     def expand(self, pcoll):
-      return pcoll | core.CombineGlobally(MeanCombineFn())
+      if self.has_defaults:
+        return pcoll | core.CombineGlobally(MeanCombineFn())
+      else:
+        return pcoll | core.CombineGlobally(MeanCombineFn()).without_defaults()
+
+    def without_defaults(self):
+      self.has_defaults = False

Review comment:
       Better to return a new copy than mutate self. 




----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 443868)
    Time Spent: 50m  (was: 40m)

> Add without_defaults to Mean 
> -----------------------------
>
>                 Key: BEAM-10209
>                 URL: https://issues.apache.org/jira/browse/BEAM-10209
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-py-core
>    Affects Versions: 2.21.0
>            Reporter: Inigo San Jose Visiers
>            Priority: P2
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> When using Windows and a Global Combiner with need to use 
> `without_defaults()`, this is not possible when using the built-in combiner 
> `Mean`, and the workaround is to do 
> `CombineGlobally(MeanCombineFn()).without_defaults()`. 
> Adding the option to use .without_defaults() directly would help both the 
> code readability and ease of use.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to