Skip to content

[BEAM-802] Add ValueProvider class for FileBasedSource I/O Transforms#1945

Closed
mariapython wants to merge 1 commit intoapache:masterfrom
mariapython:ppp_inmaster
Closed

[BEAM-802] Add ValueProvider class for FileBasedSource I/O Transforms#1945
mariapython wants to merge 1 commit intoapache:masterfrom
mariapython:ppp_inmaster

Conversation

@mariapython
Copy link
Contributor

  • Add ValueProvider class.
  • Derive StaticValueProvider and RuntimeValueProvider from ValueProvider.
  • Derive ValueProviderArgumentParser from argparse.ArgumentParser as API for the template user.
  • Modify FileBasedSource I/O transforms to accept objects of type ValueProvider.
  • Modify display_data.
  • Handle serialization / deserialization.

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.01%) to 69.595% when pulling f0dab8f on mariapython:ppp_inmaster into 8c1a577 on apache:master.

@asfbot
Copy link

asfbot commented Feb 8, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/7185/
--none--

@mariapython
Copy link
Contributor Author

R: @robertwb

@coveralls
Copy link

Coverage Status

Coverage increased (+0.005%) to 69.172% when pulling aaa6963 on mariapython:ppp_inmaster into a991c9d on apache:master.

@asfbot
Copy link

asfbot commented Feb 21, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/7633/
--none--

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.02%) to 69.284% when pulling 9560065 on mariapython:ppp_inmaster into 0d639e6 on apache:master.

@asfbot
Copy link

asfbot commented Feb 22, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/7687/
--none--

Copy link
Member

@aaltay aaltay left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @mariapython. I added my comments, let's start iterating on those.

elif isinstance(obj, ValueProvider):
if obj.is_accessible():
return to_json_value(obj.get())
return extra_types.JsonValue(is_null=True)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sammcveety

Is this the correct serialization for runtime value providers? (pipeline options will have a "key": None serialization)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

from apache_beam.transforms.display import DisplayDataItem
from apache_beam.utils.value_provider import ValueProvider
from apache_beam.utils.value_provider import StaticValueProvider
from apache_beam.utils.value_provider import RuntimeValueProvider
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does RuntimeValueProvider need to be exposed as a public interface. ValueProvider and StaticValueProvider could be the only things that is public outside the value_provider module and maybe a few places.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oops, that was a slip--fixed.

# We can't split compressed files efficiently so turn off splitting.
self._splittable = False
if validate:
if validate and not isinstance(file_pattern, RuntimeValueProvider):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could this be if validate and not file_pattern.is_accessible()? Is there a reason to check its type?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done


def _get_concat_source(self):
if self._concat_source is None:
if not self._pattern.is_accessible():
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@robertwb

Should we add a get_or_error method to ValueProvider?. It is a common pattern to check is_accessible and raising RuntimeError.

if not self.file_path_prefix.is_accessible():
raise RuntimeError('%s not accessible' % self.file_path_prefix)
file_path_prefix = self.file_path_prefix.get()
file_name_suffix = self.file_name_suffix.get()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you need to check if this is_accessible?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Forgot to pop a stash--done

)
value = (
pipeline_option.get()
if isinstance(pipeline_option, StaticValueProvider)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not use is_accessible instead of type checking?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

pipeline_options_dict = RuntimeValueProvider.pipeline_options_dict
if pipeline_options_dict is None:
raise RuntimeError('%s.get() not called from a runtime context' %self)
pipeline_option = (
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the reason here for recreating an options instance? Can we read the value out of pipeline_options_dict?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was modeled after Java's.


@classmethod
def set_runtime_options(cls, pipeline_options):
RuntimeValueProvider.pipeline_options_dict = pipeline_options
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe assert that RuntimeValueProvider.pipeline_options_dict is None

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

type=int,
help='This flag is a value provider')

# options = UserOptions(['--vp_arg', 'hola'])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove commented out code.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

assert options.vp_arg3.is_accessible() is True
assert options.vp_arg3.get() == '123'
assert options.vp_arg4.is_accessible() is True
assert options.vp_arg4.get() is None
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should not this fail? It is of RuntimeValueProvider and has no default.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

RuntimeValueProvider objects with no default and no value injected don't fail.
In Java: https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java#L191

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.01%) to 69.309% when pulling 9aff2a5 on mariapython:ppp_inmaster into 4bba380 on apache:master.

@asfbot
Copy link

asfbot commented Feb 27, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/7869/
--none--

@asfbot
Copy link

asfbot commented Mar 1, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8003/

Build result: FAILURE

[...truncated 1.55 MB...] at java.lang.Thread.run(Thread.java:745)Caused by: org.apache.maven.plugin.MojoExecutionException: Command execution failed. at org.codehaus.mojo.exec.ExecMojo.execute(ExecMojo.java:302) at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:134) at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:208) ... 31 moreCaused by: org.apache.commons.exec.ExecuteException: Process exited with an error: 1 (Exit value: 1) at org.apache.commons.exec.DefaultExecutor.executeInternal(DefaultExecutor.java:404) at org.apache.commons.exec.DefaultExecutor.execute(DefaultExecutor.java:166) at org.codehaus.mojo.exec.ExecMojo.executeCommandLine(ExecMojo.java:764) at org.codehaus.mojo.exec.ExecMojo.executeCommandLine(ExecMojo.java:711) at org.codehaus.mojo.exec.ExecMojo.execute(ExecMojo.java:289) ... 33 more2017-03-01T07:08:47.327 [ERROR] 2017-03-01T07:08:47.327 [ERROR] Re-run Maven using the -X switch to enable full debug logging.2017-03-01T07:08:47.327 [ERROR] 2017-03-01T07:08:47.327 [ERROR] For more information about the errors and possible solutions, please read the following articles:2017-03-01T07:08:47.327 [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException2017-03-01T07:08:47.328 [ERROR] 2017-03-01T07:08:47.328 [ERROR] After correcting the problems, you can resume the build with the command2017-03-01T07:08:47.328 [ERROR] mvn -rf :beam-sdks-pythonchannel stoppedSetting status of 97a3b53 to FAILURE with url https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8003/ and message: 'Build finished. 'Using context: Jenkins: Maven clean install
--none--

@coveralls
Copy link

Coverage Status

Coverage increased (+0.003%) to 69.887% when pulling 2916651 on mariapython:ppp_inmaster into b2428ea on apache:master.

@asfbot
Copy link

asfbot commented Mar 1, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8029/
--none--

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.01%) to 69.881% when pulling 9bdd605 on mariapython:ppp_inmaster into 1783819 on apache:master.

@asfbot
Copy link

asfbot commented Mar 3, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8078/
--none--

"""
def add_value_provider_argument(self, *args, **kwargs):
# Extract the option name from positional argument ['pos_arg']
if args[0][0] != '-':
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are two assumptions here. Could you assert those for a better error message.

  1. args has at least length of 1 and args[0] exists.
  2. and args[0] is indexable, like args[0] is a string.

Copy link
Contributor Author

@mariapython mariapython Mar 6, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Both are enforced by argparse's add_argument. i.e. CLI arguments are strings and an argument always requires a name. If you try to add an argument without a name, you get:
TypeError: _get_positional_kwargs() takes exactly 2 arguments (1 given)

Since the call to add_argument happens later, I have placed one assert to check for args to exist and have at least one element of size one. There is more I could add, but I don't want to obscure the code by doing heavy type checking here, since add_argument will take care of it a few lines below.

option_name = args[0]
if kwargs.get('nargs') is None: # make them optionally templated
kwargs['nargs'] = '?'
# or keyword arguments like [--kw_arg, -k, -w] or [--kw-arg]
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move this comment inside the else block

Copy link
Contributor Author

@mariapython mariapython Mar 6, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK. The reason why I usually don't do it is because rigorously, that would make me have an extra comment line:

    # Extract the option name
    if args[0][0] != '-':
      # from positional argument ['pos_arg']
      option_name = args[0]
      if kwargs.get('nargs') is None:  # make them optionally templated
        kwargs['nargs'] = '?'
    else:
      # or keyword arguments like [--kw_arg, -k, -w] or [--kw-arg]
      option_name = [i.replace('--', '') for i in args if i[:2] == '--'][0]

)


def check_accessible(fields):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a comment about what is this and how users could use it. It will have utility to developers building components.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done long ago, although forgot to respond, it seems.

if pipeline_options_dict is None:
raise RuntimeError('%s.get() not called from a runtime context' %self)
pipeline_option = (
self.pipeline_options_subclass.from_dictionary(pipeline_options_dict)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PipelineOptions.from_dictionary(pipeline_options_dict).view_as(pipeline_options_subclass).get(self.option_name)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PipelineOptions.from_dictionary(pipeline_options_dict).view_as(self.pipeline_options_subclass) would be OK, but not .get(...)

@coveralls
Copy link

Coverage Status

Coverage remained the same at 70.085% when pulling 1ff44d5 on mariapython:ppp_inmaster into 2f96bc3 on apache:master.

@asfbot
Copy link

asfbot commented Mar 6, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8132/
--none--

@mariapython mariapython force-pushed the ppp_inmaster branch 2 times, most recently from a6b5196 to c5569ca Compare March 8, 2017 06:17
@coveralls
Copy link

Coverage Status

Coverage increased (+0.007%) to 70.051% when pulling c5569ca on mariapython:ppp_inmaster into 690e1e2 on apache:master.

@coveralls
Copy link

Coverage Status

Coverage remained the same at 70.044% when pulling c5569ca on mariapython:ppp_inmaster into 690e1e2 on apache:master.

@asfbot
Copy link

asfbot commented Mar 8, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8201/
--none--

dest='output',
required=True,
help='Output file to write results to.')
pipeline_options = UserOptions(argv)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Our canonical form is pipeline_options = PipelineOptions(argv). Could you convert to that form? It should work exactly the same.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you mean this:
pipeline_options = PipelineOptions(argv).view_as(UserOptions)
Yes, it works, but I found it less readable.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is possible for a user code to define multiple options. We want them to use:

pipeline_options = PipelineOptions(argv)
user_options = pipeline_options.view_as(UserOption)

This is the standard way for the existing examples and documentation.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, I can then use those two lines, that is no problem

required=True,
help='Output file to write results to.')
known_args, pipeline_args = parser.parse_known_args(argv)
class UserOptions(PipelineOptions):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We used to have a problem with defining options here (See: #1889 (comment))

Other tests started to fail because newly defined options stay in scope affected other tests. Is it resolved? Or this is not happening any more because the options are defaulting to ValueProviders?

Copy link
Contributor Author

@mariapython mariapython Mar 9, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I haven't seen that happening, as long as there is no call to get_all_options()

for vp in [getattr(self, field) for field in fields]:
if not vp.is_accessible():
raise RuntimeError('%s not accessible' % vp)
for obj in [getattr(self, vp) for vp in value_provider_list]:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you use something other than obj. This is representing an option not just an object.

self.assertEqual(options.vp_pos_arg.get(), 'abc')

def test_runtime_value_provider_positional_argument(self):
class UserOptions(PipelineOptions):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Change UserOptions to a more unique name, they potentially cause name conflicts with other tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

self.assertFalse(options.vp_arg3.is_accessible())
self.assertFalse(options.vp_arg4.is_accessible())
self.assertFalse(options.vp_pos_arg.is_accessible())
# self.assertFalse(options.vp_pos_arg.is_accessible())
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove commented out code. And why this is accessible when other vps are not?

Copy link
Contributor Author

@mariapython mariapython Mar 13, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done. It is accessible because a positional argument value has been given as option: '1.2'

required=True,
help='Output file to write results to.')
known_args, pipeline_args = parser.parse_known_args(argv)
class UserOptions(PipelineOptions):
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

WordCountOptions maybe?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do not see the change?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wanted to push all the changes at once, and I am working on one last commit.


# provide values at graph-construction time
# (options not provided here become of the type RuntimeValueProvider)
options = UserOptions1(['1.2'])
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I may have asked this before. If that is the case point me to the answer please.

Does it make sense to wrap things as value providers when template_location flag is not set. If the flag is not set we know that RVP values will be missing because it will not be run as a template.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, you haven't.
Sure, I will add a warning for the case where there are RuntimeValueProvider arguments and template_location is missing.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should it be an error or a warning? What is the behavior in Java SDK?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not finding anything in Java to enforce the template_location, so I am not adding this at this point, in order to be able to merge this PR. Should I open a JIRA issue for this to be done_ If so, I guess two JIRA issues, one for python and one for java?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good. Please reference the JIRA issue here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.003%) to 70.131% when pulling 6351e5a on mariapython:ppp_inmaster into 781e417 on apache:master.

@asfbot
Copy link

asfbot commented Mar 13, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8347/
--none--

@coveralls
Copy link

Coverage Status

Changes Unknown when pulling 5a7e5ff on mariapython:ppp_inmaster into ** on apache:master**.

@asfbot
Copy link

asfbot commented Mar 13, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8361/
--none--

@coveralls
Copy link

Coverage Status

Coverage increased (+0.01%) to 70.181% when pulling 761b38a on mariapython:ppp_inmaster into 843b663 on apache:master.

@asfbot
Copy link

asfbot commented Mar 14, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8394/
--none--

Incorporate a BeamArgumentParser (argparse.ArgumentParser + ValueProviders).
Add StaticValueProvider and RuntimeValueProvider derived from ValueProvider.
Add serialization for ValueProvider objects.
Add testing for ValueProvider objects.
Modify FileBasedSource and FileSink to accept ValueProvider objects.
@coveralls
Copy link

Coverage Status

Coverage decreased (-0.003%) to 70.177% when pulling 0f9b1d1 on mariapython:ppp_inmaster into cc12fd3 on apache:master.

@asfbot
Copy link

asfbot commented Mar 14, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8409/
--none--

@coveralls
Copy link

Coverage Status

Coverage decreased (-0.02%) to 70.164% when pulling 0f9b1d1 on mariapython:ppp_inmaster into cc12fd3 on apache:master.

@asfbot
Copy link

asfbot commented Mar 14, 2017

@aaltay
Copy link
Member

aaltay commented Mar 15, 2017

retest this please

@coveralls
Copy link

Coverage Status

Coverage increased (+0.002%) to 70.183% when pulling 0f9b1d1 on mariapython:ppp_inmaster into cc12fd3 on apache:master.

@asfbot
Copy link

asfbot commented Mar 15, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/beam_PreCommit_Java_MavenInstall/8452/
--none--

@aaltay
Copy link
Member

aaltay commented Mar 15, 2017

LGTM. Thank you @mariapython

@asfgit asfgit closed this in 49c29f1 Mar 15, 2017
@mariapython
Copy link
Contributor Author

Run Python PostCommit

@mariapython mariapython deleted the ppp_inmaster branch April 26, 2017 22:56
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants