Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[yaml] add use-case example #30896

Merged
merged 3 commits into from
Apr 11, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
49 changes: 49 additions & 0 deletions sdks/python/apache_beam/yaml/examples/simple_filter.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
# coding=utf-8
Polber marked this conversation as resolved.
Show resolved Hide resolved
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the# Row(word='License'); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an# Row(word='AS IS' BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

# This examples reads from a public file stored on Google Cloud. This
# requires authenticating with Google Cloud, or setting the file in
#`ReadFromText` to a local file.
#
# To set up Application Default Credentials,
# see https://cloud.google.com/docs/authentication/external/set-up-adc for more
# information
#
# The following example reads mock transaction data from resources/products.csv
# then performs a simple filter for "Electronics".
pipeline:
transforms:
- type: ReadFromCsv
name: ReadInputFile
config:
path: gs://apache-beam-samples/beam-yaml-blog/products.csv
- type: Filter
name: FilterWithCategory
input: ReadInputFile
config:
language: python
keep: category == "Electronics"
- type: WriteToCsv
name: WriteOutputFile
input: FilterWithCategory
config:
path: output

# Expected:
# Row(transaction_id='T0012', product_name='Headphones', category='Electronics', price=59.99)
# Row(transaction_id='T0104', product_name='Headphones', category='Electronics', price=59.99)
# Row(transaction_id='T0302', product_name='Monitor', category='Electronics', price=249.99)
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
# coding=utf-8
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the# Row(word='License'); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an# Row(word='AS IS' BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

# This examples reads from a public file stored on Google Cloud. This
# requires authenticating with Google Cloud, or setting the file in
#`ReadFromText` to a local file.
#
# To set up Application Default Credentials,
# see https://cloud.google.com/docs/authentication/external/set-up-adc for more
# information
#
# The following example reads mock transaction data from resources/products.csv,
# performs a simple filter for "Electronics", then calculates the revenue and
# number of products sold for each product type.
pipeline:
transforms:
- type: ReadFromCsv
name: ReadInputFile
config:
path: gs://apache-beam-samples/beam-yaml-blog/products.csv
- type: Filter
name: FilterWithCategory
input: ReadInputFile
config:
language: python
keep: category == "Electronics"
- type: Combine
name: CountNumberSold
input: FilterWithCategory
config:
group_by: product_name
combine:
num_sold:
value: product_name
fn: count
total_revenue:
value: price
fn: sum
- type: WriteToCsv
name: WriteOutputFile
input: CountNumberSold
config:
path: output

options:
yaml_experimental_features: Combine

# Expected:
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

README.md can we update this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Update it how?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add this to the list?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not really a list, it is just saying there are examples that highlight specific transforms in the transforms/ subdirectory and to start with Wordcount in the root directory. All other examples are also in the root directory, but I don't think they need calling out. Their purpose is described in the header of each

# Row(product_name='Headphones', num_sold=2, total_revenue=119.98)
# Row(product_name='Monitor', num_sold=1, total_revenue=249.99)
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
from typing import Callable
from typing import Dict
from typing import List
from typing import Optional
from typing import Union
from unittest import mock

Expand All @@ -49,9 +48,20 @@ def _check_inner(actual: PCollection[str]):
return _check_inner


def products_csv():
return '\n'.join([
'transaction_id,product_name,category,price',
'T0012,Headphones,Electronics,59.99',
'T5034,Leather Jacket,Apparel,109.99',
'T0024,Aluminum Mug,Kitchen,29.99',
'T0104,Headphones,Electronics,59.99',
'T0302,Monitor,Electronics,249.99'
])


def create_test_method(
pipeline_spec_file: str,
custom_preprocessor: Optional[Callable[..., Union[Dict, List]]] = None):
custom_preprocessors: List[Callable[..., Union[Dict, List]]]):
@mock.patch('apache_beam.Pipeline', TestPipeline)
def test_yaml_example(self):
with open(pipeline_spec_file, encoding="utf-8") as f:
Expand All @@ -68,20 +78,22 @@ def test_yaml_example(self):
''.join(lines), Loader=yaml_transform.SafeLineLoader)

with TestEnvironment() as env:
if custom_preprocessor:
pipeline_spec = custom_preprocessor(pipeline_spec, expected, env)
for fn in custom_preprocessors:
pipeline_spec = fn(pipeline_spec, expected, env)
with beam.Pipeline(options=PipelineOptions(
pickle_library='cloudpickle',
**yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get(
'options', {})))) as p:
actual = yaml_transform.expand_pipeline(p, pipeline_spec)
if not actual:
actual = p.transforms_stack[0].parts[-1].outputs[None]
check_output(expected)(actual)

return test_yaml_example


class YamlExamplesTestSuite:
_test_preprocessor: Dict[str, Callable[..., Union[Dict, List]]] = {}
_test_preprocessor: Dict[str, List[Callable[..., Union[Dict, List]]]] = {}

def __init__(self, name: str, path: str):
self._test_suite = self.create_test_suite(name, path)
Expand All @@ -96,25 +108,31 @@ def parse_test_methods(cls, path: str):
files = [path]
for file in files:
test_name = f'test_{file.split(os.sep)[-1].replace(".", "_")}'
custom_preprocessor = cls._test_preprocessor.get(test_name, None)
yield test_name, create_test_method(file, custom_preprocessor)
custom_preprocessors = cls._test_preprocessor.get(test_name, [])
yield test_name, create_test_method(file, custom_preprocessors)

@classmethod
def create_test_suite(cls, name: str, path: str):
return type(name, (unittest.TestCase, ), dict(cls.parse_test_methods(path)))

@classmethod
def register_test_preprocessor(cls, test_name: str):
def register_test_preprocessor(cls, test_names: Union[str, List]):
if isinstance(test_names, str):
test_names = [test_names]

def apply(preprocessor):
cls._test_preprocessor[test_name] = preprocessor
for test_name in test_names:
if test_name not in cls._test_preprocessor:
cls._test_preprocessor[test_name] = []
cls._test_preprocessor[test_name].append(preprocessor)
return preprocessor

return apply


@YamlExamplesTestSuite.register_test_preprocessor('test_wordcount_minimal_yaml')
def _wordcount_test_preprocessor(
test_spec: str, expected: List[str], env: TestEnvironment):
test_spec: dict, expected: List[str], env: TestEnvironment):
all_words = []
for element in expected:
word = element.split('=')[1].split(',')[0].replace("'", '')
Expand All @@ -137,17 +155,55 @@ def _wordcount_test_preprocessor(
env.input_file('kinglear.txt', '\n'.join(lines)))


@YamlExamplesTestSuite.register_test_preprocessor(
['test_simple_filter_yaml', 'test_simple_filter_and_combine_yaml'])
def _file_io_write_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):

if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '').startswith('WriteTo'):
transform['type'] = 'LogForTesting'
transform['config'] = {
k: v
for k,
v in transform.get('config', {}).items() if k.startswith('__')
}

return test_spec


@YamlExamplesTestSuite.register_test_preprocessor(
['test_simple_filter_yaml', 'test_simple_filter_and_combine_yaml'])
def _file_io_read_test_preprocessor(
test_spec: dict, expected: List[str], env: TestEnvironment):

if pipeline := test_spec.get('pipeline', None):
for transform in pipeline.get('transforms', []):
if transform.get('type', '').startswith('ReadFrom'):
file_name = transform['config']['path'].split('/')[-1]
return replace_recursive(
test_spec,
transform['type'],
'path',
env.input_file(file_name, INPUT_FILES[file_name]))

return test_spec


INPUT_FILES = {'products.csv': products_csv()}

YAML_DOCS_DIR = os.path.join(os.path.dirname(__file__))
ExamplesTest = YamlExamplesTestSuite(
'ExamplesTest', os.path.join(YAML_DOCS_DIR, '*.yaml')).run()
'ExamplesTest', os.path.join(YAML_DOCS_DIR, '../*.yaml')).run()

ElementWiseTest = YamlExamplesTestSuite(
'ElementwiseExamplesTest',
os.path.join(YAML_DOCS_DIR, 'transforms/elementwise/*.yaml')).run()
os.path.join(YAML_DOCS_DIR, '../transforms/elementwise/*.yaml')).run()

AggregationTest = YamlExamplesTestSuite(
'AggregationExamplesTest',
os.path.join(YAML_DOCS_DIR, 'transforms/aggregation/*.yaml')).run()
os.path.join(YAML_DOCS_DIR, '../transforms/aggregation/*.yaml')).run()

if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
# limitations under the License.
#

# This examples reads from a public file stores on Google Cloud. This
# This examples reads from a public file stored on Google Cloud. This
# requires authenticating with Google Cloud, or setting the file in
#`ReadFromText` to a local file.
#
Expand Down
4 changes: 2 additions & 2 deletions sdks/python/apache_beam/yaml/yaml_transform.py
Original file line number Diff line number Diff line change
Expand Up @@ -920,7 +920,7 @@ def ensure_transforms_have_providers(spec):
f'for type {spec["type"]} for {identify_object(spec)}')
return spec

def preprocess_langauges(spec):
def preprocess_languages(spec):
if spec['type'] in ('AssignTimestamps',
'Combine',
'Filter',
Expand All @@ -942,7 +942,7 @@ def preprocess_langauges(spec):
ensure_transforms_have_types,
normalize_mapping,
normalize_combine,
preprocess_langauges,
preprocess_languages,
ensure_transforms_have_providers,
preprocess_source_sink,
preprocess_chain,
Expand Down
Loading