You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by Csaba Kassai <cs...@doctusoft.com> on 2017/07/26 15:52:39 UTC

Missing getOptions on Pipeline class

Hi,

we are currently migrating our pipelines written with the 1.9.x (pre-beam)
Dataflow Java SDK to the 2.0.0 version which is based on the 2.0.0 Beam SDK.
One change which cases a lot of headache is that getOptions method was
removed from the Pipeline class.
We used this method a lot during constructing the pipelines, for example in
composite PTransforms like this:

class MyTransform extends PTransform<PCollection<String>, PDone> {

   public PDone apply(PCollection<String> input) {
     PipelineOptions options = input.getPipeline().getOptions();
    ....
  }
}

Could you tell me what was the motivation for removing this method from the
pipeline?
Is there a nicer way to get the options inside a composite transformation,
than to pass it via the constructor?

Thanks,
Csabi

Re: Missing getOptions on Pipeline class

Posted by Csaba Kassai <cs...@doctusoft.com>.
Hi Eugene,

thanks for the answer, with the templates it totally makes sense.
Csabi


On Wed, 26 Jul 2017 at 18:10 Eugene Kirpichov <ki...@google.com> wrote:

> Hi Csaba,
>
> getOptions() was removed, and capturing PipelineOptions in the transform
> constructor is discouraged (or perhaps forbidden - not sure) because of the
> addition of templates (ValueProvider's) - the pipeline may be constructed,
> saved in a template, and then the template can be run with different
> PipelineOptions. Because of that, the Pipeline object itself can not have a
> defined set of options; and if you capture the PipelineOptions in the
> transform constructor, you may end up with confusing behavior if this
> pipeline is run with a different set of options than it was constructed
> with.
>
> Not sure if we have docs for that yet on the Beam website (if we don't, we
> should), but meanwhile take a look at the Dataflow docs
> https://cloud.google.com/dataflow/docs/templates/creating-templates
>
> Note that you're still allowed to access PipelineOptions, for example,
> from a DoFn, via ProcessContext.pipelineOptions() - because at runtime,
> options are always available.
>
> Let me know if this helps.
>
> On Wed, Jul 26, 2017 at 8:52 AM Csaba Kassai <cs...@doctusoft.com>
> wrote:
>
>> Hi,
>>
>> we are currently migrating our pipelines written with the 1.9.x
>> (pre-beam) Dataflow Java SDK to the 2.0.0 version which is based on the
>> 2.0.0 Beam SDK.
>> One change which cases a lot of headache is that getOptions method was
>> removed from the Pipeline class.
>> We used this method a lot during constructing the pipelines, for example
>> in composite PTransforms like this:
>>
>> class MyTransform extends PTransform<PCollection<String>, PDone> {
>>
>>    public PDone apply(PCollection<String> input) {
>>      PipelineOptions options = input.getPipeline().getOptions();
>>     ....
>>   }
>> }
>>
>> Could you tell me what was the motivation for removing this method from
>> the pipeline?
>> Is there a nicer way to get the options inside a composite
>> transformation, than to pass it via the constructor?
>>
>> Thanks,
>> Csabi
>>
>>
>>
>>
> --
--
[image: photo]
Csaba Kassai <https://www.linkedin.com/in/csabakassai>
Data Architect
M:  +36703379122
LinkedIn <https://www.linkedin.com/company/doctusoft-ltd-> *•* Facebook
<https://www.facebook.com/doctusoft> *•* Blog <http://doctusoft.com/blog/>
Doctusoft <http://doctusoft.com/>

Re: Missing getOptions on Pipeline class

Posted by Eugene Kirpichov <ki...@google.com>.
Hi Csaba,

getOptions() was removed, and capturing PipelineOptions in the transform
constructor is discouraged (or perhaps forbidden - not sure) because of the
addition of templates (ValueProvider's) - the pipeline may be constructed,
saved in a template, and then the template can be run with different
PipelineOptions. Because of that, the Pipeline object itself can not have a
defined set of options; and if you capture the PipelineOptions in the
transform constructor, you may end up with confusing behavior if this
pipeline is run with a different set of options than it was constructed
with.

Not sure if we have docs for that yet on the Beam website (if we don't, we
should), but meanwhile take a look at the Dataflow docs
https://cloud.google.com/dataflow/docs/templates/creating-templates

Note that you're still allowed to access PipelineOptions, for example, from
a DoFn, via ProcessContext.pipelineOptions() - because at runtime, options
are always available.

Let me know if this helps.

On Wed, Jul 26, 2017 at 8:52 AM Csaba Kassai <cs...@doctusoft.com>
wrote:

> Hi,
>
> we are currently migrating our pipelines written with the 1.9.x (pre-beam)
> Dataflow Java SDK to the 2.0.0 version which is based on the 2.0.0 Beam SDK.
> One change which cases a lot of headache is that getOptions method was
> removed from the Pipeline class.
> We used this method a lot during constructing the pipelines, for example
> in composite PTransforms like this:
>
> class MyTransform extends PTransform<PCollection<String>, PDone> {
>
>    public PDone apply(PCollection<String> input) {
>      PipelineOptions options = input.getPipeline().getOptions();
>     ....
>   }
> }
>
> Could you tell me what was the motivation for removing this method from
> the pipeline?
> Is there a nicer way to get the options inside a composite transformation,
> than to pass it via the constructor?
>
> Thanks,
> Csabi
>
>
>
>