Skip to content

Commit

Permalink
address review comments
Browse files Browse the repository at this point in the history
  • Loading branch information
TheNeuralBit committed Jun 14, 2022
1 parent b4a3c88 commit 7b3c936
Show file tree
Hide file tree
Showing 2 changed files with 6 additions and 8 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -137,9 +137,7 @@ def test_batch_pardo(self):
assert_that(res, equal_to([6, 12, 18]))

def test_batch_pardo_override_type_inference(self):

class ArrayMultiplyTransposedDoFn(beam.DoFn):

class ArrayMultiplyDoFnOverride(beam.DoFn):
def process_batch(self, batch, *unused_args,
**unused_kwargs) -> Iterator[np.ndarray]:
assert isinstance(batch, np.ndarray)
Expand All @@ -162,7 +160,7 @@ def get_output_batch_type(self, input_element_type):
p
| beam.Create(np.array([1, 2, 3], dtype=np.int64)).with_output_types(
np.int64)
| beam.ParDo(ArrayMultiplyTransposedDoFn())
| beam.ParDo(ArrayMultiplyDoFnOverride())
| beam.Map(lambda x: x * 3))

assert_that(res, equal_to([6, 12, 18]))
Expand Down
8 changes: 4 additions & 4 deletions sdks/python/apache_beam/transforms/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -751,8 +751,8 @@ def get_input_batch_type(
DoFn is being applied to.
Returns:
``None`` if this DoFn cannot accept batches, a Beam typehint or a native
Python typehint if the DoFn can accept batches.
``None`` if this DoFn cannot accept batches, else a Beam typehint or
a native Python typehint.
"""
if not self.process_batch_defined:
return None
Expand Down Expand Up @@ -809,8 +809,8 @@ def get_output_batch_type(
DoFn is being applied to.
Returns:
``None`` if this DoFn will never yield batches, a Beam typehint or
a native Python typehint if the DoFn can yield batches.
``None`` if this DoFn will never yield batches, else a Beam typehint or
a native Python typehint.
"""
output_batch_type = None
if self.process_defined and self.process_yields_batches:
Expand Down

0 comments on commit 7b3c936

Please sign in to comment.