Re: A Declarative API for Apache Beam

2022-12-14 Thread Chamikara Jayalath via dev
+1 for these proposals and agree that these will simplify and demystify
Beam for many new users. I think when combined with the x-lang/Schema-Aware
transform binding, these might end up being adequate solutions for many
production use-cases as well (unless users need to define custom
composites, I/O connectors, etc.).

Also, thanks for providing prototype implementations with examples.

- Cham


On Wed, Dec 14, 2022 at 3:01 PM Sachin Agarwal via dev 
wrote:

> To build on Kenn's point, if we leverage existing stuff like dbt we get
> access to a ready made community which can help drive both adoption and
> incremental innovation by bringing more folks to Beam
>
> On Wed, Dec 14, 2022 at 2:57 PM Kenneth Knowles  wrote:
>
>> 1. I love the idea. Back in the early days people talked about an "XML
>> SDK" or "JSON SDK" or "YAML SDK" and it didn't really make sense at the
>> time. Portability and specifically cross-language schema transforms gives
>> the right infrastructure so this is the perfect time: unique names (URNs)
>> for transforms and explicit lists of parameters they require.
>>
>> 2. I like the idea of re-using some existing thing like dbt if it is
>> pretty much what we were going to do anyhow. I don't think we should hold
>> ourselves back. I also don't think we'll gain anything in terms of
>> implementation. But at least it could fast-forward our design process
>> because we simply don't have to make most of the decisions because they are
>> made for us.
>>
>>
>>
>> On Wed, Dec 14, 2022 at 2:44 PM Byron Ellis via dev 
>> wrote:
>>
>>> And I guess also a PR for completeness to make it easier to find going
>>> forward instead of my random repo:
>>> https://github.com/apache/beam/pull/24670
>>>
>>> On Wed, Dec 14, 2022 at 2:37 PM Byron Ellis 
>>> wrote:
>>>
 Since Robert opened that can of worms (and we happened to talk about it
 yesterday)... :-)

 I figured I'd also share my start on a "port" of dbt to the Beam SDK.
 This would be complementary as it doesn't really provide a way of
 specifying a pipeline, more orchestrating and packaging a complex
 pipeline---dbt itself supports SQL and Python Dataframes, which both seem
 like reasonable things for Beam and it wouldn't be a stretch to include
 something like the format above. Though in my head I had imagined people
 would tend to write composite transforms in the SDK of their choosing that
 are then exposed at this layer. I decided to go with dbt as it also
 provides a number of nice "quality of life" features for its users like
 documentation, validation, environments and so on,

 I did a really quick proof-of-viability implementation here:
 https://github.com/byronellis/beam/tree/structured-pipeline-definitions

 And you can see a really simple pipeline that reads a seed file
 (TextIO), runs it through a couple of SQLTransforms and then drops it out
 to a logger via a simple DoFn here:
 https://github.com/byronellis/beam/tree/structured-pipeline-definitions/sdks/java/extensions/spd/src/test/resources/simple_pipeline

 I've also heard a rumor there might also be a textproto-based
 representation floating around too :-)

 Best,
 B





 On Wed, Dec 14, 2022 at 2:21 PM Damon Douglas via dev <
 dev@beam.apache.org> wrote:

> Hello Robert,
>
> I'm replying to say that I've been waiting for something like this
> ever since I started learning Beam and I'm grateful you are pushing this
> forward.
>
> Best,
>
> Damon
>
> On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw 
> wrote:
>
>> While Beam provides powerful APIs for authoring sophisticated data
>> processing pipelines, it often still has too high a barrier for
>> getting started and authoring simple pipelines. Even setting up the
>> environment, installing the dependencies, and setting up the project
>> can be an overwhelming amount of boilerplate for some (though
>> https://beam.apache.org/blog/beam-starter-projects/ has gone a long
>> way in making this easier). At the other extreme, the Dataflow project
>> has the notion of templates which are pre-built Beam pipelines that
>> can be easily launched from the command line, or even from your
>> browser, but they are fairly restrictive, limited to pre-assembled
>> pipelines taking a small number of parameters.
>>
>> The idea of creating a yaml-based description of pipelines has come up
>> several times in several contexts and this last week I decided to code
>> up what it could look like. Here's a proposal.
>>
>> pipeline:
>>   - type: chain
>> transforms:
>>   - type: ReadFromText
>> args:
>>  file_pattern: "wordcount.yaml"
>>   - type: PyMap
>> fn: "str.lower"
>>   - type: PyFlatMap
>> fn: "import re\nlambda 

Re: A Declarative API for Apache Beam

2022-12-14 Thread Sachin Agarwal via dev
To build on Kenn's point, if we leverage existing stuff like dbt we get
access to a ready made community which can help drive both adoption and
incremental innovation by bringing more folks to Beam

On Wed, Dec 14, 2022 at 2:57 PM Kenneth Knowles  wrote:

> 1. I love the idea. Back in the early days people talked about an "XML
> SDK" or "JSON SDK" or "YAML SDK" and it didn't really make sense at the
> time. Portability and specifically cross-language schema transforms gives
> the right infrastructure so this is the perfect time: unique names (URNs)
> for transforms and explicit lists of parameters they require.
>
> 2. I like the idea of re-using some existing thing like dbt if it is
> pretty much what we were going to do anyhow. I don't think we should hold
> ourselves back. I also don't think we'll gain anything in terms of
> implementation. But at least it could fast-forward our design process
> because we simply don't have to make most of the decisions because they are
> made for us.
>
>
>
> On Wed, Dec 14, 2022 at 2:44 PM Byron Ellis via dev 
> wrote:
>
>> And I guess also a PR for completeness to make it easier to find going
>> forward instead of my random repo:
>> https://github.com/apache/beam/pull/24670
>>
>> On Wed, Dec 14, 2022 at 2:37 PM Byron Ellis 
>> wrote:
>>
>>> Since Robert opened that can of worms (and we happened to talk about it
>>> yesterday)... :-)
>>>
>>> I figured I'd also share my start on a "port" of dbt to the Beam SDK.
>>> This would be complementary as it doesn't really provide a way of
>>> specifying a pipeline, more orchestrating and packaging a complex
>>> pipeline---dbt itself supports SQL and Python Dataframes, which both seem
>>> like reasonable things for Beam and it wouldn't be a stretch to include
>>> something like the format above. Though in my head I had imagined people
>>> would tend to write composite transforms in the SDK of their choosing that
>>> are then exposed at this layer. I decided to go with dbt as it also
>>> provides a number of nice "quality of life" features for its users like
>>> documentation, validation, environments and so on,
>>>
>>> I did a really quick proof-of-viability implementation here:
>>> https://github.com/byronellis/beam/tree/structured-pipeline-definitions
>>>
>>> And you can see a really simple pipeline that reads a seed file
>>> (TextIO), runs it through a couple of SQLTransforms and then drops it out
>>> to a logger via a simple DoFn here:
>>> https://github.com/byronellis/beam/tree/structured-pipeline-definitions/sdks/java/extensions/spd/src/test/resources/simple_pipeline
>>>
>>> I've also heard a rumor there might also be a textproto-based
>>> representation floating around too :-)
>>>
>>> Best,
>>> B
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Dec 14, 2022 at 2:21 PM Damon Douglas via dev <
>>> dev@beam.apache.org> wrote:
>>>
 Hello Robert,

 I'm replying to say that I've been waiting for something like this ever
 since I started learning Beam and I'm grateful you are pushing this 
 forward.

 Best,

 Damon

 On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw 
 wrote:

> While Beam provides powerful APIs for authoring sophisticated data
> processing pipelines, it often still has too high a barrier for
> getting started and authoring simple pipelines. Even setting up the
> environment, installing the dependencies, and setting up the project
> can be an overwhelming amount of boilerplate for some (though
> https://beam.apache.org/blog/beam-starter-projects/ has gone a long
> way in making this easier). At the other extreme, the Dataflow project
> has the notion of templates which are pre-built Beam pipelines that
> can be easily launched from the command line, or even from your
> browser, but they are fairly restrictive, limited to pre-assembled
> pipelines taking a small number of parameters.
>
> The idea of creating a yaml-based description of pipelines has come up
> several times in several contexts and this last week I decided to code
> up what it could look like. Here's a proposal.
>
> pipeline:
>   - type: chain
> transforms:
>   - type: ReadFromText
> args:
>  file_pattern: "wordcount.yaml"
>   - type: PyMap
> fn: "str.lower"
>   - type: PyFlatMap
> fn: "import re\nlambda line: re.findall('[a-z]+', line)"
>   - type: PyTransform
> name: Count
> constructor:
> "apache_beam.transforms.combiners.Count.PerElement"
>   - type: PyMap
> fn: str
>   - type: WriteToText
> file_path_prefix: "counts.txt"
>
> Some more examples at
> https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a
>
> A prototype (feedback welcome) can be found at
> https://github.com/apache/beam/pull/24667. It can be invoked as
>
> python -m 

Re: A Declarative API for Apache Beam

2022-12-14 Thread Robert Burke
I like the idea of a common spec for something like this so we can actually
cross validate all the SDK behaviours. It would make testing significantly
easier.

On Wed, Dec 14, 2022, 2:57 PM Kenneth Knowles  wrote:

> 1. I love the idea. Back in the early days people talked about an "XML
> SDK" or "JSON SDK" or "YAML SDK" and it didn't really make sense at the
> time. Portability and specifically cross-language schema transforms gives
> the right infrastructure so this is the perfect time: unique names (URNs)
> for transforms and explicit lists of parameters they require.
>
> 2. I like the idea of re-using some existing thing like dbt if it is
> pretty much what we were going to do anyhow. I don't think we should hold
> ourselves back. I also don't think we'll gain anything in terms of
> implementation. But at least it could fast-forward our design process
> because we simply don't have to make most of the decisions because they are
> made for us.
>
>
>
> On Wed, Dec 14, 2022 at 2:44 PM Byron Ellis via dev 
> wrote:
>
>> And I guess also a PR for completeness to make it easier to find going
>> forward instead of my random repo:
>> https://github.com/apache/beam/pull/24670
>>
>> On Wed, Dec 14, 2022 at 2:37 PM Byron Ellis 
>> wrote:
>>
>>> Since Robert opened that can of worms (and we happened to talk about it
>>> yesterday)... :-)
>>>
>>> I figured I'd also share my start on a "port" of dbt to the Beam SDK.
>>> This would be complementary as it doesn't really provide a way of
>>> specifying a pipeline, more orchestrating and packaging a complex
>>> pipeline---dbt itself supports SQL and Python Dataframes, which both seem
>>> like reasonable things for Beam and it wouldn't be a stretch to include
>>> something like the format above. Though in my head I had imagined people
>>> would tend to write composite transforms in the SDK of their choosing that
>>> are then exposed at this layer. I decided to go with dbt as it also
>>> provides a number of nice "quality of life" features for its users like
>>> documentation, validation, environments and so on,
>>>
>>> I did a really quick proof-of-viability implementation here:
>>> https://github.com/byronellis/beam/tree/structured-pipeline-definitions
>>>
>>> And you can see a really simple pipeline that reads a seed file
>>> (TextIO), runs it through a couple of SQLTransforms and then drops it out
>>> to a logger via a simple DoFn here:
>>> https://github.com/byronellis/beam/tree/structured-pipeline-definitions/sdks/java/extensions/spd/src/test/resources/simple_pipeline
>>>
>>> I've also heard a rumor there might also be a textproto-based
>>> representation floating around too :-)
>>>
>>> Best,
>>> B
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Dec 14, 2022 at 2:21 PM Damon Douglas via dev <
>>> dev@beam.apache.org> wrote:
>>>
 Hello Robert,

 I'm replying to say that I've been waiting for something like this ever
 since I started learning Beam and I'm grateful you are pushing this 
 forward.

 Best,

 Damon

 On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw 
 wrote:

> While Beam provides powerful APIs for authoring sophisticated data
> processing pipelines, it often still has too high a barrier for
> getting started and authoring simple pipelines. Even setting up the
> environment, installing the dependencies, and setting up the project
> can be an overwhelming amount of boilerplate for some (though
> https://beam.apache.org/blog/beam-starter-projects/ has gone a long
> way in making this easier). At the other extreme, the Dataflow project
> has the notion of templates which are pre-built Beam pipelines that
> can be easily launched from the command line, or even from your
> browser, but they are fairly restrictive, limited to pre-assembled
> pipelines taking a small number of parameters.
>
> The idea of creating a yaml-based description of pipelines has come up
> several times in several contexts and this last week I decided to code
> up what it could look like. Here's a proposal.
>
> pipeline:
>   - type: chain
> transforms:
>   - type: ReadFromText
> args:
>  file_pattern: "wordcount.yaml"
>   - type: PyMap
> fn: "str.lower"
>   - type: PyFlatMap
> fn: "import re\nlambda line: re.findall('[a-z]+', line)"
>   - type: PyTransform
> name: Count
> constructor:
> "apache_beam.transforms.combiners.Count.PerElement"
>   - type: PyMap
> fn: str
>   - type: WriteToText
> file_path_prefix: "counts.txt"
>
> Some more examples at
> https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a
>
> A prototype (feedback welcome) can be found at
> https://github.com/apache/beam/pull/24667. It can be invoked as
>
> python -m apache_beam.yaml.main --pipeline_spec_file
> 

Re: A Declarative API for Apache Beam

2022-12-14 Thread Kenneth Knowles
1. I love the idea. Back in the early days people talked about an "XML SDK"
or "JSON SDK" or "YAML SDK" and it didn't really make sense at the time.
Portability and specifically cross-language schema transforms gives the
right infrastructure so this is the perfect time: unique names (URNs) for
transforms and explicit lists of parameters they require.

2. I like the idea of re-using some existing thing like dbt if it is pretty
much what we were going to do anyhow. I don't think we should hold
ourselves back. I also don't think we'll gain anything in terms of
implementation. But at least it could fast-forward our design process
because we simply don't have to make most of the decisions because they are
made for us.



On Wed, Dec 14, 2022 at 2:44 PM Byron Ellis via dev 
wrote:

> And I guess also a PR for completeness to make it easier to find going
> forward instead of my random repo:
> https://github.com/apache/beam/pull/24670
>
> On Wed, Dec 14, 2022 at 2:37 PM Byron Ellis  wrote:
>
>> Since Robert opened that can of worms (and we happened to talk about it
>> yesterday)... :-)
>>
>> I figured I'd also share my start on a "port" of dbt to the Beam SDK.
>> This would be complementary as it doesn't really provide a way of
>> specifying a pipeline, more orchestrating and packaging a complex
>> pipeline---dbt itself supports SQL and Python Dataframes, which both seem
>> like reasonable things for Beam and it wouldn't be a stretch to include
>> something like the format above. Though in my head I had imagined people
>> would tend to write composite transforms in the SDK of their choosing that
>> are then exposed at this layer. I decided to go with dbt as it also
>> provides a number of nice "quality of life" features for its users like
>> documentation, validation, environments and so on,
>>
>> I did a really quick proof-of-viability implementation here:
>> https://github.com/byronellis/beam/tree/structured-pipeline-definitions
>>
>> And you can see a really simple pipeline that reads a seed file (TextIO),
>> runs it through a couple of SQLTransforms and then drops it out to a logger
>> via a simple DoFn here:
>> https://github.com/byronellis/beam/tree/structured-pipeline-definitions/sdks/java/extensions/spd/src/test/resources/simple_pipeline
>>
>> I've also heard a rumor there might also be a textproto-based
>> representation floating around too :-)
>>
>> Best,
>> B
>>
>>
>>
>>
>>
>> On Wed, Dec 14, 2022 at 2:21 PM Damon Douglas via dev <
>> dev@beam.apache.org> wrote:
>>
>>> Hello Robert,
>>>
>>> I'm replying to say that I've been waiting for something like this ever
>>> since I started learning Beam and I'm grateful you are pushing this forward.
>>>
>>> Best,
>>>
>>> Damon
>>>
>>> On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw 
>>> wrote:
>>>
 While Beam provides powerful APIs for authoring sophisticated data
 processing pipelines, it often still has too high a barrier for
 getting started and authoring simple pipelines. Even setting up the
 environment, installing the dependencies, and setting up the project
 can be an overwhelming amount of boilerplate for some (though
 https://beam.apache.org/blog/beam-starter-projects/ has gone a long
 way in making this easier). At the other extreme, the Dataflow project
 has the notion of templates which are pre-built Beam pipelines that
 can be easily launched from the command line, or even from your
 browser, but they are fairly restrictive, limited to pre-assembled
 pipelines taking a small number of parameters.

 The idea of creating a yaml-based description of pipelines has come up
 several times in several contexts and this last week I decided to code
 up what it could look like. Here's a proposal.

 pipeline:
   - type: chain
 transforms:
   - type: ReadFromText
 args:
  file_pattern: "wordcount.yaml"
   - type: PyMap
 fn: "str.lower"
   - type: PyFlatMap
 fn: "import re\nlambda line: re.findall('[a-z]+', line)"
   - type: PyTransform
 name: Count
 constructor: "apache_beam.transforms.combiners.Count.PerElement"
   - type: PyMap
 fn: str
   - type: WriteToText
 file_path_prefix: "counts.txt"

 Some more examples at
 https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a

 A prototype (feedback welcome) can be found at
 https://github.com/apache/beam/pull/24667. It can be invoked as

 python -m apache_beam.yaml.main --pipeline_spec_file
 [path/to/file.yaml] [other_pipene_args]

 or

 python -m apache_beam.yaml.main --pipeline_spec [yaml_contents]
 [other_pipene_args]

 For example, to play around with this one could do

 python -m apache_beam.yaml.main  \
 --pipeline_spec "$(curl

 

Re: A Declarative API for Apache Beam

2022-12-14 Thread Byron Ellis via dev
And I guess also a PR for completeness to make it easier to find going
forward instead of my random repo: https://github.com/apache/beam/pull/24670

On Wed, Dec 14, 2022 at 2:37 PM Byron Ellis  wrote:

> Since Robert opened that can of worms (and we happened to talk about it
> yesterday)... :-)
>
> I figured I'd also share my start on a "port" of dbt to the Beam SDK. This
> would be complementary as it doesn't really provide a way of specifying a
> pipeline, more orchestrating and packaging a complex pipeline---dbt itself
> supports SQL and Python Dataframes, which both seem like reasonable things
> for Beam and it wouldn't be a stretch to include something like the format
> above. Though in my head I had imagined people would tend to write
> composite transforms in the SDK of their choosing that are then exposed at
> this layer. I decided to go with dbt as it also provides a number of nice
> "quality of life" features for its users like documentation, validation,
> environments and so on,
>
> I did a really quick proof-of-viability implementation here:
> https://github.com/byronellis/beam/tree/structured-pipeline-definitions
>
> And you can see a really simple pipeline that reads a seed file (TextIO),
> runs it through a couple of SQLTransforms and then drops it out to a logger
> via a simple DoFn here:
> https://github.com/byronellis/beam/tree/structured-pipeline-definitions/sdks/java/extensions/spd/src/test/resources/simple_pipeline
>
> I've also heard a rumor there might also be a textproto-based
> representation floating around too :-)
>
> Best,
> B
>
>
>
>
>
> On Wed, Dec 14, 2022 at 2:21 PM Damon Douglas via dev 
> wrote:
>
>> Hello Robert,
>>
>> I'm replying to say that I've been waiting for something like this ever
>> since I started learning Beam and I'm grateful you are pushing this forward.
>>
>> Best,
>>
>> Damon
>>
>> On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw 
>> wrote:
>>
>>> While Beam provides powerful APIs for authoring sophisticated data
>>> processing pipelines, it often still has too high a barrier for
>>> getting started and authoring simple pipelines. Even setting up the
>>> environment, installing the dependencies, and setting up the project
>>> can be an overwhelming amount of boilerplate for some (though
>>> https://beam.apache.org/blog/beam-starter-projects/ has gone a long
>>> way in making this easier). At the other extreme, the Dataflow project
>>> has the notion of templates which are pre-built Beam pipelines that
>>> can be easily launched from the command line, or even from your
>>> browser, but they are fairly restrictive, limited to pre-assembled
>>> pipelines taking a small number of parameters.
>>>
>>> The idea of creating a yaml-based description of pipelines has come up
>>> several times in several contexts and this last week I decided to code
>>> up what it could look like. Here's a proposal.
>>>
>>> pipeline:
>>>   - type: chain
>>> transforms:
>>>   - type: ReadFromText
>>> args:
>>>  file_pattern: "wordcount.yaml"
>>>   - type: PyMap
>>> fn: "str.lower"
>>>   - type: PyFlatMap
>>> fn: "import re\nlambda line: re.findall('[a-z]+', line)"
>>>   - type: PyTransform
>>> name: Count
>>> constructor: "apache_beam.transforms.combiners.Count.PerElement"
>>>   - type: PyMap
>>> fn: str
>>>   - type: WriteToText
>>> file_path_prefix: "counts.txt"
>>>
>>> Some more examples at
>>> https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a
>>>
>>> A prototype (feedback welcome) can be found at
>>> https://github.com/apache/beam/pull/24667. It can be invoked as
>>>
>>> python -m apache_beam.yaml.main --pipeline_spec_file
>>> [path/to/file.yaml] [other_pipene_args]
>>>
>>> or
>>>
>>> python -m apache_beam.yaml.main --pipeline_spec [yaml_contents]
>>> [other_pipene_args]
>>>
>>> For example, to play around with this one could do
>>>
>>> python -m apache_beam.yaml.main  \
>>> --pipeline_spec "$(curl
>>>
>>> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
>>> )"
>>> \
>>> --runner=apache_beam.runners.render.RenderRunner \
>>> --render_out=out.png
>>>
>>> Alternatively one can run it as a docker container with no need to
>>> install any SDK
>>>
>>> docker run --rm \
>>> --entrypoint /usr/local/bin/python \
>>> gcr.io/apache-beam-testing/yaml_template:dev
>>> /dataflow/template/main.py \
>>> --pipeline_spec="$(curl
>>>
>>> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
>>> )"
>>>
>>> Though of course one would have to set up the appropriate mount points
>>> to do any local filesystem io and/or credentials.
>>>
>>> This is also available as a Dataflow template and can be invoked as
>>>
>>> gcloud dataflow flex-template run \
>>> 

Re: A Declarative API for Apache Beam

2022-12-14 Thread Byron Ellis via dev
Since Robert opened that can of worms (and we happened to talk about it
yesterday)... :-)

I figured I'd also share my start on a "port" of dbt to the Beam SDK. This
would be complementary as it doesn't really provide a way of specifying a
pipeline, more orchestrating and packaging a complex pipeline---dbt itself
supports SQL and Python Dataframes, which both seem like reasonable things
for Beam and it wouldn't be a stretch to include something like the format
above. Though in my head I had imagined people would tend to write
composite transforms in the SDK of their choosing that are then exposed at
this layer. I decided to go with dbt as it also provides a number of nice
"quality of life" features for its users like documentation, validation,
environments and so on,

I did a really quick proof-of-viability implementation here:
https://github.com/byronellis/beam/tree/structured-pipeline-definitions

And you can see a really simple pipeline that reads a seed file (TextIO),
runs it through a couple of SQLTransforms and then drops it out to a logger
via a simple DoFn here:
https://github.com/byronellis/beam/tree/structured-pipeline-definitions/sdks/java/extensions/spd/src/test/resources/simple_pipeline

I've also heard a rumor there might also be a textproto-based
representation floating around too :-)

Best,
B





On Wed, Dec 14, 2022 at 2:21 PM Damon Douglas via dev 
wrote:

> Hello Robert,
>
> I'm replying to say that I've been waiting for something like this ever
> since I started learning Beam and I'm grateful you are pushing this forward.
>
> Best,
>
> Damon
>
> On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw 
> wrote:
>
>> While Beam provides powerful APIs for authoring sophisticated data
>> processing pipelines, it often still has too high a barrier for
>> getting started and authoring simple pipelines. Even setting up the
>> environment, installing the dependencies, and setting up the project
>> can be an overwhelming amount of boilerplate for some (though
>> https://beam.apache.org/blog/beam-starter-projects/ has gone a long
>> way in making this easier). At the other extreme, the Dataflow project
>> has the notion of templates which are pre-built Beam pipelines that
>> can be easily launched from the command line, or even from your
>> browser, but they are fairly restrictive, limited to pre-assembled
>> pipelines taking a small number of parameters.
>>
>> The idea of creating a yaml-based description of pipelines has come up
>> several times in several contexts and this last week I decided to code
>> up what it could look like. Here's a proposal.
>>
>> pipeline:
>>   - type: chain
>> transforms:
>>   - type: ReadFromText
>> args:
>>  file_pattern: "wordcount.yaml"
>>   - type: PyMap
>> fn: "str.lower"
>>   - type: PyFlatMap
>> fn: "import re\nlambda line: re.findall('[a-z]+', line)"
>>   - type: PyTransform
>> name: Count
>> constructor: "apache_beam.transforms.combiners.Count.PerElement"
>>   - type: PyMap
>> fn: str
>>   - type: WriteToText
>> file_path_prefix: "counts.txt"
>>
>> Some more examples at
>> https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a
>>
>> A prototype (feedback welcome) can be found at
>> https://github.com/apache/beam/pull/24667. It can be invoked as
>>
>> python -m apache_beam.yaml.main --pipeline_spec_file
>> [path/to/file.yaml] [other_pipene_args]
>>
>> or
>>
>> python -m apache_beam.yaml.main --pipeline_spec [yaml_contents]
>> [other_pipene_args]
>>
>> For example, to play around with this one could do
>>
>> python -m apache_beam.yaml.main  \
>> --pipeline_spec "$(curl
>>
>> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
>> )"
>> \
>> --runner=apache_beam.runners.render.RenderRunner \
>> --render_out=out.png
>>
>> Alternatively one can run it as a docker container with no need to
>> install any SDK
>>
>> docker run --rm \
>> --entrypoint /usr/local/bin/python \
>> gcr.io/apache-beam-testing/yaml_template:dev
>> /dataflow/template/main.py \
>> --pipeline_spec="$(curl
>>
>> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
>> )"
>>
>> Though of course one would have to set up the appropriate mount points
>> to do any local filesystem io and/or credentials.
>>
>> This is also available as a Dataflow template and can be invoked as
>>
>> gcloud dataflow flex-template run \
>> "yaml-template-job" \
>>  --template-file-gcs-location
>> gs://apache-beam-testing-robertwb/yaml_template.json \
>> --parameters ^~^pipeline_spec="$(curl
>>
>> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
>> )"
>> \
>> --parameters 

Re: [PROPOSAL] Preparing for Apache Beam 2.44.0 Release

2022-12-14 Thread Kenneth Knowles
I've edited the subject for this update. There are no more open bugs
targeting the release milestone. I will prepare RC1 shortly.

Kenn

On Thu, Dec 1, 2022 at 12:55 PM Kenneth Knowles  wrote:

> Just an update that the branch is cut.
>
> There are 8 issues targeted to the release milestone:
> https://github.com/apache/beam/milestone/7 (thanks Cham for the correct
> link!)
>
> Please help to close these out or triage them off the milestone. I will be
> looking at them now.
>
> Kenn
>
> On Thu, Nov 17, 2022 at 2:27 PM Chamikara Jayalath via dev <
> dev@beam.apache.org> wrote:
>
>>
>> Thanks Kenn.
>> BTW the correct milestone for the 2.44.0 release should be this one:
>> https://github.com/apache/beam/milestone/7
>>
>> - Cham
>>
>>
>> On Thu, Nov 17, 2022 at 9:12 AM Ahmet Altay via dev 
>> wrote:
>>
>>> Thank you Kenn! :)
>>>
>>> On Wed, Nov 16, 2022 at 12:45 PM Kenneth Knowles 
>>> wrote:
>>>
 Hi all,

 The 2.44.0 release cut is scheduled for Nov 30th [1]. I'd like to
 volunteer to do this release.

 As usual, my plan would be to cut right on that date and cherry
 pick critical fixes.

 Help me and the release by:
 - Making sure that any unresolved release blocking issues for 2.44.0
 have their "Milestone" marked as "2.44.0 Release" [2].
 - Reviewing the current release blockers [2] and removing the Milestone
 if they don't meet the criteria at [3].

 Kenn

 [1]
 https://calendar.google.com/calendar/u/0/embed?src=0p73sl034k80oob7seouani...@group.calendar.google.com
 [2] https://github.com/apache/beam/milestone/5
 [3] https://beam.apache.org/contribute/release-blocking/

 Kenn

>>>


Re: A Declarative API for Apache Beam

2022-12-14 Thread Damon Douglas via dev
Hello Robert,

I'm replying to say that I've been waiting for something like this ever
since I started learning Beam and I'm grateful you are pushing this forward.

Best,

Damon

On Wed, Dec 14, 2022 at 2:05 PM Robert Bradshaw  wrote:

> While Beam provides powerful APIs for authoring sophisticated data
> processing pipelines, it often still has too high a barrier for
> getting started and authoring simple pipelines. Even setting up the
> environment, installing the dependencies, and setting up the project
> can be an overwhelming amount of boilerplate for some (though
> https://beam.apache.org/blog/beam-starter-projects/ has gone a long
> way in making this easier). At the other extreme, the Dataflow project
> has the notion of templates which are pre-built Beam pipelines that
> can be easily launched from the command line, or even from your
> browser, but they are fairly restrictive, limited to pre-assembled
> pipelines taking a small number of parameters.
>
> The idea of creating a yaml-based description of pipelines has come up
> several times in several contexts and this last week I decided to code
> up what it could look like. Here's a proposal.
>
> pipeline:
>   - type: chain
> transforms:
>   - type: ReadFromText
> args:
>  file_pattern: "wordcount.yaml"
>   - type: PyMap
> fn: "str.lower"
>   - type: PyFlatMap
> fn: "import re\nlambda line: re.findall('[a-z]+', line)"
>   - type: PyTransform
> name: Count
> constructor: "apache_beam.transforms.combiners.Count.PerElement"
>   - type: PyMap
> fn: str
>   - type: WriteToText
> file_path_prefix: "counts.txt"
>
> Some more examples at
> https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a
>
> A prototype (feedback welcome) can be found at
> https://github.com/apache/beam/pull/24667. It can be invoked as
>
> python -m apache_beam.yaml.main --pipeline_spec_file
> [path/to/file.yaml] [other_pipene_args]
>
> or
>
> python -m apache_beam.yaml.main --pipeline_spec [yaml_contents]
> [other_pipene_args]
>
> For example, to play around with this one could do
>
> python -m apache_beam.yaml.main  \
> --pipeline_spec "$(curl
>
> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
> )"
> \
> --runner=apache_beam.runners.render.RenderRunner \
> --render_out=out.png
>
> Alternatively one can run it as a docker container with no need to
> install any SDK
>
> docker run --rm \
> --entrypoint /usr/local/bin/python \
> gcr.io/apache-beam-testing/yaml_template:dev
> /dataflow/template/main.py \
> --pipeline_spec="$(curl
>
> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
> )"
>
> Though of course one would have to set up the appropriate mount points
> to do any local filesystem io and/or credentials.
>
> This is also available as a Dataflow template and can be invoked as
>
> gcloud dataflow flex-template run \
> "yaml-template-job" \
>  --template-file-gcs-location
> gs://apache-beam-testing-robertwb/yaml_template.json \
> --parameters ^~^pipeline_spec="$(curl
>
> https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml
> )"
> \
> --parameters pickle_library=cloudpickle \
> --project=apache-beam-testing \
> --region us-central1
>
> (Note the escaping required for the parameter (use cat for a local
> file), and the debug cycle here could be greatly improved, so I'd
> recommend trying things locally first.)
>
> A key point of this implementation is that it heavily uses the
> expansion service and cross language transforms, tying into the
> proposal at  https://s.apache.org/easy-multi-language . Though all the
> examples use transforms defined in the Beam SDK, any appropriately
> packaged libraries may be used.
>
> There are many ways this could be extended. For example
>
> * It would be useful to be able to templatize yaml descriptions. This
> could be done with $SIGIL type notation or some other way. This would
> even allow one to define reusable, parameterized composite PTransform
> types in yaml itself.
>
> * It would be good to have a more principled way of merging
> environments. Currently each set of dependencies is a unique Beam
> environment, and while Beam has sophisticated cross-language
> capabilities, it would be nice if environments sharing the same
> language (and likely also the same Beam version) could be fused
> in-process (e.g. with separate class loaders or compatibility checks
> for packages).
>
> * Publishing and discovery of transformations could be improved,
> possibly via shared standards and some kind of a transform catalog. An
> ecosystem of easily sharable transforms (similar to what huggingface

A Declarative API for Apache Beam

2022-12-14 Thread Robert Bradshaw via dev
While Beam provides powerful APIs for authoring sophisticated data
processing pipelines, it often still has too high a barrier for
getting started and authoring simple pipelines. Even setting up the
environment, installing the dependencies, and setting up the project
can be an overwhelming amount of boilerplate for some (though
https://beam.apache.org/blog/beam-starter-projects/ has gone a long
way in making this easier). At the other extreme, the Dataflow project
has the notion of templates which are pre-built Beam pipelines that
can be easily launched from the command line, or even from your
browser, but they are fairly restrictive, limited to pre-assembled
pipelines taking a small number of parameters.

The idea of creating a yaml-based description of pipelines has come up
several times in several contexts and this last week I decided to code
up what it could look like. Here's a proposal.

pipeline:
  - type: chain
transforms:
  - type: ReadFromText
args:
 file_pattern: "wordcount.yaml"
  - type: PyMap
fn: "str.lower"
  - type: PyFlatMap
fn: "import re\nlambda line: re.findall('[a-z]+', line)"
  - type: PyTransform
name: Count
constructor: "apache_beam.transforms.combiners.Count.PerElement"
  - type: PyMap
fn: str
  - type: WriteToText
file_path_prefix: "counts.txt"

Some more examples at
https://gist.github.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a

A prototype (feedback welcome) can be found at
https://github.com/apache/beam/pull/24667. It can be invoked as

python -m apache_beam.yaml.main --pipeline_spec_file
[path/to/file.yaml] [other_pipene_args]

or

python -m apache_beam.yaml.main --pipeline_spec [yaml_contents]
[other_pipene_args]

For example, to play around with this one could do

python -m apache_beam.yaml.main  \
--pipeline_spec "$(curl
https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml)"
\
--runner=apache_beam.runners.render.RenderRunner \
--render_out=out.png

Alternatively one can run it as a docker container with no need to
install any SDK

docker run --rm \
--entrypoint /usr/local/bin/python \
gcr.io/apache-beam-testing/yaml_template:dev
/dataflow/template/main.py \
--pipeline_spec="$(curl
https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml)"

Though of course one would have to set up the appropriate mount points
to do any local filesystem io and/or credentials.

This is also available as a Dataflow template and can be invoked as

gcloud dataflow flex-template run \
"yaml-template-job" \
 --template-file-gcs-location
gs://apache-beam-testing-robertwb/yaml_template.json \
--parameters ^~^pipeline_spec="$(curl
https://gist.githubusercontent.com/robertwb/0bab10a4ebf1001e187bbe3f5241023a/raw/e08dc4ccdf7c7ec9ea607e530ce6fd8f40109d3a/math.yaml)"
\
--parameters pickle_library=cloudpickle \
--project=apache-beam-testing \
--region us-central1

(Note the escaping required for the parameter (use cat for a local
file), and the debug cycle here could be greatly improved, so I'd
recommend trying things locally first.)

A key point of this implementation is that it heavily uses the
expansion service and cross language transforms, tying into the
proposal at  https://s.apache.org/easy-multi-language . Though all the
examples use transforms defined in the Beam SDK, any appropriately
packaged libraries may be used.

There are many ways this could be extended. For example

* It would be useful to be able to templatize yaml descriptions. This
could be done with $SIGIL type notation or some other way. This would
even allow one to define reusable, parameterized composite PTransform
types in yaml itself.

* It would be good to have a more principled way of merging
environments. Currently each set of dependencies is a unique Beam
environment, and while Beam has sophisticated cross-language
capabilities, it would be nice if environments sharing the same
language (and likely also the same Beam version) could be fused
in-process (e.g. with separate class loaders or compatibility checks
for packages).

* Publishing and discovery of transformations could be improved,
possibly via shared standards and some kind of a transform catalog. An
ecosystem of easily sharable transforms (similar to what huggingface
provides for ML models) could provide a useful platform for making it
easy to build pipelines and open up Beam to a whole new set of users.

Let me know what you think.

- Robert


DRAFT - Apache Beam Board Report - December 2022

2022-12-14 Thread Kenneth Knowles
Hi all,

The next Beam board report is due this Friday, December 16. Please help me
to draft it at https://s.apache.org/beam-draft-report-2022-12. I've opened
edit access to anyone with the link to minimize friction of drafting.

Ideas:

 - highlights from CHANGES.md
 - interesting technical discussions
 - integrations with other projects
 - community events
 - major user facing addition/deprecation

Past reports are at https://whimsy.apache.org/board/minutes/Beam.html for
examples.

I will edit the final version from everyone's suggestions.

Thanks,

Kenn


Re: [Proposal] | Move FileIO and TextIO from :sdks:java:core to :sdks:java:io:file

2022-12-14 Thread Ahmet Altay via dev
I agree with Sachin. Keeping components that users will have to bring
together anyway leads to a better user experience. Counter example to that
is GCP libraries in my opinion. It was a frequent struggle for users to
find a working set of libraries until there was a BOM. And even after the
BOM it is still somewhat of a struggle for users and the developers of
those various libraries need to take on some of the toil of testing those
various libraries together anyway.

re: Talk it with a grain of salt since I'm not even a committer - All
inputs are welcome here. I do not think my comments should carry more
weight just because I am a committer.

On Wed, Dec 14, 2022 at 9:36 AM Sachin Agarwal via dev 
wrote:

> I strongly believe that we should continue to have Beam optimize for the
> user - and while having separate components would allow those of us who are
> contributors and committers move faster, the downsides of not having
> everything "in one box" for a new user where the components are all
> relatively guaranteed to work together at that version level are very high.
>
> Beam having everything included is absolutely a competitive advantage for
> Beam and I would not want to lose that.
>
> On Wed, Dec 14, 2022 at 9:31 AM Byron Ellis via dev 
> wrote:
>
>> Talk it with a grain of salt since I'm not even a committer, but is
>> perhaps the reorganization of Beam into smaller components the real work of
>> a 3.0 effort? Splitting of Beam into smaller more independently managed
>> components would be a pretty huge breaking change from a dependency
>> management perspective which would potentially be largely separate from any
>> code changes.
>>
>> Best,
>> B
>>
>> On Wed, Dec 14, 2022 at 9:23 AM Alexey Romanenko <
>> aromanenko@gmail.com> wrote:
>>
>>> On 12 Dec 2022, at 22:23, Robert Bradshaw via dev 
>>> wrote:
>>>
>>>
>>> Saving up all the breaking changes until a major release definitely
>>> has its downsides (look at Python 3). The migration path is often as
>>> important (if not more so) than the final destination.
>>>
>>>
>>> Actually, it proves that the major releases *should not* be delayed for
>>> a long period of time and *should* be issued more often to reduce the
>>> number of breaking changes (that, of course, likely may happen). That will
>>> help users to do much more smooth and less risky upgrades, and developers
>>> to not keep burden forever. Beam 2.0.0 was released back in may 2017 and
>>> we've almost never talked about Beam 3.0 and what are the criteria for it.
>>> I understand that it’s a completely different discussion but seems that
>>> this time has come =)
>>>
>>> As for this particular change, I would question how the benefit (it's
>>> unclear what the exact benefit is--better internal organization?)
>>> exceeds the pain of making every user refactor their code. I think a
>>> stronger case can be made for things like the Avro dependency that
>>> cause real pain.
>>>
>>>
>>> Agree. I think that if it doesn’t bring any pain with additional
>>> external dependecies and this code is used in almost every other SDK
>>> module, then there are no reasons for such breaking changes. On the other
>>> hand, Avro case, that you mentioned above, is a good example why sometimes
>>> it would be better to keep such code outside of “core”.
>>>
>>> As for the pipeline update feature, we've long discussed having
>>> "pick-your-implementation" transforms that specify alternative,
>>> equivalent implementations. Upgrades can choose the old one whereas
>>> new pipelines can get the latest and greatest. It won't solve all
>>> issues, and requires keeping old codepaths around, but could be an
>>> important step forward.
>>>
>>> On Mon, Dec 12, 2022 at 10:20 AM Kenneth Knowles 
>>> wrote:
>>>
>>>
>>> I agree with Mortiz. To answer a few specifics in my own words:
>>>
>>> - It is a perfectly sensible refactor, but as a counterpoint without
>>> file-based IO the SDK isn't functional so it is also a reasonable design
>>> point to have this included. There are other things in the core SDK that
>>> are far less "core" and could be moved out with greater benefit. The main
>>> goal for any separation of modules would be lighter weight transitive
>>> dependencies, IMO.
>>>
>>> - No, Beam has not made any deliberate breaking changes of this nature.
>>> Hence we are still on major version 2. We have made some bugfixes for data
>>> loss risks that could be called "breaking changes" but since the feature
>>> was unsafe to use in the first place we did not bump the major version.
>>>
>>> - It is sometimes possible to do such a refactor and have the deprecated
>>> location proxy to the new location. In this case that seems hard to achieve.
>>>
>>> - It is not actually necessary to maintain both locations, as we can
>>> declare the old location will be unmaintained (but left alone) and all new
>>> development goes to the new location. That isn't a great choice for users
>>> who may simply upgrade their SDK version 

Re: [Proposal] | Move FileIO and TextIO from :sdks:java:core to :sdks:java:io:file

2022-12-14 Thread Sachin Agarwal via dev
I strongly believe that we should continue to have Beam optimize for the
user - and while having separate components would allow those of us who are
contributors and committers move faster, the downsides of not having
everything "in one box" for a new user where the components are all
relatively guaranteed to work together at that version level are very high.

Beam having everything included is absolutely a competitive advantage for
Beam and I would not want to lose that.

On Wed, Dec 14, 2022 at 9:31 AM Byron Ellis via dev 
wrote:

> Talk it with a grain of salt since I'm not even a committer, but is
> perhaps the reorganization of Beam into smaller components the real work of
> a 3.0 effort? Splitting of Beam into smaller more independently managed
> components would be a pretty huge breaking change from a dependency
> management perspective which would potentially be largely separate from any
> code changes.
>
> Best,
> B
>
> On Wed, Dec 14, 2022 at 9:23 AM Alexey Romanenko 
> wrote:
>
>> On 12 Dec 2022, at 22:23, Robert Bradshaw via dev 
>> wrote:
>>
>>
>> Saving up all the breaking changes until a major release definitely
>> has its downsides (look at Python 3). The migration path is often as
>> important (if not more so) than the final destination.
>>
>>
>> Actually, it proves that the major releases *should not* be delayed for
>> a long period of time and *should* be issued more often to reduce the
>> number of breaking changes (that, of course, likely may happen). That will
>> help users to do much more smooth and less risky upgrades, and developers
>> to not keep burden forever. Beam 2.0.0 was released back in may 2017 and
>> we've almost never talked about Beam 3.0 and what are the criteria for it.
>> I understand that it’s a completely different discussion but seems that
>> this time has come =)
>>
>> As for this particular change, I would question how the benefit (it's
>> unclear what the exact benefit is--better internal organization?)
>> exceeds the pain of making every user refactor their code. I think a
>> stronger case can be made for things like the Avro dependency that
>> cause real pain.
>>
>>
>> Agree. I think that if it doesn’t bring any pain with additional external
>> dependecies and this code is used in almost every other SDK module, then
>> there are no reasons for such breaking changes. On the other hand, Avro
>> case, that you mentioned above, is a good example why sometimes it would be
>> better to keep such code outside of “core”.
>>
>> As for the pipeline update feature, we've long discussed having
>> "pick-your-implementation" transforms that specify alternative,
>> equivalent implementations. Upgrades can choose the old one whereas
>> new pipelines can get the latest and greatest. It won't solve all
>> issues, and requires keeping old codepaths around, but could be an
>> important step forward.
>>
>> On Mon, Dec 12, 2022 at 10:20 AM Kenneth Knowles  wrote:
>>
>>
>> I agree with Mortiz. To answer a few specifics in my own words:
>>
>> - It is a perfectly sensible refactor, but as a counterpoint without
>> file-based IO the SDK isn't functional so it is also a reasonable design
>> point to have this included. There are other things in the core SDK that
>> are far less "core" and could be moved out with greater benefit. The main
>> goal for any separation of modules would be lighter weight transitive
>> dependencies, IMO.
>>
>> - No, Beam has not made any deliberate breaking changes of this nature.
>> Hence we are still on major version 2. We have made some bugfixes for data
>> loss risks that could be called "breaking changes" but since the feature
>> was unsafe to use in the first place we did not bump the major version.
>>
>> - It is sometimes possible to do such a refactor and have the deprecated
>> location proxy to the new location. In this case that seems hard to achieve.
>>
>> - It is not actually necessary to maintain both locations, as we can
>> declare the old location will be unmaintained (but left alone) and all new
>> development goes to the new location. That isn't a great choice for users
>> who may simply upgrade their SDK version and not notice that their old code
>> is now pointing at a version that will not receive e.g. security updates.
>>
>> - I like the style where if/when we transition from Beam 2 to Beam 3 we
>> should have the exact functionality of Beam 3 available as an opt-in flag
>> first. So if a user passes --beam-3 they get exactly what will be the
>> default functionality when we bump the major version. It really is a
>> problem to do a whole bunch of stuff feverishly before a major version
>> bump. The other style that I think works well is the linux kernel style
>> where major versions alternate between stable and unstable (in other words,
>> returning to the 0.x style with every alternating version).
>>
>> - I do think Beam suffers from fear and inability to do significant code
>> gardening. I don't think backwards compatibility in the 

[Question] Github Actions Migration - Error with tests that use cython in self hosted runners.

2022-12-14 Thread Andoni Guzman Becerra
Hi All!
We are working on the effort to migrate tests from jenkins to github
actions in self hosted runners, but we are facing an issue related with
tests that use cython and tox.
This only happens in our self hosted runners, not in another runner like
"ubuntu-latest" from github. Our self hosted runners are based on
"ubuntu-latest" runner from Github
We tried installing the pythondev package and other dependencies but the
error is still present.
This is the error
gcc -pthread -Wno-unused-result -Wsign-compare -DNDEBUG -g -fwrapv -O3
-Wall -fPIC
-I/home/actions/_work/beam/beam/sdks/python/test-suites/tox/py37/build/srcs/sdks/python/target/.tox-py37-cython/py37-cython/lib/python3.7/site-packages/numpy/core/include
-I/home/actions/_work/beam/beam/sdks/python/test-suites/tox/py37/build/srcs/sdks/python/target/.tox-py37-cython/py37-cython/include
-I/opt/hostedtoolcache/Python/3.7.15/x64/include/python3.7m -c
apache_beam/coders/coder_impl_row_encoders.c -o
build/temp.linux-x86_64-cpython-37/apache_beam/coders/coder_impl_row_encoders.o

2509

apache_beam/coders/coder_impl_row_encoders.c:19:10: fatal error: Python.h:
No such file or directory
2510

19 | #include "Python.h"
2511

| ^~
2512

compilation terminated.
2513

error: command '/usr/bin/gcc' failed with exit code 1
2514

Any idea of what is missing? Maybe something related to tox?

Thanks!



-- 

Andoni Guzman | WIZELINE

Software Engineer II

andoni.guz...@wizeline.com

Amado Nervo 2200, Esfera P6, Col. Ciudad del Sol, 45050 Zapopan, Jal.

-- 
*This email and its contents (including any attachments) are being sent to
you on the condition of confidentiality and may be protected by legal
privilege. Access to this email by anyone other than the intended recipient
is unauthorized. If you are not the intended recipient, please immediately
notify the sender by replying to this message and delete the material
immediately from your system. Any further use, dissemination, distribution
or reproduction of this email is strictly prohibited. Further, no
representation is made with respect to any content contained in this email.*


Re: [Proposal] | Move FileIO and TextIO from :sdks:java:core to :sdks:java:io:file

2022-12-14 Thread Byron Ellis via dev
Talk it with a grain of salt since I'm not even a committer, but is perhaps
the reorganization of Beam into smaller components the real work of a 3.0
effort? Splitting of Beam into smaller more independently managed
components would be a pretty huge breaking change from a dependency
management perspective which would potentially be largely separate from any
code changes.

Best,
B

On Wed, Dec 14, 2022 at 9:23 AM Alexey Romanenko 
wrote:

> On 12 Dec 2022, at 22:23, Robert Bradshaw via dev 
> wrote:
>
>
> Saving up all the breaking changes until a major release definitely
> has its downsides (look at Python 3). The migration path is often as
> important (if not more so) than the final destination.
>
>
> Actually, it proves that the major releases *should not* be delayed for a
> long period of time and *should* be issued more often to reduce the
> number of breaking changes (that, of course, likely may happen). That will
> help users to do much more smooth and less risky upgrades, and developers
> to not keep burden forever. Beam 2.0.0 was released back in may 2017 and
> we've almost never talked about Beam 3.0 and what are the criteria for it.
> I understand that it’s a completely different discussion but seems that
> this time has come =)
>
> As for this particular change, I would question how the benefit (it's
> unclear what the exact benefit is--better internal organization?)
> exceeds the pain of making every user refactor their code. I think a
> stronger case can be made for things like the Avro dependency that
> cause real pain.
>
>
> Agree. I think that if it doesn’t bring any pain with additional external
> dependecies and this code is used in almost every other SDK module, then
> there are no reasons for such breaking changes. On the other hand, Avro
> case, that you mentioned above, is a good example why sometimes it would be
> better to keep such code outside of “core”.
>
> As for the pipeline update feature, we've long discussed having
> "pick-your-implementation" transforms that specify alternative,
> equivalent implementations. Upgrades can choose the old one whereas
> new pipelines can get the latest and greatest. It won't solve all
> issues, and requires keeping old codepaths around, but could be an
> important step forward.
>
> On Mon, Dec 12, 2022 at 10:20 AM Kenneth Knowles  wrote:
>
>
> I agree with Mortiz. To answer a few specifics in my own words:
>
> - It is a perfectly sensible refactor, but as a counterpoint without
> file-based IO the SDK isn't functional so it is also a reasonable design
> point to have this included. There are other things in the core SDK that
> are far less "core" and could be moved out with greater benefit. The main
> goal for any separation of modules would be lighter weight transitive
> dependencies, IMO.
>
> - No, Beam has not made any deliberate breaking changes of this nature.
> Hence we are still on major version 2. We have made some bugfixes for data
> loss risks that could be called "breaking changes" but since the feature
> was unsafe to use in the first place we did not bump the major version.
>
> - It is sometimes possible to do such a refactor and have the deprecated
> location proxy to the new location. In this case that seems hard to achieve.
>
> - It is not actually necessary to maintain both locations, as we can
> declare the old location will be unmaintained (but left alone) and all new
> development goes to the new location. That isn't a great choice for users
> who may simply upgrade their SDK version and not notice that their old code
> is now pointing at a version that will not receive e.g. security updates.
>
> - I like the style where if/when we transition from Beam 2 to Beam 3 we
> should have the exact functionality of Beam 3 available as an opt-in flag
> first. So if a user passes --beam-3 they get exactly what will be the
> default functionality when we bump the major version. It really is a
> problem to do a whole bunch of stuff feverishly before a major version
> bump. The other style that I think works well is the linux kernel style
> where major versions alternate between stable and unstable (in other words,
> returning to the 0.x style with every alternating version).
>
> - I do think Beam suffers from fear and inability to do significant code
> gardening. I don't think backwards compatibility in the code sense is the
> biggest blocker. I think the "pipeline update" feature is perhaps the thing
> most holding Beam back from making radical rapid forward progress.
>
> Kenn
>
> On Mon, Dec 12, 2022 at 2:25 AM Moritz Mack  wrote:
>
>
> Hi Damon,
>
>
>
> I fear the current release / versioning strategy of Beam doesn’t lend
> itself well for such breaking changes. Alexey and I have spent quite some
> time discussing how to proceed with the problematic Avro dependency in core
> (and respectively AvroIO, of course).
>
> Such changes essentially always require duplicating code to continue
> supporting a deprecated legacy code path to not 

Re: [Proposal] | Move FileIO and TextIO from :sdks:java:core to :sdks:java:io:file

2022-12-14 Thread Alexey Romanenko
On 12 Dec 2022, at 22:23, Robert Bradshaw via dev  wrote:
> 
> Saving up all the breaking changes until a major release definitely
> has its downsides (look at Python 3). The migration path is often as
> important (if not more so) than the final destination.

Actually, it proves that the major releases should not be delayed for a long 
period of time and should be issued more often to reduce the number of breaking 
changes (that, of course, likely may happen). That will help users to do much 
more smooth and less risky upgrades, and developers to not keep burden forever. 
Beam 2.0.0 was released back in may 2017 and we've almost never talked about 
Beam 3.0 and what are the criteria for it. I understand that it’s a completely 
different discussion but seems that this time has come =)

> As for this particular change, I would question how the benefit (it's
> unclear what the exact benefit is--better internal organization?)
> exceeds the pain of making every user refactor their code. I think a
> stronger case can be made for things like the Avro dependency that
> cause real pain.

Agree. I think that if it doesn’t bring any pain with additional external 
dependecies and this code is used in almost every other SDK module, then there 
are no reasons for such breaking changes. On the other hand, Avro case, that 
you mentioned above, is a good example why sometimes it would be better to keep 
such code outside of “core”.

> As for the pipeline update feature, we've long discussed having
> "pick-your-implementation" transforms that specify alternative,
> equivalent implementations. Upgrades can choose the old one whereas
> new pipelines can get the latest and greatest. It won't solve all
> issues, and requires keeping old codepaths around, but could be an
> important step forward.
> 
> On Mon, Dec 12, 2022 at 10:20 AM Kenneth Knowles  wrote:
>> 
>> I agree with Mortiz. To answer a few specifics in my own words:
>> 
>> - It is a perfectly sensible refactor, but as a counterpoint without 
>> file-based IO the SDK isn't functional so it is also a reasonable design 
>> point to have this included. There are other things in the core SDK that are 
>> far less "core" and could be moved out with greater benefit. The main goal 
>> for any separation of modules would be lighter weight transitive 
>> dependencies, IMO.
>> 
>> - No, Beam has not made any deliberate breaking changes of this nature. 
>> Hence we are still on major version 2. We have made some bugfixes for data 
>> loss risks that could be called "breaking changes" but since the feature was 
>> unsafe to use in the first place we did not bump the major version.
>> 
>> - It is sometimes possible to do such a refactor and have the deprecated 
>> location proxy to the new location. In this case that seems hard to achieve.
>> 
>> - It is not actually necessary to maintain both locations, as we can declare 
>> the old location will be unmaintained (but left alone) and all new 
>> development goes to the new location. That isn't a great choice for users 
>> who may simply upgrade their SDK version and not notice that their old code 
>> is now pointing at a version that will not receive e.g. security updates.
>> 
>> - I like the style where if/when we transition from Beam 2 to Beam 3 we 
>> should have the exact functionality of Beam 3 available as an opt-in flag 
>> first. So if a user passes --beam-3 they get exactly what will be the 
>> default functionality when we bump the major version. It really is a problem 
>> to do a whole bunch of stuff feverishly before a major version bump. The 
>> other style that I think works well is the linux kernel style where major 
>> versions alternate between stable and unstable (in other words, returning to 
>> the 0.x style with every alternating version).
>> 
>> - I do think Beam suffers from fear and inability to do significant code 
>> gardening. I don't think backwards compatibility in the code sense is the 
>> biggest blocker. I think the "pipeline update" feature is perhaps the thing 
>> most holding Beam back from making radical rapid forward progress.
>> 
>> Kenn
>> 
>> On Mon, Dec 12, 2022 at 2:25 AM Moritz Mack  wrote:
>>> 
>>> Hi Damon,
>>> 
>>> 
>>> 
>>> I fear the current release / versioning strategy of Beam doesn’t lend 
>>> itself well for such breaking changes. Alexey and I have spent quite some 
>>> time discussing how to proceed with the problematic Avro dependency in core 
>>> (and respectively AvroIO, of course).
>>> 
>>> Such changes essentially always require duplicating code to continue 
>>> supporting a deprecated legacy code path to not break users’ code. But this 
>>> comes at a very high price. Until the deprecated code path can be finally 
>>> removed again, it must be maintained in two places.
>>> 
>>> Unfortunately, the removal of deprecated code is rather problematic without 
>>> a major version release as it would break semantic versioning and people’s 
>>> expectations. With that deprecations 

SingleStoreIO SchemaTransform

2022-12-14 Thread Adalbert Makarovych
Hi

Can someone review this PR https://github.com/apache/beam/pull/24290
Thanks for your attention.

-- 
Adalbert Makarovych
Software Engineer at SingleStore




Beam High Priority Issue Report (30)

2022-12-14 Thread beamactions
This is your daily summary of Beam's current high priority issues that may need 
attention.

See https://beam.apache.org/contribute/issue-priorities for the meaning and 
expectations around issue priorities.

Unassigned P1 Issues:

https://github.com/apache/beam/issues/24655 [Bug]: Pipeline fusion should break 
at @RequiresStableInput boundary
https://github.com/apache/beam/issues/24649 [Bug]: serviceAccount option was 
dropped from PipelineOptions DisplayData starting Beam 2.32.0 
https://github.com/apache/beam/issues/24389 [Failing Test]: 
HadoopFormatIOElasticTest.classMethod ExceptionInInitializerError 
ContainerFetchException
https://github.com/apache/beam/issues/24367 [Bug]: workflow.tar.gz cannot be 
passed to flink runner
https://github.com/apache/beam/issues/24313 [Flaky]: 
apache_beam/runners/portability/portable_runner_test.py::PortableRunnerTestWithSubprocesses::test_pardo_state_with_custom_key_coder
https://github.com/apache/beam/issues/24267 [Failing Test]: Timeout waiting to 
lock gradle
https://github.com/apache/beam/issues/23944  beam_PreCommit_Python_Cron 
regularily failing - test_pardo_large_input flaky
https://github.com/apache/beam/issues/23709 [Flake]: Spark batch flakes in 
ParDoLifecycleTest.testTeardownCalledAfterExceptionInProcessElement and 
ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundle
https://github.com/apache/beam/issues/23286 [Bug]: 
beam_PerformanceTests_InfluxDbIO_IT Flaky > 50 % Fail 
https://github.com/apache/beam/issues/22969 Discrepancy in behavior of 
`DoFn.process()` when `yield` is combined with `return` statement, or vice versa
https://github.com/apache/beam/issues/22961 [Bug]: WriteToBigQuery silently 
skips most of records without job fail
https://github.com/apache/beam/issues/22913 [Bug]: 
beam_PostCommit_Java_ValidatesRunner_Flink is flakes in 
org.apache.beam.sdk.transforms.GroupByKeyTest$BasicTests.testAfterProcessingTimeContinuationTriggerUsingState
https://github.com/apache/beam/issues/21713 404s in BigQueryIO don't get output 
to Failed Inserts PCollection
https://github.com/apache/beam/issues/21695 DataflowPipelineResult does not 
raise exception for unsuccessful states.
https://github.com/apache/beam/issues/21643 FnRunnerTest with non-trivial 
(order 1000 elements) numpy input flakes in non-cython environment
https://github.com/apache/beam/issues/21469 beam_PostCommit_XVR_Flink flaky: 
Connection refused
https://github.com/apache/beam/issues/21262 Python AfterAny, AfterAll do not 
follow spec
https://github.com/apache/beam/issues/21260 Python DirectRunner does not emit 
data at GC time
https://github.com/apache/beam/issues/21121 
apache_beam.examples.streaming_wordcount_it_test.StreamingWordCountIT.test_streaming_wordcount_it
 flakey
https://github.com/apache/beam/issues/21104 Flaky: 
apache_beam.runners.portability.fn_api_runner.fn_runner_test.FnApiRunnerTestWithGrpcAndMultiWorkers
https://github.com/apache/beam/issues/20976 
apache_beam.runners.portability.flink_runner_test.FlinkRunnerTestOptimized.test_flink_metrics
 is flaky
https://github.com/apache/beam/issues/20974 Python GHA PreCommits flake with 
grpc.FutureTimeoutError on SDK harness startup
https://github.com/apache/beam/issues/20689 Kafka commitOffsetsInFinalize OOM 
on Flink
https://github.com/apache/beam/issues/20108 Python direct runner doesn't emit 
empty pane when it should
https://github.com/apache/beam/issues/19814 Flink streaming flakes in 
ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundleStateful and 
ParDoLifecycleTest.testTeardownCalledAfterExceptionInProcessElementStateful
https://github.com/apache/beam/issues/19465 Explore possibilities to lower 
in-use IP address quota footprint.
https://github.com/apache/beam/issues/19241 Python Dataflow integration tests 
should export the pipeline Job ID and console output to Jenkins Test Result 
section


P1 Issues with no update in the last week:

https://github.com/apache/beam/issues/24464 [Epic]: Implement 
FileWriteSchemaTransformProvider
https://github.com/apache/beam/issues/21714 
PulsarIOTest.testReadFromSimpleTopic is very flaky
https://github.com/apache/beam/issues/21424 Java VR (Dataflow, V2, Streaming) 
failing: ParDoTest$TimestampTests/OnWindowExpirationTests




Re: @RequiresStableInput and Pipeline fusion

2022-12-14 Thread Jan Lukavský

Filled https://github.com/apache/beam/issues/24655.

 Jan

On 12/14/22 00:52, Luke Cwik via dev wrote:
This is definitely not working for portable pipelines since the 
GreedyPipelineFuser doesn't create a fusion boundary which as you 
pointed out causes a single stage that has a non-deterministic 
function followed by one that requires stable input. It seems as 
though we should have runners check the requirements on the 
Pipeline[1] to ensure that they can faithfully process such a pipeline 
and reject anything they don't support early on.


Making the GreedyPipelineFuser insert that fusion break is likely the 
way to go. Runners should be able to look at the ParDoPayload 
requires_stable_input field for the ExecutableStage to see if any 
special handling is necessary on their end before they pass data to 
that stage.


[1]: 
https://github.com/apache/beam/blob/77af3237521d94f0399ab405ebac09bbbeded38c/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto#L111 



On Tue, Dec 13, 2022 at 1:44 AM Jan Lukavský  wrote:

Hi,

I have a question about @RequiresStableInput functionality. I'm
trying to make it work for portable Flink runner [1], [2]. We have
an integration test (which should probably be turned into
Validates runner test, but that is a different story) [3]. The
test creates random key for input element, processes it once,
fails the pipeline and then reprocesses it. This works well
provided there is a checkpoint (shuffle in case of dataflow)
exactly between assigning random key (via PairWithRandomKeyFn) and
processing it with (via MakeSideEffectAndThenFailFn), this works well.

The problem is that GreedyPipelineFuser fuses the transform
PairWithRandomKeyFn andMakeSideEffectAndThenFailFn into single
ExecutableStage. This is then executed with the
@RequiresStableInput requirement, but this obviously assigns a
different key to the reprocessed element(s). This looks like we
need to fix that in the PipelineFuser, is this right? Does this
mean the @RequiresStableInput functionality is actually broken for
all runners that use the default fusion?

Another possibility is that we need to fix test by adding an
explicit reshuffle (verified, this works), but I think that the
test is actually correct, users would probably not expect
transforms to be fused when crossing the @RequiresStableInput
boundary.

Thoughts?

 Jan


[1] https://github.com/apache/beam/issues/20812
[2] https://github.com/apache/beam/pull/22889
[3]

https://github.com/apache/beam/blob/master/sdks/java/core/src/test/java/org/apache/beam/sdk/RequiresStableInputIT.java