github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/dofn_lifecycle_test.py (about)

     1  #
     2  # Licensed to the Apache Software Foundation (ASF) under one or more
     3  # contributor license agreements.  See the NOTICE file distributed with
     4  # this work for additional information regarding copyright ownership.
     5  # The ASF licenses this file to You under the Apache License, Version 2.0
     6  # (the "License"); you may not use this file except in compliance with
     7  # the License.  You may obtain a copy of the License at
     8  #
     9  #    http://www.apache.org/licenses/LICENSE-2.0
    10  #
    11  # Unless required by applicable law or agreed to in writing, software
    12  # distributed under the License is distributed on an "AS IS" BASIS,
    13  # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    14  # See the License for the specific language governing permissions and
    15  # limitations under the License.
    16  #
    17  
    18  """UnitTests for DoFn lifecycle and bundle methods"""
    19  
    20  # pytype: skip-file
    21  
    22  import unittest
    23  
    24  import pytest
    25  
    26  import apache_beam as beam
    27  from apache_beam.testing.test_pipeline import TestPipeline
    28  
    29  
    30  class CallSequenceEnforcingDoFn(beam.DoFn):
    31    def __init__(self):
    32      self._setup_called = False
    33      self._start_bundle_calls = 0
    34      self._finish_bundle_calls = 0
    35      self._teardown_called = False
    36  
    37    def setup(self):
    38      assert not self._setup_called, 'setup should not be called twice'
    39      assert self._start_bundle_calls == 0, \
    40        'setup should be called before start_bundle'
    41      assert self._finish_bundle_calls == 0, \
    42        'setup should be called before finish_bundle'
    43      assert not self._teardown_called, 'setup should be called before teardown'
    44      self._setup_called = True
    45  
    46    def start_bundle(self):
    47      assert self._setup_called, 'setup should have been called'
    48      assert self._start_bundle_calls == self._finish_bundle_calls, \
    49        'there should be as many start_bundle calls as finish_bundle calls'
    50      assert not self._teardown_called, 'teardown should not have been called'
    51      self._start_bundle_calls += 1
    52  
    53    def process(self, element):
    54      assert self._setup_called, 'setup should have been called'
    55      assert self._start_bundle_calls > 0, 'start_bundle should have been called'
    56      assert self._start_bundle_calls == self._finish_bundle_calls + 1, \
    57        'there should be one start_bundle call with no call to finish_bundle'
    58      assert not self._teardown_called, 'teardown should not have been called'
    59      return [element * element]
    60  
    61    def finish_bundle(self):
    62      assert self._setup_called, 'setup should have been called'
    63      assert self._start_bundle_calls > 0, 'start_bundle should have been called'
    64      assert self._start_bundle_calls == self._finish_bundle_calls + 1, \
    65        'there should be one start_bundle call with no call to finish_bundle'
    66      assert not self._teardown_called, 'teardown should not have been called'
    67      self._finish_bundle_calls += 1
    68  
    69    def teardown(self):
    70      assert self._setup_called, 'setup should have been called'
    71      assert self._start_bundle_calls == self._finish_bundle_calls, \
    72        'there should be as many start_bundle calls as finish_bundle calls'
    73      assert not self._teardown_called, 'teardown should not be called twice'
    74      self._teardown_called = True
    75  
    76  
    77  @pytest.mark.it_validatesrunner
    78  class DoFnLifecycleTest(unittest.TestCase):
    79    def test_dofn_lifecycle(self):
    80      with TestPipeline() as p:
    81        _ = (
    82            p
    83            | 'Start' >> beam.Create([1, 2, 3])
    84            | 'Do' >> beam.ParDo(CallSequenceEnforcingDoFn()))
    85      # Assumes that the worker is run in the same process as the test.
    86  
    87  
    88  class LocalDoFnLifecycleTest(unittest.TestCase):
    89    def test_dofn_lifecycle(self):
    90      from apache_beam.runners.direct import direct_runner
    91      from apache_beam.runners.portability import fn_api_runner
    92      runners = [
    93          direct_runner.BundleBasedDirectRunner(), fn_api_runner.FnApiRunner()
    94      ]
    95      for r in runners:
    96        with TestPipeline(runner=r) as p:
    97          _ = (
    98              p
    99              | 'Start' >> beam.Create([1, 2, 3])
   100              | 'Do' >> beam.ParDo(CallSequenceEnforcingDoFn()))
   101        # Assumes that the worker is run in the same process as the test.
   102  
   103  
   104  if __name__ == '__main__':
   105    unittest.main()