Skip to content

Commit

Permalink
[yaml] Doc improvements (#32117)
Browse files Browse the repository at this point in the history
* [yaml] explicitly define beam imports for callables in docs

Signed-off-by: Jeffrey Kinard <[email protected]>

* [yaml] add windowing time suffix in docs

Signed-off-by: Jeffrey Kinard <[email protected]>

* [yaml] update gcloud yaml run command to use non-beta

Signed-off-by: Jeffrey Kinard <[email protected]>

* [yaml] fix sql combine to align with doc example

Signed-off-by: Jeffrey Kinard <[email protected]>

* remove import beam from yaml docs

Signed-off-by: Jeffrey Kinard <[email protected]>

---------

Signed-off-by: Jeffrey Kinard <[email protected]>
  • Loading branch information
Polber authored Aug 15, 2024
1 parent ced67ec commit 65550a7
Show file tree
Hide file tree
Showing 4 changed files with 5 additions and 7 deletions.
2 changes: 1 addition & 1 deletion sdks/python/apache_beam/yaml/yaml_combine.py
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ def normalize_combine(spec):
def normalize_agg(dest, agg):
if isinstance(agg, str):
agg = {'fn': agg}
if 'value' not in agg and spec.get('language') != 'sql':
if 'value' not in agg and config.get('language') != 'sql':
agg['value'] = dest
if isinstance(agg['fn'], str):
agg['fn'] = {'type': agg['fn']}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ field name
col2: sum
```

One can aggregate over may fields at once
One can aggregate over many fields at once

```
- type: Combine
Expand Down Expand Up @@ -103,7 +103,7 @@ As with all transforms, `Combine` can take a windowing parameter
- type: Combine
windowing:
type: fixed
size: 60
size: 60s
config:
group_by: col1
combine:
Expand All @@ -118,7 +118,7 @@ parameters from upstream, e.g.
- type: WindowInto
windowing:
type: fixed
size: 60
size: 60s
- type: Combine
config:
group_by: col1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,8 +103,6 @@ is interpreted as the Python code. For example
constructor: __constructor__
kwargs:
source: |
import apache_beam as beam
def create_my_transform(inc):
return beam.Map(lambda x: beam.Row(a=x.col2 + inc))
Expand Down
2 changes: 1 addition & 1 deletion website/www/site/content/en/documentation/sdks/yaml.md
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ from the YAML file, use the
command:

```
gcloud beta dataflow yaml run $JOB_NAME \
gcloud dataflow yaml run $JOB_NAME \
--yaml-pipeline-file=pipeline.yaml \
--region=$REGION
```
Expand Down

0 comments on commit 65550a7

Please sign in to comment.