Skip to content

[Bug]: TestPipeline fails with DEBUG log level and pytest #37386

@m-morozov-kw

Description

@m-morozov-kw

What happened?

Hello!

We test our code with different log levels, and we noticed that test pipeline fails when the debug log level is set.

To reproduce, do:

  1. use Python 3.13(not sure that is important here)
  2. pip install "apache-beam[gcp]==2.70.*" pytest
  3. add code from [1] to some file like /tmp/1.py
  4. run tests pytest --log-cli-level=DEBUG /tmp/1.py
  5. You should get the error from [2]

Please note that the --log-cli-level=INFO option works. Please let me know if my code setup in [1] is incorrect. Thank you.

[1]

import apache_beam as beam
from apache_beam import Create, ParDo
from apache_beam.testing.test_pipeline import TestPipeline


def test_job():
    class DoFN(beam.DoFn):
        def process(self, element: tuple[str, str, dict], *args, **kwargs):
            print(f"Processing element: {element}")
            return [None]

    with TestPipeline() as pipeline:
        _ = (
            pipeline
            | "1" >> Create([("data", "message", {"key": "value"})])
            | "2" >> ParDo(DoFN())
        )

[2] error:

click to expand

Error details

============================================================================================================== short test summary info ==============================================================================================================
FAILED ../../../../../tmp/1.py::test_job - RuntimeError: Pipeline job-001 failed in state FAILED: bundle inst001 stage-001 failed:Traceback (most recent call last):
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/runners/worker/sdk_worker.py", line 316, in _execute
    response = task()
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/runners/worker/sdk_worker.py", line 390, in <lambda>
    lambda: self.create_worker().do_instruction(request), request)
            ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~^^^^^^^^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/runners/worker/sdk_worker.py", line 669, in do_instruction
    return getattr(self, request_type)(
           ~~~~~~~~~~~~~~~~~~~~~~~~~~~^
        getattr(request, request_type), request.instruction_id)
        ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/runners/worker/sdk_worker.py", line 707, in process_bundle
    bundle_processor.process_bundle(instruction_id))
    ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~^^^^^^^^^^^^^^^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/runners/worker/bundle_processor.py", line 1254, in process_bundle
    _LOGGER.debug('start %s', op)
    ~~~~~~~~~~~~~^^^^^^^^^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1511, in debug
    self._log(DEBUG, msg, args, **kwargs)
    ~~~~~~~~~^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1668, in _log
    self.handle(record)
    ~~~~~~~~~~~^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1684, in handle
    self.callHandlers(record)
    ~~~~~~~~~~~~~~~~~^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1740, in callHandlers
    hdlr.handle(record)
    ~~~~~~~~~~~^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1030, in handle
    self.emit(record)
    ~~~~~~~~~^^^^^^^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/_pytest/logging.py", line 384, in emit
    super().emit(record)
    ~~~~~~~~~~~~^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1162, in emit
    self.handleError(record)
    ~~~~~~~~~~~~~~~~^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1154, in emit
    msg = self.format(record)
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 1002, in format
    return fmt.format(record)
           ~~~~~~~~~~^^^^^^^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/_pytest/logging.py", line 137, in format
    return super().format(record)
           ~~~~~~~~~~~~~~^^^^^^^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 715, in format
    raise e
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 712, in format
    record.message = record.getMessage()
                     ~~~~~~~~~~~~~~~~~^^
  File "/Users/user/.local/share/uv/python/cpython-3.13.0-macos-aarch64-none/lib/python3.13/logging/__init__.py", line 400, in getMessage
    msg = msg % self.args
          ~~~~^~~~~~~~~~~
  File "apache_beam/runners/worker/operations.py", line 665, in apache_beam.runners.worker.operations.Operation.__str__
  File "apache_beam/runners/worker/operations.py", line 691, in apache_beam.runners.worker.operations.Operation.str_internal
  File "apache_beam/runners/worker/operations.py", line 234, in apache_beam.runners.worker.operations.ConsumerSet.__repr__
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/coders/coders.py", line 1580, in __repr__
    f'value_coder={self.value_coder()}]')
                   ~~~~~~~~~~~~~~~~^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/coders/coders.py", line 1575, in value_coder
    return self.wrapped_value_coder.value_coder()
           ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~^^
  File "/Users/user/project/.venv/lib/python3.13/site-packages/apache_beam/coders/coders.py", line 1343, in value_coder
    raise ValueError('TupleCoder does not have exactly 2 components.')
ValueError: TupleCoder does not have exactly 2 components.

Issue Priority

Priority: 2 (default / most bugs should be filed as P2)

Issue Components

  • Component: Python SDK
  • Component: Java SDK
  • Component: Go SDK
  • Component: Typescript SDK
  • Component: IO connector
  • Component: Beam YAML
  • Component: Beam examples
  • Component: Beam playground
  • Component: Beam katas
  • Component: Website
  • Component: Infrastructure
  • Component: Spark Runner
  • Component: Flink Runner
  • Component: Samza Runner
  • Component: Twister2 Runner
  • Component: Hazelcast Jet Runner
  • Component: Google Cloud Dataflow Runner

Metadata

Metadata

Assignees

No one assigned

    Type

    No type

    Projects

    No projects

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions