Skip to content

Commit

Permalink
[BEAM-5878] avoid using exec for kw args test
Browse files Browse the repository at this point in the history
  • Loading branch information
lazylynx committed Aug 7, 2019
1 parent 599ff8f commit db1c343
Showing 1 changed file with 55 additions and 64 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,94 +20,85 @@
from __future__ import absolute_import

import logging
import sys
import unittest

import apache_beam as beam # pylint: disable=unused-import
from apache_beam.testing.test_pipeline import TestPipeline # pylint: disable=unused-import
from apache_beam.testing.util import assert_that # pylint: disable=unused-import
from apache_beam.testing.util import equal_to # pylint: disable=unused-import
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

IS_PYTHON_2 = sys.version_info[0] == 2


# pylint: disable=exec-used
@unittest.skipIf(IS_PYTHON_2, 'test only in python 3 with kwonly args')
class KeywordOnlyArgsTests(unittest.TestCase):
# TODO(BEAM-7836): using `exec` as work around must be avoided

# Enable nose tests running in parallel
_multiprocess_can_split_ = True

def test_side_input_keyword_only_args(self):
exec('''
pipeline = TestPipeline()
pipeline = TestPipeline()

def sort_with_side_inputs(x, *s, reverse=False):
for y in s:
yield sorted([x] + y, reverse=reverse)
def sort_with_side_inputs(x, *s, reverse=False):
for y in s:
yield sorted([x] + y, reverse=reverse)

pcol = pipeline | 'start' >> beam.Create([1, 2])
side = pipeline | 'side' >> beam.Create([3, 4]) # 2 values in side input.
result1 = pcol | 'compute1' >> beam.FlatMap(
sort_with_side_inputs,
beam.pvalue.AsList(side), reverse=True)
assert_that(result1, equal_to([[4,3,1], [4,3,2]]), label='assert1')
pcol = pipeline | 'start' >> beam.Create([1, 2])
side = pipeline | 'side' >> beam.Create([3, 4]) # 2 values in side input.
result1 = pcol | 'compute1' >> beam.FlatMap(
sort_with_side_inputs,
beam.pvalue.AsList(side), reverse=True)
assert_that(result1, equal_to([[4,3,1], [4,3,2]]), label='assert1')

result2 = pcol | 'compute2' >> beam.FlatMap(
sort_with_side_inputs,
beam.pvalue.AsList(side))
assert_that(result2, equal_to([[1,3,4], [2,3,4]]), label='assert2')
result2 = pcol | 'compute2' >> beam.FlatMap(
sort_with_side_inputs,
beam.pvalue.AsList(side))
assert_that(result2, equal_to([[1,3,4], [2,3,4]]), label='assert2')

result3 = pcol | 'compute3' >> beam.FlatMap(
sort_with_side_inputs)
assert_that(result3, equal_to([]), label='assert3')
result3 = pcol | 'compute3' >> beam.FlatMap(
sort_with_side_inputs)
assert_that(result3, equal_to([]), label='assert3')

result4 = pcol | 'compute4' >> beam.FlatMap(
sort_with_side_inputs, reverse=True)
assert_that(result4, equal_to([]), label='assert4')
result4 = pcol | 'compute4' >> beam.FlatMap(
sort_with_side_inputs, reverse=True)
assert_that(result4, equal_to([]), label='assert4')

pipeline.run()''')
pipeline.run()

def test_combine_keyword_only_args(self):
exec('''
pipeline = TestPipeline()
pipeline = TestPipeline()

def bounded_sum(values, *s, bound=500):
return min(sum(values) + sum(s), bound)
def bounded_sum(values, *s, bound=500):
return min(sum(values) + sum(s), bound)

pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
result1 = pcoll | 'sum1' >> beam.CombineGlobally(bounded_sum, 5, 8, bound=20)
result2 = pcoll | 'sum2' >> beam.CombineGlobally(bounded_sum, 5, 8)
result3 = pcoll | 'sum3' >> beam.CombineGlobally(bounded_sum)
result4 = pcoll | 'sum4' >> beam.CombineGlobally(bounded_sum, bound=5)
pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
result1 = pcoll | 'sum1' >> beam.CombineGlobally(bounded_sum, 5, 8, bound=20)
result2 = pcoll | 'sum2' >> beam.CombineGlobally(bounded_sum, 5, 8)
result3 = pcoll | 'sum3' >> beam.CombineGlobally(bounded_sum)
result4 = pcoll | 'sum4' >> beam.CombineGlobally(bounded_sum, bound=5)

assert_that(result1, equal_to([20]), label='assert1')
assert_that(result2, equal_to([49]), label='assert2')
assert_that(result3, equal_to([10]), label='assert3')
assert_that(result4, equal_to([5]), label='assert4')
assert_that(result1, equal_to([20]), label='assert1')
assert_that(result2, equal_to([49]), label='assert2')
assert_that(result3, equal_to([10]), label='assert3')
assert_that(result4, equal_to([5]), label='assert4')

pipeline.run()''')
pipeline.run()

def test_do_fn_keyword_only_args(self):
exec('''
pipeline = TestPipeline()
class MyDoFn(beam.DoFn):
def process(self, element, *s, bound=500):
return [min(sum(s) + element, bound)]
pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
result1 = pcoll | 'sum1' >> beam.ParDo(MyDoFn(), 5, 8, bound=15)
result2 = pcoll | 'sum2' >> beam.ParDo(MyDoFn(), 5, 8)
result3 = pcoll | 'sum3' >> beam.ParDo(MyDoFn())
result4 = pcoll | 'sum4' >> beam.ParDo(MyDoFn(), bound=5)
assert_that(result1, equal_to([15,15,14]), label='assert1')
assert_that(result2, equal_to([19,16,14]), label='assert2')
assert_that(result3, equal_to([6,3,1]), label='assert3')
assert_that(result4, equal_to([5,3,1]), label='assert4')
pipeline.run()''')
pipeline = TestPipeline()

class MyDoFn(beam.DoFn):
def process(self, element, *s, bound=500):
return [min(sum(s) + element, bound)]

pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
result1 = pcoll | 'sum1' >> beam.ParDo(MyDoFn(), 5, 8, bound=15)
result2 = pcoll | 'sum2' >> beam.ParDo(MyDoFn(), 5, 8)
result3 = pcoll | 'sum3' >> beam.ParDo(MyDoFn())
result4 = pcoll | 'sum4' >> beam.ParDo(MyDoFn(), bound=5)

assert_that(result1, equal_to([15,15,14]), label='assert1')
assert_that(result2, equal_to([19,16,14]), label='assert2')
assert_that(result3, equal_to([6,3,1]), label='assert3')
assert_that(result4, equal_to([5,3,1]), label='assert4')
pipeline.run()


if __name__ == '__main__':
Expand Down

0 comments on commit db1c343

Please sign in to comment.