Skip to content

Commit

Permalink
Make PeriodicImpulse generates unbounded PCollection (apache#21815)
Browse files Browse the repository at this point in the history
  • Loading branch information
Abacn authored and bullet03 committed Jun 20, 2022
1 parent 845a5cf commit eeeda86
Show file tree
Hide file tree
Showing 6 changed files with 36 additions and 6 deletions.
8 changes: 5 additions & 3 deletions sdks/python/apache_beam/pvalue.py
Original file line number Diff line number Diff line change
Expand Up @@ -166,14 +166,16 @@ def __reduce_ex__(self, unused_version):
return _InvalidUnpickledPCollection, ()

@staticmethod
def from_(pcoll):
# type: (PValue) -> PCollection
def from_(pcoll, is_bounded=None):
# type: (PValue, Optional[bool]) -> PCollection

"""Create a PCollection, using another PCollection as a starting point.
Transfers relevant attributes.
"""
return PCollection(pcoll.pipeline, is_bounded=pcoll.is_bounded)
if is_bounded is None:
is_bounded = pcoll.is_bounded
return PCollection(pcoll.pipeline, is_bounded=is_bounded)

def to_runner_api(self, context):
# type: (PipelineContext) -> beam_runner_api_pb2.PCollection
Expand Down
18 changes: 18 additions & 0 deletions sdks/python/apache_beam/runners/common_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,24 @@ def process(self, element, mykey=DoFn.KeyParam):
test_stream = (TestStream().advance_watermark_to(10).add_elements([1, 2]))
(p | test_stream | beam.ParDo(DoFnProcessWithKeyparam()))

def test_pardo_with_unbounded_per_element_dofn(self):
class UnboundedDoFn(beam.DoFn):
@beam.DoFn.unbounded_per_element()
def process(self, element):
pass

class BoundedDoFn(beam.DoFn):
def process(self, element):
pass

with TestPipeline() as p:
source = p | beam.Impulse()
unbounded_pcoll = source | beam.ParDo(UnboundedDoFn())
bounded_pcoll = source | beam.ParDo(BoundedDoFn())

self.assertEqual(unbounded_pcoll.is_bounded, False)
self.assertEqual(bounded_pcoll.is_bounded, True)


class TestOffsetRestrictionProvider(RestrictionProvider):
def restriction_size(self, element, restriction):
Expand Down
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/runners/sdf_utils.py
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@


class ThreadsafeRestrictionTracker(object):
"""A thread-safe wrapper which wraps a `RestritionTracker`.
"""A thread-safe wrapper which wraps a `RestrictionTracker`.
This wrapper guarantees synchronization of modifying restrictions across
multi-thread.
Expand Down
7 changes: 6 additions & 1 deletion sdks/python/apache_beam/transforms/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -1482,9 +1482,14 @@ def expand(self, pcoll):
key_coder,
self)

if self._signature.is_unbounded_per_element():
is_bounded = False
else:
is_bounded = pcoll.is_bounded

self.infer_batch_converters(pcoll.element_type)

return pvalue.PCollection.from_(pcoll)
return pvalue.PCollection.from_(pcoll, is_bounded=is_bounded)

def with_outputs(self, *tags, main=None, allow_unknown_tags=None):
"""Returns a tagged tuple allowing access to the outputs of a
Expand Down
3 changes: 3 additions & 0 deletions sdks/python/apache_beam/transforms/periodicsequence.py
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ class ImpulseSeqGenDoFn(beam.DoFn):
ImpulseSeqGenDoFn guarantees that elements would not be output prior to
given runtime timestamp.
'''
@beam.DoFn.unbounded_per_element()
def process(
self,
element,
Expand Down Expand Up @@ -123,6 +124,7 @@ class PeriodicSequence(PTransform):
PeriodicSequence can't guarantee that each element is output at exact time.
PeriodicSequence guarantees that elements would not be output prior to given
runtime timestamp.
The PCollection generated by PeriodicSequence is unbounded.
'''

# pylint: disable=unused-private-member
Expand All @@ -143,6 +145,7 @@ class PeriodicImpulse(PTransform):
PeriodicImpulse transform behaves same as {@link PeriodicSequence} transform,
but can be used as first transform in pipeline.
The PCollection generated by PeriodicImpulse is unbounded.
'''
def __init__(
self,
Expand Down
4 changes: 3 additions & 1 deletion sdks/python/apache_beam/transforms/periodicsequence_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ def test_periodicsequence_outputs_valid_sequence(self):
start_time + x * interval
for x in range(0, int(duration / interval), 1)
]

self.assertEqual(result.is_bounded, False)
assert_that(result, equal_to(k))

def test_periodicimpulse_windowing_on_si(self):
Expand Down Expand Up @@ -92,6 +92,7 @@ def test_periodicimpulse_default_start(self):
result = p | 'PeriodicImpulse' >> PeriodicImpulse(it, et, interval)

k = [it + x * interval for x in range(0, int(duration / interval))]
self.assertEqual(result.is_bounded, False)
assert_that(result, equal_to(k))

def test_periodicsequence_outputs_valid_sequence_in_past(self):
Expand All @@ -108,6 +109,7 @@ def test_periodicsequence_outputs_valid_sequence_in_past(self):
| 'ImpulseSeqGen' >> PeriodicSequence())

k = [it + x * interval for x in range(0, int(duration / interval), 1)]
self.assertEqual(result.is_bounded, False)
assert_that(result, equal_to(k))


Expand Down

0 comments on commit eeeda86

Please sign in to comment.