Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/io/gcp/bigquery_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -967,7 +967,7 @@ def test_to_from_runner_api(self):
schema=schema)

# pylint: disable=expression-not-assigned
p | 'MyWriteToBigQuery' >> original
p | beam.Create([]) | 'MyWriteToBigQuery' >> original

# Run the pipeline through to generate a pipeline proto from an empty
# context. This ensures that the serialization code ran.
Expand Down
5 changes: 5 additions & 0 deletions sdks/python/apache_beam/pipeline.py
Original file line number Diff line number Diff line change
Expand Up @@ -797,6 +797,11 @@ def apply(
type_options = self._options.view_as(TypeOptions)
if type_options.pipeline_type_check:
transform.type_check_inputs(pvalueish)
if isinstance(pvalueish, pvalue.PBegin) and isinstance(transform, ParDo):
full_label = self._current_transform().full_label
raise TypeCheckError(
f"Transform '{full_label}' expects a PCollection as input. "
"Got a PBegin/Pipeline instead.")

pvalueish_result = self.runner.apply(transform, pvalueish, self._options)

Expand Down
17 changes: 17 additions & 0 deletions sdks/python/apache_beam/pipeline_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -702,6 +702,23 @@ def test_track_pcoll_unbounded_flatten(self):
self.assertIs(pcoll2_unbounded.is_bounded, False)
self.assertIs(merged.is_bounded, False)

def test_incompatible_pcollection_errmsg(self):
with pytest.raises(Exception,
match=r".*Map\(print\).*Got a PBegin/Pipeline instead."):
with beam.Pipeline() as pipeline:
_ = (pipeline | beam.Map(print))

class ParentTransform(PTransform):
def expand(self, pcoll):
return pcoll | beam.Map(print)

with pytest.raises(
Exception,
match=r".*ParentTransform/Map\(print\).*Got a PBegin/Pipeline instead."
):
with beam.Pipeline() as pipeline:
_ = (pipeline | ParentTransform())

def test_incompatible_submission_and_runtime_envs_fail_pipeline(self):
with mock.patch(
'apache_beam.transforms.environments.sdk_base_version_capability'
Expand Down
Loading