Polber opened a new pull request, #30117:
URL: https://github.com/apache/beam/pull/30117
This PR adds support for defining **python** `callable` parameters for the
built-in `Combine` transform as well as any transform defined using
`PyTransform` or `ExternalProvider` registered with `python` or `pythonPackage`.
External transforms defined using a `python` registered `ExternalProvider`
use the `InlineProvider` when no python packages are defined. In this case, a
special flag is passed to `InlineProvider` to tell it to allow `callable`
parameters. This is to avoid interfering with built-in transform behavior,
while allowing a user to expose the functionality within their own custom
transform provider.
This will allow for more complex pipelines to be defined such as the one
below:
```
pipeline:
type: chain
transforms:
- type: Create
config:
elements:
- {'season': spring, 'produce': '🥕 Carrot'}
- {'season': spring, 'produce': '🍓 Strawberry'}
- {'season': summer, 'produce': '🥕 Carrot'}
- {'season': summer, 'produce': '🌽 Corn'}
- {'season': summer, 'produce': '🍏 Green apple'}
- {'season': fall, 'produce': '🥕 Carrot'}
- {'season': fall, 'produce': '🍏 Green apple'}
- {'season': winter, 'produce': '🍆 Eggplant'}
- type: ToCase
config:
fn_map:
upper:
callable: str.upper
lower:
callable: str.lower
suffix: 'x'
- type: Combine
config:
language: python
group_by: season
combine:
shortest:
value: produce
fn:
type: apache_beam.transforms.combiners.TopCombineFn
config:
n: 2
key:
callable: 'lambda x: len(x)'
reverse: true
- type: Explode
config:
fields: [ shortest ]
- type: Filter
config:
language: python
keep:
callable: "lambda x: x.season == 'SPRINGx'"
- type: PyTransform
config:
constructor: apache_beam.transforms.core.Map
kwargs:
fn:
callable: print
options:
yaml_experimental_features: Combine
providers:
- type: python
config: {}
transforms:
'ToCase': |
@beam.ptransform_fn
def ToCase(pcoll, fn_map, suffix=''):
return pcoll | beam.Map(lambda x:
beam.Row(season=fn_map['upper'](x.season) + suffix, produce=x.produce))
```
Running the above pipeline prints the following output:
```
Row(season='SPRINGx', shortest='🥕 Carrot')
Row(season='SPRINGx', shortest='🍓 Strawberry')
```
------------------------
Thank you for your contribution! Follow this checklist to help us
incorporate your contribution quickly and easily:
- [ ] Mention the appropriate issue in your description (for example:
`addresses #123`), if applicable. This will automatically add a link to the
pull request in the issue. If you would like the issue to automatically close
on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
- [ ] Update `CHANGES.md` with noteworthy changes.
- [ ] If this contribution is large, please file an Apache [Individual
Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
See the [Contributor Guide](https://beam.apache.org/contribute) for more
tips on [how to make review process
smoother](https://github.com/apache/beam/blob/master/CONTRIBUTING.md#make-the-reviewers-job-easier).
To check the build health, please visit
[https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
GitHub Actions Tests Status (on master branch)
------------------------------------------------------------------------------------------------
[](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
[](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more
information about GitHub Actions CI or the [workflows
README](https://github.com/apache/beam/blob/master/.github/workflows/README.md)
to see a list of phrases to trigger workflows.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]