We do the latter of treating constants as StaticValueProviders in the pipeline right now.
On Tue, Jul 11, 2017 at 4:47 PM Dmitry Demeshchuk <[email protected]> wrote: > Thanks a lot for the input, folks! > > Also, thanks for telling me about the concept of ValueProvider, Kenneth! > This was a good reminder to myself that some stuff that's described in the > Dataflow docs (I discovered > https://cloud.google.com/dataflow/docs/templates/creating-templates after > having read your reply) doesn't necessarily exist in the Beam > documentation. > > I do agree with Thomas' (and Robert's, in the JIRA bug) point that we may > often want to supply separate credentials for separate steps. It increases > the verbosity, and raises a question of what to do about filesystems > (ReadFromText and WriteToText), but it also has a lot of value. > > As of accessing pipeline options, what if PTransforms were treating > pipeline options as a NestedValueProvider of a sort? > > class MyDoFn(beam.DoFn): > def process(self, item): > # We fetch pipeline options in runtime > # or, it could look like opts = self.pipeline_options() > opts = self.pipeline_options.get() > > > Alternatively, we could treat each individual option as a ValueProvider > object, even if really it's just a constant. > > > On Tue, Jul 11, 2017 at 4:00 PM, Robert Bradshaw < > [email protected]> wrote: > > > Templates, including ValueProviders, were recently added to the Python > > SDK. +1 to pursuing this train of thought (and as I mentioned on the > > bug, and has been mentioned here, we don't want to add PipelineOptions > > access to PTransforms/at construction time). > > > > On Tue, Jul 11, 2017 at 3:21 PM, Kenneth Knowles <[email protected] > > > > wrote: > > > Hi Dmitry, > > > > > > This is a very worthwhile discussion that has recently come up on > > > StackOverflow, here: https://stackoverflow.com/a/45024542/4820657 > > > > > > We actually recently _removed_ the PipelineOptions from Pipeline.apply > in > > > Java since they tend to cause transforms to have implicit changes that > > make > > > them non-portable. Baking in credentials would probably fall into this > > > category. > > > > > > The other aspect to this is that we want to be able to build a pipeline > > and > > > run it later, in an environment chosen when we decide to run it. So > > > PipelineOptions are really for running, not building, a Pipeline. You > can > > > still use them for arg parsing and passing specific values to > transforms > > - > > > that is essentially orthogonal and just accidentally conflated. > > > > > > I can't speak to the state of Python SDK's maturity in this regard, but > > > there is a concept of a "ValueProvider" that is a deferred value that > can > > > be specified by PipelineOptions when you run your pipeline. This may be > > > what you want. You build a PTransform passing some of its configuration > > > parameters as ValueProvider and at run time you set them to actual > values > > > that are passed to the UDFs in your pipeline. > > > > > > Hope this helps. Despite not being deeply involved in Python, I wanted > to > > > lay out the territory so someone else could comment further without > > having > > > to go into background. > > > > > > Kenn > > > > > > On Tue, Jul 11, 2017 at 3:03 PM, Dmitry Demeshchuk < > [email protected] > > > > > > wrote: > > > > > >> Hi folks, > > >> > > >> Sometimes, it would be very useful if PTransforms had access to global > > >> pipeline options, such as various credentials, settings and so on. > > >> > > >> Per conversation in https://issues.apache.org/jira/browse/BEAM-2572, > > I'd > > >> like to kick off a discussion about that. > > >> > > >> This would be beneficial for at least one major use case: support for > > >> different cloud providers (AWS, Azure, etc) and an ability to specify > > each > > >> provider's credentials just once in the pipeline options. > > >> > > >> It looks like the trickiest part is not to make the PTransform objects > > have > > >> access to pipeline options (we could possibly just modified the > > >> Pipeline.apply > > >> <https://github.com/apache/beam/blob/master/sdks/python/ > > >> apache_beam/pipeline.py#L355> > > >> method), but to actually pass these options down the road, such as to > > DoFn > > >> objects and FileSystem objects. > > >> > > >> I'm still in the process of reading the code and understanding of what > > this > > >> could look like, so any input would be really appreciated. > > >> > > >> Thank you. > > >> > > >> -- > > >> Best regards, > > >> Dmitry Demeshchuk. > > >> > > > > > > -- > Best regards, > Dmitry Demeshchuk. >
