Skip to content

Commit

Permalink
Merge pull request #17482 from ihji/BEAM-14374
Browse files Browse the repository at this point in the history
[BEAM-14374] Fix module import error in FullyQualifiedNamedTransform
  • Loading branch information
ihji authored May 7, 2022
2 parents f432136 + 38ce39f commit 70b7567
Show file tree
Hide file tree
Showing 2 changed files with 29 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -65,11 +65,11 @@ def _resolve(cls, fully_qualified_name):
o = None
path = ''
for segment in fully_qualified_name.split('.'):
path = '.'.join([path, segment]) if path else segment
if o is not None and hasattr(o, segment):
o = getattr(o, segment)
else:
o = importlib.import_module(segment, path)
path = '.'.join([path, segment])
o = importlib.import_module(path)
return o

def to_runner_api_parameter(self, unused_context):
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@

import unittest

from mock import call
from mock import patch

import apache_beam as beam
from apache_beam.runners.portability import expansion_service
from apache_beam.testing.util import assert_that
Expand Down Expand Up @@ -116,6 +119,30 @@ def test_glob_filter(self):
with self.assertRaises(ValueError):
FullyQualifiedNamedTransform._resolve('apache_beam.Row')

@patch('importlib.import_module')
def test_resolve_by_path_segment(self, mock_import_module):
mock_import_module.return_value = None
with FullyQualifiedNamedTransform.with_filter('*'):
FullyQualifiedNamedTransform._resolve('a.b.c.d')
mock_import_module.assert_has_calls(
[call('a'), call('a.b'), call('a.b.c'), call('a.b.c.d')])

def test_resolve(self):
# test _resolve with the module that is not exposed to the top level
with FullyQualifiedNamedTransform.with_filter('*'):
dataframe_transform = FullyQualifiedNamedTransform._resolve(
'apache_beam.dataframe.transforms.DataframeTransform')
from apache_beam.dataframe.transforms import DataframeTransform
self.assertIs(dataframe_transform, DataframeTransform)

# test _resolve with the module that will never be exposed
# to the top level in the future
with FullyQualifiedNamedTransform.with_filter('*'):
argument_placeholder = FullyQualifiedNamedTransform._resolve(
'apache_beam.internal.util.ArgumentPlaceholder')
from apache_beam.internal.util import ArgumentPlaceholder
self.assertIs(argument_placeholder, ArgumentPlaceholder)


class _TestTransform(beam.PTransform):
@classmethod
Expand Down

0 comments on commit 70b7567

Please sign in to comment.