Skip to content

Commit b20384b

Browse files
committed
add windowing support when passing in a json string
1 parent a4b6203 commit b20384b

File tree

1 file changed

+14
-0
lines changed

1 file changed

+14
-0
lines changed

sdks/python/apache_beam/yaml/yaml_transform.py

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1033,6 +1033,20 @@ def preprocess_windowing(spec):
10331033
if 'windowing' in spec:
10341034
spec['config'] = spec.get('config', {})
10351035
spec['config']['windowing'] = spec.pop('windowing')
1036+
1037+
if spec.get('config', {}).get('windowing'):
1038+
windowing_config = spec['config']['windowing']
1039+
if isinstance(windowing_config, str):
1040+
try:
1041+
# PyYAML can load a JSON string.
1042+
parsed_config = yaml.safe_load(windowing_config)
1043+
if not isinstance(parsed_config, dict):
1044+
raise TypeError('Windowing config string must be a YAML/JSON map.')
1045+
spec['config']['windowing'] = parsed_config
1046+
except Exception as e:
1047+
raise ValueError(
1048+
f'Error parsing windowing config string at \
1049+
{identify_object(spec)}: {e}') from e
10361050
return spec
10371051
elif 'windowing' not in spec:
10381052
# Nothing to do.

0 commit comments

Comments
 (0)