From 0ad521c8ed00839a522725485e1288062e2984b9 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 15 Dec 2023 09:44:20 -0500 Subject: [PATCH 01/45] enrichment v1 --- .../apache_beam/io/requestresponseio.py | 2 +- .../apache_beam/transforms/enrichment.py | 87 +++++++++++++++ .../transforms/enrichment_it_test.py | 103 ++++++++++++++++++ 3 files changed, 191 insertions(+), 1 deletion(-) create mode 100644 sdks/python/apache_beam/transforms/enrichment.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_it_test.py diff --git a/sdks/python/apache_beam/io/requestresponseio.py b/sdks/python/apache_beam/io/requestresponseio.py index 0ec586e640184..250c27c2a2f34 100644 --- a/sdks/python/apache_beam/io/requestresponseio.py +++ b/sdks/python/apache_beam/io/requestresponseio.py @@ -206,7 +206,7 @@ def process(self, request, *args, **kwargs): with concurrent.futures.ThreadPoolExecutor() as executor: future = executor.submit(self._caller, request) try: - yield future.result(timeout=self._timeout) + return future.result(timeout=self._timeout) except concurrent.futures.TimeoutError: raise UserCodeTimeoutException( f'Timeout {self._timeout} exceeded ' diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py new file mode 100644 index 0000000000000..041a0eeb30590 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -0,0 +1,87 @@ +# +# 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 "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 "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. +# + +from typing import TypeVar, Callable, Optional + +import apache_beam as beam +from apache_beam.io.requestresponseio import Caller +from apache_beam.io.requestresponseio import CacheReader +from apache_beam.io.requestresponseio import CacheWriter +from apache_beam.io.requestresponseio import DEFAULT_TIMEOUT_SECS +from apache_beam.io.requestresponseio import PreCallThrottler +from apache_beam.io.requestresponseio import Repeater +from apache_beam.io.requestresponseio import RequestResponseIO +from apache_beam.io.requestresponseio import ShouldBackOff + +InputT = TypeVar('InputT') +OutputT = TypeVar('OutputT') + + +def cross_join(element): + right_dict = element[1].as_dict() + left_dict = element[0].as_dict() + for k, v in right_dict.items(): + left_dict[k] = v + return beam.Row(**left_dict) + + +class EnrichmentSourceHandler(Caller): + pass + + +class HTTPSourceHandler(EnrichmentSourceHandler): + def __init__(self, url): + self._url = url + + def __call__(self, *args, **kwargs): + pass + + +class Enrichment(beam.PTransform[beam.PCollection[InputT], + beam.PCollection[OutputT]]): + def __init__( + self, + source_handler: EnrichmentSourceHandler, + join_fn: Callable = cross_join, + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + should_backoff: Optional[ShouldBackOff] = None, + repeater: Optional[Repeater] = None, + cache_reader: Optional[CacheReader] = None, + cache_writer: Optional[CacheWriter] = None, + throttler: Optional[PreCallThrottler] = None): + self._source_handler = source_handler + self._join_fn = join_fn + self._timeout = timeout + self._should_backoff = should_backoff + self._repeater = repeater + self._cache_reader = cache_reader + self._cache_writer = cache_writer + self._throttler = throttler + self.output_type = None + + def expand(self, input_row: InputT) -> OutputT: + fetched_data = input_row | RequestResponseIO( + caller=self._source_handler, + timeout=self._timeout, + should_backoff=self._should_backoff, + repeater=self._repeater, + cache_reader=self._cache_reader, + cache_writer=self._cache_writer, + throttler=self._throttler).with_output_types(dict) + + # RequestResponseIO returns a tuple of (request,response) + return fetched_data | beam.ParDo(self._join_fn) diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py new file mode 100644 index 0000000000000..4df9018e89d91 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -0,0 +1,103 @@ +# +# 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 "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 "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. +# + +import unittest +from typing import Union, Tuple + +import urllib3 + +import apache_beam as beam +from apache_beam.io.requestresponseio import UserCodeExecutionException +from apache_beam.io.requestresponseio import UserCodeQuotaException +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.transforms.enrichment import Enrichment +from apache_beam.transforms.enrichment import EnrichmentSourceHandler +from apache_beam.io.requestresponseio_it_test import _PAYLOAD +from apache_beam.io.requestresponseio_it_test import EchoITOptions + + +class SampleHTTPEnrichment(EnrichmentSourceHandler): + """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s + HTTP handler. + """ + def __init__(self, url: str): + self.url = url + '/v1/echo' # append path to the mock API. + + def __call__(self, request: beam.Row, *args, + **kwargs) -> Tuple[beam.Row, beam.Row]: + """Overrides ``Caller``'s call method invoking the + ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning + either a successful ``EchoResponse`` or throwing either a + ``UserCodeExecutionException``, ``UserCodeTimeoutException``, + or a ``UserCodeQuotaException``. + """ + try: + resp = urllib3.request( + "POST", + self.url, + json={ + "id": request.id, "payload": str(request.payload, 'utf-8') + }, + retries=False) + + if resp.status < 300: + resp_body = resp.json() + resp_id = resp_body['id'] + payload = resp_body['payload'] + yield ( + beam.Row(id=request.id, payload=request.payload), + beam.Row(id=resp_id, resp_payload=bytes(payload, 'utf-8'))) + + if resp.status == 429: # Too Many Requests + raise UserCodeQuotaException(resp.reason) + elif resp.status != 200: + raise UserCodeExecutionException(resp.status, resp.reason, request) + + except urllib3.exceptions.HTTPError as e: + raise UserCodeExecutionException(e) + + +class TestEnrichment(unittest.TestCase): + options: Union[EchoITOptions, None] = None + client: Union[SampleHTTPEnrichment, None] = None + + @classmethod + def setUpClass(cls) -> None: + cls.options = EchoITOptions() + http_endpoint_address = 'http://10.138.0.32:8080' + cls.client = SampleHTTPEnrichment(http_endpoint_address) + + @classmethod + def _get_client_and_options( + cls) -> Tuple[SampleHTTPEnrichment, EchoITOptions]: + assert cls.options is not None + assert cls.client is not None + return cls.client, cls.options + + def test_http_enrichment(self): + client, options = TestEnrichment._get_client_and_options() + req = beam.Row(id=options.never_exceed_quota_id, payload=_PAYLOAD) + with TestPipeline(is_integration_test=True) as test_pipeline: + output = ( + test_pipeline + | 'Create PCollection' >> beam.Create([req]) + | 'Enrichment Transform' >> Enrichment(client)) + self.assertIsNotNone(output) + + +if __name__ == '__main__': + unittest.main() From e83bad7ece3e9c9db021fff88c22b217be8d1284 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 15 Dec 2023 11:28:23 -0500 Subject: [PATCH 02/45] add documentation --- .../apache_beam/transforms/enrichment.py | 45 +++++++++++++++---- 1 file changed, 37 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 041a0eeb30590..354769ddc1607 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -15,7 +15,10 @@ # limitations under the License. # -from typing import TypeVar, Callable, Optional +from typing import Callable +from typing import Generic +from typing import Optional +from typing import TypeVar import apache_beam as beam from apache_beam.io.requestresponseio import Caller @@ -32,6 +35,17 @@ def cross_join(element): + """cross_join performs a cross join between two beam.Row objects. + + Joins the columns of the right `beam.Row` onto the left `beam.Row`. + + Args: + element (Tuple): A tuple containing two beam.Row objects - + request and response. + + Returns: + beam.Row: `beam.Row` containing the merged columns. + """ right_dict = element[1].as_dict() left_dict = element[0].as_dict() for k, v in right_dict.items(): @@ -40,19 +54,34 @@ def cross_join(element): class EnrichmentSourceHandler(Caller): + """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller`""" pass -class HTTPSourceHandler(EnrichmentSourceHandler): - def __init__(self, url): - self._url = url +class Enrichment(beam.PTransform[beam.PCollection[InputT], + beam.PCollection[OutputT]], + Generic[InputT, OutputT]): + """A :class:`Enrichment` transform to enrich elements in a PCollection. - def __call__(self, *args, **kwargs): - pass + Uses the :class:`apache_beam.transforms.EnrichmentSourceHandler` to enrich + elements by joining the metadata from external source. + Processes an input :class:`~apache_beam.pvalue.PCollection` of `beam.Row` by + applying a :class:`apache_beam.transforms.EnrichmentSourceHandler` to each + element and returning the enriched :class:`~apache_beam.pvalue.PCollection`. -class Enrichment(beam.PTransform[beam.PCollection[InputT], - beam.PCollection[OutputT]]): + Args: + source_handler: Handles source lookup and metadata retrieval. + Implements the :class:`apache_beam.transforms.EnrichmentSourceHandler` + join_fn: A lambda function to join original element with lookup metadata. + Defaults to `CROSS_JOIN`. + timeout: (Optional) timeout for source requests. Defaults to 30 seconds. + should_backoff: (Optional) backoff strategy function. + repeater: (Optional) retry Repeater. + cache_reader: (Optional) CacheReader for reading cache. + cache_writer: (Optional) CacheWriter for writing cache. + throttler: (Optional) Throttler mechanism to throttle source requests. + """ def __init__( self, source_handler: EnrichmentSourceHandler, From 516296006a72d4881b943019e0c84174e29f6b61 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 15 Dec 2023 11:41:10 -0500 Subject: [PATCH 03/45] add doc comment --- sdks/python/apache_beam/transforms/enrichment.py | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 354769ddc1607..fbe78ad11004b 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -54,7 +54,12 @@ def cross_join(element): class EnrichmentSourceHandler(Caller): - """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller`""" + """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller` + + Ensure that the implementation of ``__call__`` method returns a tuple + of `beam.Row` objects. + """ + pass @@ -112,5 +117,5 @@ def expand(self, input_row: InputT) -> OutputT: cache_writer=self._cache_writer, throttler=self._throttler).with_output_types(dict) - # RequestResponseIO returns a tuple of (request,response) - return fetched_data | beam.ParDo(self._join_fn) + # EnrichmentSourceHandler returns a tuple of (request,response). + return fetched_data | beam.Map(self._join_fn) From a45392dd0e543266e74c30587f97c8504832c60a Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 09:12:50 -0500 Subject: [PATCH 04/45] rerun --- sdks/python/apache_beam/transforms/enrichment.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index fbe78ad11004b..05e56ac72ab80 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -54,7 +54,7 @@ def cross_join(element): class EnrichmentSourceHandler(Caller): - """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller` + """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller`. Ensure that the implementation of ``__call__`` method returns a tuple of `beam.Row` objects. From 9fdbeb3b794326fe70cbc1f3d92ecc4addf1547d Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 10:36:15 -0500 Subject: [PATCH 05/45] update docs, lint --- sdks/python/apache_beam/io/requestresponseio.py | 3 ++- sdks/python/apache_beam/transforms/enrichment.py | 15 +++++++++++---- .../apache_beam/transforms/enrichment_it_test.py | 4 ++-- 3 files changed, 15 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponseio.py b/sdks/python/apache_beam/io/requestresponseio.py index 250c27c2a2f34..1c7208ee4c527 100644 --- a/sdks/python/apache_beam/io/requestresponseio.py +++ b/sdks/python/apache_beam/io/requestresponseio.py @@ -99,7 +99,8 @@ class PreCallThrottler(abc.ABC): class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], - beam.PCollection[ResponseT]]): + beam.PCollection[ResponseT]], + Generic[RequestT, ResponseT]): """A :class:`RequestResponseIO` transform to read and write to APIs. Processes an input :class:`~apache_beam.pvalue.PCollection` of requests diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 05e56ac72ab80..560496a1d9e35 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -30,12 +30,17 @@ from apache_beam.io.requestresponseio import RequestResponseIO from apache_beam.io.requestresponseio import ShouldBackOff +__all__ = [ + "EnrichmentSourceHandler", + "Enrichment", +] + InputT = TypeVar('InputT') OutputT = TypeVar('OutputT') def cross_join(element): - """cross_join performs a cross join between two beam.Row objects. + """cross_join performs a cross join between two `beam.Row` objects. Joins the columns of the right `beam.Row` onto the left `beam.Row`. @@ -45,7 +50,7 @@ def cross_join(element): Returns: beam.Row: `beam.Row` containing the merged columns. - """ + """ right_dict = element[1].as_dict() left_dict = element[0].as_dict() for k, v in right_dict.items(): @@ -66,7 +71,8 @@ class EnrichmentSourceHandler(Caller): class Enrichment(beam.PTransform[beam.PCollection[InputT], beam.PCollection[OutputT]], Generic[InputT, OutputT]): - """A :class:`Enrichment` transform to enrich elements in a PCollection. + """A :class:`apache_beam.transforms.Enrichment` transform to enrich elements + in a PCollection. Uses the :class:`apache_beam.transforms.EnrichmentSourceHandler` to enrich elements by joining the metadata from external source. @@ -107,7 +113,8 @@ def __init__( self._throttler = throttler self.output_type = None - def expand(self, input_row: InputT) -> OutputT: + def expand(self, + input_row: beam.PCollection[InputT]) -> beam.PCollection[OutputT]: fetched_data = input_row | RequestResponseIO( caller=self._source_handler, timeout=self._timeout, diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 4df9018e89d91..fb40ed6c526f6 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -30,7 +30,7 @@ from apache_beam.io.requestresponseio_it_test import EchoITOptions -class SampleHTTPEnrichment(EnrichmentSourceHandler): +class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. """ @@ -58,7 +58,7 @@ def __call__(self, request: beam.Row, *args, resp_body = resp.json() resp_id = resp_body['id'] payload = resp_body['payload'] - yield ( + return ( beam.Row(id=request.id, payload=request.payload), beam.Row(id=resp_id, resp_payload=bytes(payload, 'utf-8'))) From c54114805bc74a6dd8116cf5532dd3acd61e30c7 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 10:56:30 -0500 Subject: [PATCH 06/45] update docs, lint --- sdks/python/apache_beam/io/requestresponseio.py | 4 +++- sdks/python/apache_beam/transforms/enrichment.py | 4 ++-- sdks/python/apache_beam/transforms/enrichment_it_test.py | 7 ++++--- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponseio.py b/sdks/python/apache_beam/io/requestresponseio.py index 1c7208ee4c527..a4e9c6868694e 100644 --- a/sdks/python/apache_beam/io/requestresponseio.py +++ b/sdks/python/apache_beam/io/requestresponseio.py @@ -50,7 +50,9 @@ class UserCodeTimeoutException(UserCodeExecutionException): """Extends ``UserCodeExecutionException`` to signal a user code timeout.""" -class Caller(contextlib.AbstractContextManager, abc.ABC): +class Caller(contextlib.AbstractContextManager, + abc.ABC, + Generic[RequestT, ResponseT]): """Interface for user custom code intended for API calls. For setup and teardown of clients when applicable, implement the ``__enter__`` and ``__exit__`` methods respectively.""" diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 560496a1d9e35..8e8060529d879 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -21,10 +21,10 @@ from typing import TypeVar import apache_beam as beam -from apache_beam.io.requestresponseio import Caller +from apache_beam.io.requestresponseio import DEFAULT_TIMEOUT_SECS from apache_beam.io.requestresponseio import CacheReader from apache_beam.io.requestresponseio import CacheWriter -from apache_beam.io.requestresponseio import DEFAULT_TIMEOUT_SECS +from apache_beam.io.requestresponseio import Caller from apache_beam.io.requestresponseio import PreCallThrottler from apache_beam.io.requestresponseio import Repeater from apache_beam.io.requestresponseio import RequestResponseIO diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index fb40ed6c526f6..c5afe66af6f63 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -16,18 +16,19 @@ # import unittest -from typing import Union, Tuple +from typing import Tuple +from typing import Union import urllib3 import apache_beam as beam from apache_beam.io.requestresponseio import UserCodeExecutionException from apache_beam.io.requestresponseio import UserCodeQuotaException +from apache_beam.io.requestresponseio_it_test import _PAYLOAD +from apache_beam.io.requestresponseio_it_test import EchoITOptions from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment import EnrichmentSourceHandler -from apache_beam.io.requestresponseio_it_test import _PAYLOAD -from apache_beam.io.requestresponseio_it_test import EchoITOptions class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): From 5c9be0e3de93649e851648cc68b4ae7aa82f4ce8 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 11:13:14 -0500 Subject: [PATCH 07/45] add generic type --- sdks/python/apache_beam/transforms/enrichment_it_test.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index c5afe66af6f63..7708ba69ccadd 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -38,8 +38,7 @@ class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): def __init__(self, url: str): self.url = url + '/v1/echo' # append path to the mock API. - def __call__(self, request: beam.Row, *args, - **kwargs) -> Tuple[beam.Row, beam.Row]: + def __call__(self, request: beam.Row, *args, **kwargs): """Overrides ``Caller``'s call method invoking the ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning either a successful ``EchoResponse`` or throwing either a @@ -59,7 +58,7 @@ def __call__(self, request: beam.Row, *args, resp_body = resp.json() resp_id = resp_body['id'] payload = resp_body['payload'] - return ( + yield ( beam.Row(id=request.id, payload=request.payload), beam.Row(id=resp_id, resp_payload=bytes(payload, 'utf-8'))) From 9df679c4c72ce7791fe2bc556835be6007d00da5 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 11:31:10 -0500 Subject: [PATCH 08/45] add generic type --- sdks/python/apache_beam/transforms/enrichment.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 8e8060529d879..84937d6f2f1e6 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -58,7 +58,7 @@ def cross_join(element): return beam.Row(**left_dict) -class EnrichmentSourceHandler(Caller): +class EnrichmentSourceHandler(Caller[InputT, OutputT]): """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller`. Ensure that the implementation of ``__call__`` method returns a tuple From 883ff0de51b51f9b060f0c36bbf1d8a2f17690bd Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 11:36:43 -0500 Subject: [PATCH 09/45] adjust doc path --- sdks/python/apache_beam/transforms/enrichment.py | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 84937d6f2f1e6..0d4e970f9a014 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -71,19 +71,21 @@ class EnrichmentSourceHandler(Caller[InputT, OutputT]): class Enrichment(beam.PTransform[beam.PCollection[InputT], beam.PCollection[OutputT]], Generic[InputT, OutputT]): - """A :class:`apache_beam.transforms.Enrichment` transform to enrich elements - in a PCollection. + """A :class:`apache_beam.transforms.enrichment.Enrichment` transform to + enrich elements in a PCollection. - Uses the :class:`apache_beam.transforms.EnrichmentSourceHandler` to enrich - elements by joining the metadata from external source. + Uses the :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` + to enrich elements by joining the metadata from external source. Processes an input :class:`~apache_beam.pvalue.PCollection` of `beam.Row` by - applying a :class:`apache_beam.transforms.EnrichmentSourceHandler` to each - element and returning the enriched :class:`~apache_beam.pvalue.PCollection`. + applying a :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` + to each element and returning the enriched + :class:`~apache_beam.pvalue.PCollection`. Args: source_handler: Handles source lookup and metadata retrieval. - Implements the :class:`apache_beam.transforms.EnrichmentSourceHandler` + Implements the + :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` join_fn: A lambda function to join original element with lookup metadata. Defaults to `CROSS_JOIN`. timeout: (Optional) timeout for source requests. Defaults to 30 seconds. From 818bb8a9eba786bf7aed35de605cbafdb653aab0 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 12:10:09 -0500 Subject: [PATCH 10/45] create test row --- sdks/python/apache_beam/transforms/enrichment_it_test.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 7708ba69ccadd..e988b6a27225d 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -16,9 +16,11 @@ # import unittest +from typing import NamedTuple from typing import Tuple from typing import Union +import pytest import urllib3 import apache_beam as beam @@ -31,6 +33,12 @@ from apache_beam.transforms.enrichment import EnrichmentSourceHandler +class Request(NamedTuple): + """Simple request type to store id and payload for requests.""" + id: str # mock API quota id + payload: bytes # byte payload + + class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. @@ -71,6 +79,7 @@ def __call__(self, request: beam.Row, *args, **kwargs): raise UserCodeExecutionException(e) +@pytest.mark.it_postcommit class TestEnrichment(unittest.TestCase): options: Union[EchoITOptions, None] = None client: Union[SampleHTTPEnrichment, None] = None From e1feeb80675a7abd9b2680d10a3065eec3a95c0e Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 12:42:12 -0500 Subject: [PATCH 11/45] use request type --- sdks/python/apache_beam/transforms/enrichment_it_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index e988b6a27225d..a4a8353510584 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -46,7 +46,7 @@ class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): def __init__(self, url: str): self.url = url + '/v1/echo' # append path to the mock API. - def __call__(self, request: beam.Row, *args, **kwargs): + def __call__(self, request: Request, *args, **kwargs): """Overrides ``Caller``'s call method invoking the ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning either a successful ``EchoResponse`` or throwing either a @@ -99,7 +99,7 @@ def _get_client_and_options( def test_http_enrichment(self): client, options = TestEnrichment._get_client_and_options() - req = beam.Row(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) with TestPipeline(is_integration_test=True) as test_pipeline: output = ( test_pipeline From 40275e9265085b4074a2cd65f07b16f9f6ec290e Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 18 Dec 2023 12:59:09 -0500 Subject: [PATCH 12/45] use request type --- sdks/python/apache_beam/transforms/enrichment_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index a4a8353510584..e702aff334cc4 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -39,7 +39,7 @@ class Request(NamedTuple): payload: bytes # byte payload -class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): +class SampleHTTPEnrichment(EnrichmentSourceHandler[Request, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. """ From be67a883a348d9e8f8055f92291591c5e44c1280 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 20 Dec 2023 14:46:09 -0500 Subject: [PATCH 13/45] change module name --- ...equestresponseio.py => requestresponse.py} | 20 +++++++++---------- ..._it_test.py => requestresponse_it_test.py} | 8 ++++---- ...onseio_test.py => requestresponse_test.py} | 8 ++++---- .../apache_beam/transforms/enrichment.py | 18 ++++++++--------- .../transforms/enrichment_it_test.py | 8 ++++---- 5 files changed, 31 insertions(+), 31 deletions(-) rename sdks/python/apache_beam/io/{requestresponseio.py => requestresponse.py} (90%) rename sdks/python/apache_beam/io/{requestresponseio_it_test.py => requestresponse_it_test.py} (96%) rename sdks/python/apache_beam/io/{requestresponseio_test.py => requestresponse_test.py} (91%) diff --git a/sdks/python/apache_beam/io/requestresponseio.py b/sdks/python/apache_beam/io/requestresponse.py similarity index 90% rename from sdks/python/apache_beam/io/requestresponseio.py rename to sdks/python/apache_beam/io/requestresponse.py index a4e9c6868694e..075b6d86119b4 100644 --- a/sdks/python/apache_beam/io/requestresponseio.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -123,18 +123,18 @@ def __init__( Instantiates a RequestResponseIO transform. Args: - caller (~apache_beam.io.requestresponseio.Caller): an implementation of + caller (~apache_beam.io.requestresponse.Caller): an implementation of `Caller` object that makes call to the API. timeout (float): timeout value in seconds to wait for response from API. - should_backoff (~apache_beam.io.requestresponseio.ShouldBackOff): + should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponseio.Repeater): (Optional) + repeater (~apache_beam.io.requestresponse.Repeater): (Optional) provides methods to repeat requests to API. - cache_reader (~apache_beam.io.requestresponseio.CacheReader): (Optional) + cache_reader (~apache_beam.io.requestresponse.CacheReader): (Optional) provides methods to read external cache. - cache_writer (~apache_beam.io.requestresponseio.CacheWriter): (Optional) + cache_writer (~apache_beam.io.requestresponse.CacheWriter): (Optional) provides methods to write to external cache. - throttler (~apache_beam.io.requestresponseio.PreCallThrottler): + throttler (~apache_beam.io.requestresponse.PreCallThrottler): (Optional) provides methods to pre-throttle a request. """ self._caller = caller @@ -165,7 +165,7 @@ class _Call(beam.PTransform[beam.PCollection[RequestT], regulate the duration of each call, defaults to 30 seconds. Args: - caller (:class:`apache_beam.io.requestresponseio.Caller`): a callable + caller (:class:`apache_beam.io.requestresponse.Caller`): a callable object that invokes API call. timeout (float): timeout value in seconds to wait for response from API. """ @@ -178,12 +178,12 @@ def __init__( ): """Initialize the _Call transform. Args: - caller (:class:`apache_beam.io.requestresponseio.Caller`): a callable + caller (:class:`apache_beam.io.requestresponse.Caller`): a callable object that invokes API call. timeout (float): timeout value in seconds to wait for response from API. - should_backoff (~apache_beam.io.requestresponseio.ShouldBackOff): + should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponseio.Repeater): (Optional) provides + repeater (~apache_beam.io.requestresponse.Repeater): (Optional) provides methods to repeat requests to API. """ self._caller = caller diff --git a/sdks/python/apache_beam/io/requestresponseio_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py similarity index 96% rename from sdks/python/apache_beam/io/requestresponseio_it_test.py rename to sdks/python/apache_beam/io/requestresponse_it_test.py index aae6b4e6ef2c7..2e94aa7cdec9c 100644 --- a/sdks/python/apache_beam/io/requestresponseio_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -24,10 +24,10 @@ import urllib3 import apache_beam as beam -from apache_beam.io.requestresponseio import Caller -from apache_beam.io.requestresponseio import RequestResponseIO -from apache_beam.io.requestresponseio import UserCodeExecutionException -from apache_beam.io.requestresponseio import UserCodeQuotaException +from apache_beam.io.requestresponse import Caller +from apache_beam.io.requestresponse import RequestResponseIO +from apache_beam.io.requestresponse import UserCodeExecutionException +from apache_beam.io.requestresponse import UserCodeQuotaException from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing.test_pipeline import TestPipeline diff --git a/sdks/python/apache_beam/io/requestresponseio_test.py b/sdks/python/apache_beam/io/requestresponse_test.py similarity index 91% rename from sdks/python/apache_beam/io/requestresponseio_test.py rename to sdks/python/apache_beam/io/requestresponse_test.py index 2828a3578871d..780570e6b4997 100644 --- a/sdks/python/apache_beam/io/requestresponseio_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -18,10 +18,10 @@ import unittest import apache_beam as beam -from apache_beam.io.requestresponseio import Caller -from apache_beam.io.requestresponseio import RequestResponseIO -from apache_beam.io.requestresponseio import UserCodeExecutionException -from apache_beam.io.requestresponseio import UserCodeTimeoutException +from apache_beam.io.requestresponse import Caller +from apache_beam.io.requestresponse import RequestResponseIO +from apache_beam.io.requestresponse import UserCodeExecutionException +from apache_beam.io.requestresponse import UserCodeTimeoutException from apache_beam.testing.test_pipeline import TestPipeline diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 0d4e970f9a014..d712783c0471f 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -21,14 +21,14 @@ from typing import TypeVar import apache_beam as beam -from apache_beam.io.requestresponseio import DEFAULT_TIMEOUT_SECS -from apache_beam.io.requestresponseio import CacheReader -from apache_beam.io.requestresponseio import CacheWriter -from apache_beam.io.requestresponseio import Caller -from apache_beam.io.requestresponseio import PreCallThrottler -from apache_beam.io.requestresponseio import Repeater -from apache_beam.io.requestresponseio import RequestResponseIO -from apache_beam.io.requestresponseio import ShouldBackOff +from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS +from apache_beam.io.requestresponse import CacheReader +from apache_beam.io.requestresponse import CacheWriter +from apache_beam.io.requestresponse import Caller +from apache_beam.io.requestresponse import PreCallThrottler +from apache_beam.io.requestresponse import Repeater +from apache_beam.io.requestresponse import RequestResponseIO +from apache_beam.io.requestresponse import ShouldBackOff __all__ = [ "EnrichmentSourceHandler", @@ -59,7 +59,7 @@ def cross_join(element): class EnrichmentSourceHandler(Caller[InputT, OutputT]): - """Wrapper class for :class:`apache_beam.io.requestresponseio.Caller`. + """Wrapper class for :class:`apache_beam.io.requestresponse.Caller`. Ensure that the implementation of ``__call__`` method returns a tuple of `beam.Row` objects. diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index e702aff334cc4..d4655ee1a8e31 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -24,10 +24,10 @@ import urllib3 import apache_beam as beam -from apache_beam.io.requestresponseio import UserCodeExecutionException -from apache_beam.io.requestresponseio import UserCodeQuotaException -from apache_beam.io.requestresponseio_it_test import _PAYLOAD -from apache_beam.io.requestresponseio_it_test import EchoITOptions +from apache_beam.io.requestresponse import UserCodeExecutionException +from apache_beam.io.requestresponse import UserCodeQuotaException +from apache_beam.io.requestresponse_it_test import _PAYLOAD +from apache_beam.io.requestresponse_it_test import EchoITOptions from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment import EnrichmentSourceHandler From 27ed2500fca5b0669ad19768a9f9709fc78ffd84 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 2 Jan 2024 17:56:56 -0500 Subject: [PATCH 14/45] more tests --- .../apache_beam/transforms/enrichment.py | 2 +- .../transforms/enrichment_it_test.py | 62 ++++++++++++++++--- .../apache_beam/transforms/enrichment_test.py | 35 +++++++++++ 3 files changed, 88 insertions(+), 11 deletions(-) create mode 100644 sdks/python/apache_beam/transforms/enrichment_test.py diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index d712783c0471f..b5ba5768c6111 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -33,6 +33,7 @@ __all__ = [ "EnrichmentSourceHandler", "Enrichment", + "cross_join", ] InputT = TypeVar('InputT') @@ -64,7 +65,6 @@ class EnrichmentSourceHandler(Caller[InputT, OutputT]): Ensure that the implementation of ``__call__`` method returns a tuple of `beam.Row` objects. """ - pass diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index d4655ee1a8e31..381db66a96b00 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # - +import time import unittest from typing import NamedTuple from typing import Tuple @@ -29,27 +29,35 @@ from apache_beam.io.requestresponse_it_test import _PAYLOAD from apache_beam.io.requestresponse_it_test import EchoITOptions from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import BeamAssertException from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment import EnrichmentSourceHandler -class Request(NamedTuple): +class _Request(NamedTuple): """Simple request type to store id and payload for requests.""" id: str # mock API quota id payload: bytes # byte payload -class SampleHTTPEnrichment(EnrichmentSourceHandler[Request, beam.Row]): +def _custom_join(element): + """custom_join returns the id and resp_payload along with a timestamp""" + right_dict = element[1].as_dict() + right_dict['timestamp'] = time.time() + return beam.Row(**right_dict) + + +class SampleHTTPEnrichment(EnrichmentSourceHandler[_Request, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. """ def __init__(self, url: str): self.url = url + '/v1/echo' # append path to the mock API. - def __call__(self, request: Request, *args, **kwargs): + def __call__(self, request: _Request, *args, **kwargs): """Overrides ``Caller``'s call method invoking the - ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning - either a successful ``EchoResponse`` or throwing either a + ``EchoServiceGrpc``'s HTTP handler with an ``_Request``, returning + either a successful ``Tuple[beam.Row,beam.Row]`` or throwing either a ``UserCodeExecutionException``, ``UserCodeTimeoutException``, or a ``UserCodeQuotaException``. """ @@ -79,6 +87,24 @@ def __call__(self, request: Request, *args, **kwargs): raise UserCodeExecutionException(e) +class ValidateFields(beam.DoFn): + """ValidateFields validates if a PCollection of `beam.Row` + has certain fields.""" + def __init__(self, fields): + self._fields = fields + + def process(self, element: beam.Row): + element_dict = element.as_dict() + if len(element_dict.keys()) != 3: + raise BeamAssertException( + "Expected three fields in enriched pcollection:" + " id, payload and resp_payload") + + for field in self._fields: + if field not in element_dict or element_dict[field] is None: + raise BeamAssertException(f"Expected a not None field: {field}") + + @pytest.mark.it_postcommit class TestEnrichment(unittest.TestCase): options: Union[EchoITOptions, None] = None @@ -98,14 +124,30 @@ def _get_client_and_options( return cls.client, cls.options def test_http_enrichment(self): + """Tests Enrichment Transform against the Mock-API HTTP endpoint + with the default cross join.""" + client, options = TestEnrichment._get_client_and_options() + req = _Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + fields = ['id', 'payload', 'resp_payload'] + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | 'Create PCollection' >> beam.Create([req]) + | 'Enrichment Transform' >> Enrichment(client) + | 'Assert Fields' >> beam.ParDo(ValidateFields(fields=fields))) + + def test_http_enrichment_custom_join(self): + """Tests Enrichment Transform against the Mock-API HTTP endpoint + with a custom join function.""" client, options = TestEnrichment._get_client_and_options() - req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = _Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + fields = ['id', 'resp_payload', 'timestamp'] with TestPipeline(is_integration_test=True) as test_pipeline: - output = ( + _ = ( test_pipeline | 'Create PCollection' >> beam.Create([req]) - | 'Enrichment Transform' >> Enrichment(client)) - self.assertIsNotNone(output) + | 'Enrichment Transform' >> Enrichment(client, join_fn=_custom_join) + | 'Assert Fields' >> beam.ParDo(ValidateFields(fields=fields))) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/transforms/enrichment_test.py b/sdks/python/apache_beam/transforms/enrichment_test.py new file mode 100644 index 0000000000000..687954612372f --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_test.py @@ -0,0 +1,35 @@ +# +# 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 "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 "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. +# + +import logging +import unittest + +import apache_beam as beam +from apache_beam.transforms.enrichment import cross_join + + +class TestEnrichmentTransform(unittest.TestCase): + def test_cross_join(self): + rows = (beam.Row(id=1, key='city'), beam.Row(id=1, value='durham')) + expected = beam.Row(id=1, key='city', value='durham') + output = cross_join(rows) + self.assertEqual(expected, output) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() From 4af90f579378312c814f74ec5d2c82800112ecb3 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 3 Jan 2024 09:23:43 -0500 Subject: [PATCH 15/45] remove non-functional params --- .../apache_beam/transforms/enrichment.py | 41 ++++--------------- 1 file changed, 9 insertions(+), 32 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index b5ba5768c6111..cdd58dc36ee44 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -18,17 +18,13 @@ from typing import Callable from typing import Generic from typing import Optional +from typing import Tuple from typing import TypeVar import apache_beam as beam from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS -from apache_beam.io.requestresponse import CacheReader -from apache_beam.io.requestresponse import CacheWriter from apache_beam.io.requestresponse import Caller -from apache_beam.io.requestresponse import PreCallThrottler -from apache_beam.io.requestresponse import Repeater from apache_beam.io.requestresponse import RequestResponseIO -from apache_beam.io.requestresponse import ShouldBackOff __all__ = [ "EnrichmentSourceHandler", @@ -39,8 +35,10 @@ InputT = TypeVar('InputT') OutputT = TypeVar('OutputT') +JoinFn = Callable[[Tuple[beam.Row, beam.Row]], beam.Row] -def cross_join(element): + +def cross_join(element: Tuple[beam.Row, beam.Row]) -> beam.Row: """cross_join performs a cross join between two `beam.Row` objects. Joins the columns of the right `beam.Row` onto the left `beam.Row`. @@ -73,7 +71,8 @@ class Enrichment(beam.PTransform[beam.PCollection[InputT], Generic[InputT, OutputT]): """A :class:`apache_beam.transforms.enrichment.Enrichment` transform to enrich elements in a PCollection. - + **NOTE:** This transform and its implementation are under development and + do not provide backward compatibility guarantees. Uses the :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` to enrich elements by joining the metadata from external source. @@ -89,42 +88,20 @@ class Enrichment(beam.PTransform[beam.PCollection[InputT], join_fn: A lambda function to join original element with lookup metadata. Defaults to `CROSS_JOIN`. timeout: (Optional) timeout for source requests. Defaults to 30 seconds. - should_backoff: (Optional) backoff strategy function. - repeater: (Optional) retry Repeater. - cache_reader: (Optional) CacheReader for reading cache. - cache_writer: (Optional) CacheWriter for writing cache. - throttler: (Optional) Throttler mechanism to throttle source requests. """ def __init__( self, source_handler: EnrichmentSourceHandler, - join_fn: Callable = cross_join, - timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, - should_backoff: Optional[ShouldBackOff] = None, - repeater: Optional[Repeater] = None, - cache_reader: Optional[CacheReader] = None, - cache_writer: Optional[CacheWriter] = None, - throttler: Optional[PreCallThrottler] = None): + join_fn: JoinFn = cross_join, + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS): self._source_handler = source_handler self._join_fn = join_fn self._timeout = timeout - self._should_backoff = should_backoff - self._repeater = repeater - self._cache_reader = cache_reader - self._cache_writer = cache_writer - self._throttler = throttler - self.output_type = None def expand(self, input_row: beam.PCollection[InputT]) -> beam.PCollection[OutputT]: fetched_data = input_row | RequestResponseIO( - caller=self._source_handler, - timeout=self._timeout, - should_backoff=self._should_backoff, - repeater=self._repeater, - cache_reader=self._cache_reader, - cache_writer=self._cache_writer, - throttler=self._throttler).with_output_types(dict) + caller=self._source_handler, timeout=self._timeout) # EnrichmentSourceHandler returns a tuple of (request,response). return fetched_data | beam.Map(self._join_fn) From 041fcd02a8bdd4f04134cb2e2ea3c0170803236e Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 3 Jan 2024 10:10:25 -0500 Subject: [PATCH 16/45] lint, doc --- sdks/python/apache_beam/transforms/enrichment.py | 4 ++-- sdks/python/apache_beam/transforms/enrichment_it_test.py | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index cdd58dc36ee44..2093c053a0140 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -44,11 +44,11 @@ def cross_join(element: Tuple[beam.Row, beam.Row]) -> beam.Row: Joins the columns of the right `beam.Row` onto the left `beam.Row`. Args: - element (Tuple): A tuple containing two beam.Row objects - + element (Tuple): A tuple containing two `beam.Row` objects - request and response. Returns: - beam.Row: `beam.Row` containing the merged columns. + `beam.Row` containing the merged columns. """ right_dict = element[1].as_dict() left_dict = element[0].as_dict() diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 381db66a96b00..1cd78146498b3 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -93,11 +93,11 @@ class ValidateFields(beam.DoFn): def __init__(self, fields): self._fields = fields - def process(self, element: beam.Row): + def process(self, element: beam.Row, *args, **kwargs): element_dict = element.as_dict() if len(element_dict.keys()) != 3: raise BeamAssertException( - "Expected three fields in enriched pcollection:" + "Expected three fields in enriched PCollection:" " id, payload and resp_payload") for field in self._fields: From 91f58b57740e20b935a87c452b3b0beb2d8f93fd Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 4 Jan 2024 10:08:25 -0500 Subject: [PATCH 17/45] change types for general use --- sdks/python/apache_beam/io/requestresponse.py | 16 ++++----- .../apache_beam/io/requestresponse_it_test.py | 3 +- .../apache_beam/transforms/enrichment.py | 23 +++++++------ .../transforms/enrichment_it_test.py | 33 ++++++++----------- .../apache_beam/transforms/enrichment_test.py | 2 +- 5 files changed, 36 insertions(+), 41 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 075b6d86119b4..6b3c1ae7615fc 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -26,7 +26,6 @@ from typing import TypeVar import apache_beam as beam -from apache_beam.pvalue import PCollection RequestT = TypeVar('RequestT') ResponseT = TypeVar('ResponseT') @@ -101,8 +100,7 @@ class PreCallThrottler(abc.ABC): class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], - beam.PCollection[ResponseT]], - Generic[RequestT, ResponseT]): + beam.PCollection[ResponseT]]): """A :class:`RequestResponseIO` transform to read and write to APIs. Processes an input :class:`~apache_beam.pvalue.PCollection` of requests @@ -111,7 +109,7 @@ class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], """ def __init__( self, - caller: [Caller], + caller: Caller, timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, repeater: Optional[Repeater] = None, @@ -145,7 +143,9 @@ def __init__( self._cache_writer = cache_writer self._throttler = throttler - def expand(self, requests: PCollection[RequestT]) -> PCollection[ResponseT]: + def expand( + self, + requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: # TODO(riteshghorse): add Cache and Throttle PTransforms. return requests | _Call( caller=self._caller, @@ -197,7 +197,7 @@ def expand( return requests | beam.ParDo(_CallDoFn(self._caller, self._timeout)) -class _CallDoFn(beam.DoFn, Generic[RequestT, ResponseT]): +class _CallDoFn(beam.DoFn): def setup(self): self._caller.__enter__() @@ -205,11 +205,11 @@ def __init__(self, caller: Caller, timeout: float): self._caller = caller self._timeout = timeout - def process(self, request, *args, **kwargs): + def process(self, request: RequestT, *args, **kwargs): with concurrent.futures.ThreadPoolExecutor() as executor: future = executor.submit(self._caller, request) try: - return future.result(timeout=self._timeout) + yield future.result(timeout=self._timeout) except concurrent.futures.TimeoutError: raise UserCodeTimeoutException( f'Timeout {self._timeout} exceeded ' diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index 2e94aa7cdec9c..79160e645afee 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -75,7 +75,7 @@ class EchoResponse: payload: bytes -class EchoHTTPCaller(Caller): +class EchoHTTPCaller(Caller[EchoRequest, EchoResponse]): """Implements ``Caller`` to call the ``EchoServiceGrpc``'s HTTP handler. The purpose of ``EchoHTTPCaller`` is to support integration tests. """ @@ -121,6 +121,7 @@ class EchoHTTPCallerTestIT(unittest.TestCase): def setUpClass(cls) -> None: cls.options = EchoITOptions() http_endpoint_address = cls.options.http_endpoint_address + http_endpoint_address = 'http://localhost:8080' if not http_endpoint_address or http_endpoint_address == '': raise unittest.SkipTest(f'{_HTTP_ENDPOINT_ADDRESS_FLAG} is required.') diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 2093c053a0140..e985442f96cd3 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -16,7 +16,6 @@ # from typing import Callable -from typing import Generic from typing import Optional from typing import Tuple from typing import TypeVar @@ -38,23 +37,24 @@ JoinFn = Callable[[Tuple[beam.Row, beam.Row]], beam.Row] -def cross_join(element: Tuple[beam.Row, beam.Row]) -> beam.Row: - """cross_join performs a cross join between two `beam.Row` objects. +def cross_join(element: Tuple[dict, dict]) -> beam.Row: + """cross_join performs a cross join on two `dict` objects. - Joins the columns of the right `beam.Row` onto the left `beam.Row`. + Joins the columns of the right row onto the left row. Args: - element (Tuple): A tuple containing two `beam.Row` objects - + element (Tuple): A tuple containing two `dict` objects - request and response. Returns: `beam.Row` containing the merged columns. """ - right_dict = element[1].as_dict() - left_dict = element[0].as_dict() - for k, v in right_dict.items(): - left_dict[k] = v - return beam.Row(**left_dict) + left, right = element + for k, v in right.items(): + if k not in left: + # Don't override the values in left. + left[k] = v + return beam.Row(**left) class EnrichmentSourceHandler(Caller[InputT, OutputT]): @@ -67,8 +67,7 @@ class EnrichmentSourceHandler(Caller[InputT, OutputT]): class Enrichment(beam.PTransform[beam.PCollection[InputT], - beam.PCollection[OutputT]], - Generic[InputT, OutputT]): + beam.PCollection[OutputT]]): """A :class:`apache_beam.transforms.enrichment.Enrichment` transform to enrich elements in a PCollection. **NOTE:** This transform and its implementation are under development and diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 1cd78146498b3..e9ce12b4883c2 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -16,7 +16,6 @@ # import time import unittest -from typing import NamedTuple from typing import Tuple from typing import Union @@ -34,30 +33,24 @@ from apache_beam.transforms.enrichment import EnrichmentSourceHandler -class _Request(NamedTuple): - """Simple request type to store id and payload for requests.""" - id: str # mock API quota id - payload: bytes # byte payload - - def _custom_join(element): """custom_join returns the id and resp_payload along with a timestamp""" - right_dict = element[1].as_dict() + right_dict = element[1] right_dict['timestamp'] = time.time() return beam.Row(**right_dict) -class SampleHTTPEnrichment(EnrichmentSourceHandler[_Request, beam.Row]): +class SampleHTTPEnrichment(EnrichmentSourceHandler[dict, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. """ def __init__(self, url: str): self.url = url + '/v1/echo' # append path to the mock API. - def __call__(self, request: _Request, *args, **kwargs): + def __call__(self, request: dict, *args, **kwargs): """Overrides ``Caller``'s call method invoking the - ``EchoServiceGrpc``'s HTTP handler with an ``_Request``, returning - either a successful ``Tuple[beam.Row,beam.Row]`` or throwing either a + ``EchoServiceGrpc``'s HTTP handler with an `dict`, returning + either a successful ``Tuple[dict,dict]`` or throwing either a ``UserCodeExecutionException``, ``UserCodeTimeoutException``, or a ``UserCodeQuotaException``. """ @@ -66,7 +59,7 @@ def __call__(self, request: _Request, *args, **kwargs): "POST", self.url, json={ - "id": request.id, "payload": str(request.payload, 'utf-8') + "id": request['id'], "payload": str(request['payload'], 'utf-8') }, retries=False) @@ -74,9 +67,10 @@ def __call__(self, request: _Request, *args, **kwargs): resp_body = resp.json() resp_id = resp_body['id'] payload = resp_body['payload'] - yield ( - beam.Row(id=request.id, payload=request.payload), - beam.Row(id=resp_id, resp_payload=bytes(payload, 'utf-8'))) + return ( + request, { + 'id': resp_id, 'resp_payload': bytes(payload, 'utf-8') + }) if resp.status == 429: # Too Many Requests raise UserCodeQuotaException(resp.reason) @@ -113,7 +107,8 @@ class TestEnrichment(unittest.TestCase): @classmethod def setUpClass(cls) -> None: cls.options = EchoITOptions() - http_endpoint_address = 'http://10.138.0.32:8080' + # http_endpoint_address = 'http://10.138.0.32:8080' + http_endpoint_address = 'http://localhost:8080' cls.client = SampleHTTPEnrichment(http_endpoint_address) @classmethod @@ -127,7 +122,7 @@ def test_http_enrichment(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint with the default cross join.""" client, options = TestEnrichment._get_client_and_options() - req = _Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = {'id': options.never_exceed_quota_id, 'payload': _PAYLOAD} fields = ['id', 'payload', 'resp_payload'] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( @@ -140,7 +135,7 @@ def test_http_enrichment_custom_join(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint with a custom join function.""" client, options = TestEnrichment._get_client_and_options() - req = _Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = {'id': options.never_exceed_quota_id, 'payload': _PAYLOAD} fields = ['id', 'resp_payload', 'timestamp'] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( diff --git a/sdks/python/apache_beam/transforms/enrichment_test.py b/sdks/python/apache_beam/transforms/enrichment_test.py index 687954612372f..8cc8802db28c8 100644 --- a/sdks/python/apache_beam/transforms/enrichment_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_test.py @@ -24,7 +24,7 @@ class TestEnrichmentTransform(unittest.TestCase): def test_cross_join(self): - rows = (beam.Row(id=1, key='city'), beam.Row(id=1, value='durham')) + rows = ({'id': 1, 'key': 'city'}, {'id': 1, 'value': 'durham'}) expected = beam.Row(id=1, key='city', value='durham') output = cross_join(rows) self.assertEqual(expected, output) From 9fd68138d9d566285f42d7def3f58b22efc37a76 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 4 Jan 2024 11:37:07 -0500 Subject: [PATCH 18/45] callable type --- sdks/python/apache_beam/io/requestresponse_it_test.py | 1 - sdks/python/apache_beam/transforms/enrichment.py | 6 ++++-- sdks/python/apache_beam/transforms/enrichment_it_test.py | 3 +-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index 79160e645afee..4550745a9c02d 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -121,7 +121,6 @@ class EchoHTTPCallerTestIT(unittest.TestCase): def setUpClass(cls) -> None: cls.options = EchoITOptions() http_endpoint_address = cls.options.http_endpoint_address - http_endpoint_address = 'http://localhost:8080' if not http_endpoint_address or http_endpoint_address == '': raise unittest.SkipTest(f'{_HTTP_ENDPOINT_ADDRESS_FLAG} is required.') diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index e985442f96cd3..1a9199b267d14 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -15,7 +15,9 @@ # limitations under the License. # +from typing import Any from typing import Callable +from typing import Dict from typing import Optional from typing import Tuple from typing import TypeVar @@ -34,10 +36,10 @@ InputT = TypeVar('InputT') OutputT = TypeVar('OutputT') -JoinFn = Callable[[Tuple[beam.Row, beam.Row]], beam.Row] +JoinFn = Callable[[Tuple[Dict[Any], Dict[Any]]], beam.Row] -def cross_join(element: Tuple[dict, dict]) -> beam.Row: +def cross_join(element: Tuple[Dict[Any], Dict[Any]]) -> beam.Row: """cross_join performs a cross join on two `dict` objects. Joins the columns of the right row onto the left row. diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index e9ce12b4883c2..58b743937612c 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -107,8 +107,7 @@ class TestEnrichment(unittest.TestCase): @classmethod def setUpClass(cls) -> None: cls.options = EchoITOptions() - # http_endpoint_address = 'http://10.138.0.32:8080' - http_endpoint_address = 'http://localhost:8080' + http_endpoint_address = 'http://10.138.0.32:8080' # endpoint of mock api cls.client = SampleHTTPEnrichment(http_endpoint_address) @classmethod From 036ecebc5b8827de4966426cec0aad52b40bf375 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 4 Jan 2024 12:10:01 -0500 Subject: [PATCH 19/45] dict type --- sdks/python/apache_beam/transforms/enrichment.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 1a9199b267d14..213931f474087 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -36,10 +36,10 @@ InputT = TypeVar('InputT') OutputT = TypeVar('OutputT') -JoinFn = Callable[[Tuple[Dict[Any], Dict[Any]]], beam.Row] +JoinFn = Callable[[Tuple[Dict[str, Any], Dict[str, Any]]], beam.Row] -def cross_join(element: Tuple[Dict[Any], Dict[Any]]) -> beam.Row: +def cross_join(element: Tuple[Dict[str, Any], Dict[str, Any]]) -> beam.Row: """cross_join performs a cross join on two `dict` objects. Joins the columns of the right row onto the left row. From 021f9c4470b45a639b6e305c7da5ce95fee56346 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 9 Jan 2024 14:58:39 -0500 Subject: [PATCH 20/45] update signatures --- sdks/python/apache_beam/io/requestresponse.py | 6 +++--- sdks/python/apache_beam/io/requestresponse_test.py | 2 +- sdks/python/apache_beam/transforms/enrichment.py | 12 +++++------- .../apache_beam/transforms/enrichment_it_test.py | 7 +++---- 4 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 6b3c1ae7615fc..0007d3ae2747e 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -109,7 +109,7 @@ class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], """ def __init__( self, - caller: Caller, + caller: Caller[RequestT, ResponseT], timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, repeater: Optional[Repeater] = None, @@ -171,7 +171,7 @@ class _Call(beam.PTransform[beam.PCollection[RequestT], """ def __init__( self, - caller: Caller, + caller: Caller[RequestT, ResponseT], timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, repeater: Optional[Repeater] = None, @@ -201,7 +201,7 @@ class _CallDoFn(beam.DoFn): def setup(self): self._caller.__enter__() - def __init__(self, caller: Caller, timeout: float): + def __init__(self, caller: Caller[RequestT, ResponseT], timeout: float): self._caller = caller self._timeout = timeout diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index 780570e6b4997..735d67c63e783 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -25,7 +25,7 @@ from apache_beam.testing.test_pipeline import TestPipeline -class AckCaller(Caller): +class AckCaller(Caller[str, str]): """AckCaller acknowledges the incoming request by returning a request with ACK.""" def __enter__(self): diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 213931f474087..c8ba7436d747b 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -19,7 +19,6 @@ from typing import Callable from typing import Dict from typing import Optional -from typing import Tuple from typing import TypeVar import apache_beam as beam @@ -36,22 +35,21 @@ InputT = TypeVar('InputT') OutputT = TypeVar('OutputT') -JoinFn = Callable[[Tuple[Dict[str, Any], Dict[str, Any]]], beam.Row] +JoinFn = Callable[[Dict[str, Any], Dict[str, Any]], beam.Row] -def cross_join(element: Tuple[Dict[str, Any], Dict[str, Any]]) -> beam.Row: +def cross_join(left: Dict[str, Any], right: Dict[str, Any]) -> beam.Row: """cross_join performs a cross join on two `dict` objects. Joins the columns of the right row onto the left row. Args: - element (Tuple): A tuple containing two `dict` objects - - request and response. + left (Dict[str, Any]): input request dictionary. + right (Dict[str, Any]): response dictionary from the API. Returns: `beam.Row` containing the merged columns. """ - left, right = element for k, v in right.items(): if k not in left: # Don't override the values in left. @@ -105,4 +103,4 @@ def expand(self, caller=self._source_handler, timeout=self._timeout) # EnrichmentSourceHandler returns a tuple of (request,response). - return fetched_data | beam.Map(self._join_fn) + return fetched_data | beam.Map(lambda x: self._join_fn(x[0], x[1])) diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 58b743937612c..5efe75ea1b84e 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -33,11 +33,10 @@ from apache_beam.transforms.enrichment import EnrichmentSourceHandler -def _custom_join(element): +def _custom_join(left, right): """custom_join returns the id and resp_payload along with a timestamp""" - right_dict = element[1] - right_dict['timestamp'] = time.time() - return beam.Row(**right_dict) + right['timestamp'] = time.time() + return beam.Row(**right) class SampleHTTPEnrichment(EnrichmentSourceHandler[dict, beam.Row]): From 062b9efdcbed6446bfe9216ee84270ed35c91423 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 9 Jan 2024 15:22:24 -0500 Subject: [PATCH 21/45] fix unit test --- sdks/python/apache_beam/transforms/enrichment_test.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_test.py b/sdks/python/apache_beam/transforms/enrichment_test.py index 8cc8802db28c8..74545bb548a74 100644 --- a/sdks/python/apache_beam/transforms/enrichment_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_test.py @@ -24,9 +24,10 @@ class TestEnrichmentTransform(unittest.TestCase): def test_cross_join(self): - rows = ({'id': 1, 'key': 'city'}, {'id': 1, 'value': 'durham'}) + left = {'id': 1, 'key': 'city'} + right = {'id': 1, 'value': 'durham'} expected = beam.Row(id=1, key='city', value='durham') - output = cross_join(rows) + output = cross_join(left, right) self.assertEqual(expected, output) From b11d3ea5a5c1ebf49da9a0b121f1bf37cb0eaa95 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 11 Jan 2024 15:35:48 -0500 Subject: [PATCH 22/45] bigtable with column family, ids, rrio-throttler --- sdks/python/apache_beam/io/requestresponse.py | 253 +++++++++++++++--- .../apache_beam/io/requestresponse_it_test.py | 38 +++ .../apache_beam/io/requestresponse_test.py | 41 +++ .../apache_beam/transforms/enrichment.py | 2 +- .../enrichment_handlers/__init__.py | 16 ++ .../enrichment_handlers/bigtable.py | 104 +++++++ .../enrichment_handlers/bigtable_it_test.py | 101 +++++++ .../enrichment_handlers/bigtable_test.py | 0 8 files changed, 522 insertions(+), 33 deletions(-) create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py create mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_test.py diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 0007d3ae2747e..57ee20661aaa9 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -21,11 +21,18 @@ import contextlib import logging import sys +import time from typing import Generic from typing import Optional from typing import TypeVar +from google.api_core.exceptions import TooManyRequests + import apache_beam as beam +from apache_beam.io.components.adaptive_throttler import AdaptiveThrottler +from apache_beam.metrics import Metrics +from apache_beam.ml.inference.vertex_ai_inference import MSEC_TO_SEC +from apache_beam.utils import retry RequestT = TypeVar('RequestT') ResponseT = TypeVar('ResponseT') @@ -49,6 +56,36 @@ class UserCodeTimeoutException(UserCodeExecutionException): """Extends ``UserCodeExecutionException`` to signal a user code timeout.""" +def retry_on_exception(exception: Exception): + """retry on exceptions caused by unavailability of the remote server.""" + return isinstance( + exception, + (TooManyRequests, UserCodeTimeoutException, UserCodeQuotaException)) + + +class _MetricsCollector: + """A metrics collector that tracks RequestResponseIO related usage.""" + def __init__(self, namespace: str): + """ + Args: + namespace: Namespace for the metrics. + """ + self.requests = Metrics.counter(namespace, 'requests') + self.responses = Metrics.counter(namespace, 'responses') + self.failures = Metrics.counter(namespace, 'failures') + self.throttled_requests = Metrics.counter(namespace, 'throttled_requests') + self.throttled_secs = Metrics.counter( + namespace, 'cumulativeThrottlingSeconds') + self.timeout_requests = Metrics.counter(namespace, 'requests_timed_out') + self.call_counter = Metrics.counter(namespace, 'call_invocations') + self.setup_counter = Metrics.counter(namespace, 'setup_counter') + self.teardown_counter = Metrics.counter(namespace, 'teardown_counter') + self.backoff_counter = Metrics.counter(namespace, 'backoff_counter') + self.sleeper_counter = Metrics.counter(namespace, 'sleeper_counter') + self.should_backoff_counter = Metrics.counter( + namespace, 'should_backoff_counter') + + class Caller(contextlib.AbstractContextManager, abc.ABC, Generic[RequestT, ResponseT]): @@ -81,7 +118,101 @@ class ShouldBackOff(abc.ABC): class Repeater(abc.ABC): """Repeater provides mechanism to repeat requests for a configurable condition.""" - pass + @abc.abstractmethod + def repeat( + self, + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector]) -> ResponseT: + """ + repeat method is called from the RequestResponseIO when a repeater is + enabled. + + Args: + caller: :class:`apache_beam.io.requestresponse.Caller` object that calls + the API. + request: input request to repeat. + timeout: time to wait for the request to complete. + metrics_collector: (Optional) a + :class:`apache_beam.io.requestresponse._MetricsCollector` object to + collect the metrics for RequestResponseIO. + """ + pass + + +def _execute_request( + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector] = None) -> ResponseT: + with concurrent.futures.ThreadPoolExecutor() as executor: + future = executor.submit(caller, request) + try: + return future.result(timeout=timeout) + except TooManyRequests as e: + _LOGGER.warning( + 'request could not be completed. got code %i from the service.', + e.code) + raise e + except concurrent.futures.TimeoutError: + if metrics_collector: + metrics_collector.timeout_requests.inc(1) + raise UserCodeTimeoutException( + f'Timeout {timeout} exceeded ' + f'while completing request: {request}') + except RuntimeError: + if metrics_collector: + metrics_collector.failures.inc(1) + raise UserCodeExecutionException('could not complete request') + + +class ExponentialBackOffRepeater(Repeater): + """Exponential BackOff Repeater uses exponential backoff retry strategy for + exceptions due to the remote service such as TooManyRequests (HTTP 429), + UserCodeTimeoutException, UserCodeQuotaException. + + It utilizes the decorator + :func:`apache_beam.utils.retry.with_exponential_backoff`. + """ + def __init__(self): + pass + + @retry.with_exponential_backoff( + num_retries=2, retry_filter=retry_on_exception) + def repeat( + self, + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector] = None) -> ResponseT: + """ + repeat method is called from the RequestResponseIO when a repeater is + enabled. + + Args: + caller: :class:`apache_beam.io.requestresponse.Caller` object that calls + the API. + request: input request to repeat. + timeout: time to wait for the request to complete. + metrics_collector: (Optional) a + :class:`apache_beam.io.requestresponse._MetricsCollector` object to + collect the metrics for RequestResponseIO. + """ + return _execute_request(caller, request, timeout, metrics_collector) + + +class NoOpsRepeater(Repeater): + """ + NoOpsRepeater executes a request just once irrespective of any exception. + """ + def repeat( + self, + caller: Caller[RequestT, ResponseT], + request: RequestT, + timeout: float, + metrics_collector: Optional[_MetricsCollector]) -> ResponseT: + return _execute_request(caller, request, timeout, metrics_collector) class CacheReader(abc.ABC): @@ -99,6 +230,29 @@ class PreCallThrottler(abc.ABC): pass +class DefaultThrottler(PreCallThrottler): + """Default throttler that uses + :class:`apache_beam.io.components.adaptive_throttler.AdaptiveThrottler` + + Args: + window_ms (int): length of history to consider, in ms, to set throttling. + bucket_ms (int): granularity of time buckets that we store data in, in ms. + overload_ratio (float): the target ratio between requests sent and + successful requests. This is "K" in the formula in + https://landing.google.com/sre/book/chapters/handling-overload.html. + delay_secs (int): minimum number of seconds to throttle a request. + """ + def __init__( + self, + window_ms: int = 1, + bucket_ms: int = 1, + overload_ratio: float = 2, + delay_secs: int = 5): + self.throttler = AdaptiveThrottler( + window_ms=window_ms, bucket_ms=bucket_ms, overload_ratio=overload_ratio) + self.delay_secs = delay_secs + + class RequestResponseIO(beam.PTransform[beam.PCollection[RequestT], beam.PCollection[ResponseT]]): """A :class:`RequestResponseIO` transform to read and write to APIs. @@ -112,10 +266,10 @@ def __init__( caller: Caller[RequestT, ResponseT], timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, - repeater: Optional[Repeater] = None, + repeater: Optional[Repeater] = ExponentialBackOffRepeater(), cache_reader: Optional[CacheReader] = None, cache_writer: Optional[CacheWriter] = None, - throttler: Optional[PreCallThrottler] = None, + throttler: Optional[PreCallThrottler] = DefaultThrottler(), ): """ Instantiates a RequestResponseIO transform. @@ -138,7 +292,10 @@ def __init__( self._caller = caller self._timeout = timeout self._should_backoff = should_backoff - self._repeater = repeater + if repeater: + self._repeater = repeater + else: + self._repeater = NoOpsRepeater() self._cache_reader = cache_reader self._cache_writer = cache_writer self._throttler = throttler @@ -146,12 +303,20 @@ def __init__( def expand( self, requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: - # TODO(riteshghorse): add Cache and Throttle PTransforms. - return requests | _Call( - caller=self._caller, - timeout=self._timeout, - should_backoff=self._should_backoff, - repeater=self._repeater) + # TODO(riteshghorse): handle Cache and Throttle PTransforms when available. + if isinstance(self._throttler, DefaultThrottler): + return requests | _Call( + caller=self._caller, + timeout=self._timeout, + should_backoff=self._should_backoff, + repeater=self._repeater, + throttler=self._throttler) + else: + return requests | _Call( + caller=self._caller, + timeout=self._timeout, + should_backoff=self._should_backoff, + repeater=self._repeater) class _Call(beam.PTransform[beam.PCollection[RequestT], @@ -168,6 +333,12 @@ class _Call(beam.PTransform[beam.PCollection[RequestT], caller (:class:`apache_beam.io.requestresponse.Caller`): a callable object that invokes API call. timeout (float): timeout value in seconds to wait for response from API. + should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): + (Optional) provides methods for backoff. + repeater (~apache_beam.io.requestresponse.Repeater): (Optional) provides + methods to repeat requests to API. + throttler (~apache_beam.io.requestresponse.PreCallThrottler): + (Optional) provides methods to pre-throttle a request. """ def __init__( self, @@ -175,47 +346,65 @@ def __init__( timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, repeater: Optional[Repeater] = None, + throttler: Optional[PreCallThrottler] = None, ): - """Initialize the _Call transform. - Args: - caller (:class:`apache_beam.io.requestresponse.Caller`): a callable - object that invokes API call. - timeout (float): timeout value in seconds to wait for response from API. - should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): - (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponse.Repeater): (Optional) provides - methods to repeat requests to API. - """ self._caller = caller self._timeout = timeout self._should_backoff = should_backoff self._repeater = repeater + self._throttler = throttler def expand( self, requests: beam.PCollection[RequestT]) -> beam.PCollection[ResponseT]: - return requests | beam.ParDo(_CallDoFn(self._caller, self._timeout)) + return requests | beam.ParDo( + _CallDoFn(self._caller, self._timeout, self._repeater, self._throttler)) class _CallDoFn(beam.DoFn): def setup(self): self._caller.__enter__() + self._metrics_collector = _MetricsCollector(self._caller.__str__()) + self._metrics_collector.setup_counter.inc(1) - def __init__(self, caller: Caller[RequestT, ResponseT], timeout: float): + def __init__( + self, + caller: Caller[RequestT, ResponseT], + timeout: float, + repeater: Repeater, + throttler: PreCallThrottler): + self._metrics_collector = None self._caller = caller self._timeout = timeout + self._repeater = repeater + self._throttler = throttler def process(self, request: RequestT, *args, **kwargs): - with concurrent.futures.ThreadPoolExecutor() as executor: - future = executor.submit(self._caller, request) - try: - yield future.result(timeout=self._timeout) - except concurrent.futures.TimeoutError: - raise UserCodeTimeoutException( - f'Timeout {self._timeout} exceeded ' - f'while completing request: {request}') - except RuntimeError: - raise UserCodeExecutionException('could not complete request') + self._metrics_collector.requests.inc(1) + + is_throttled_request = False + if self._throttler: + while self._throttler.throttler.throttle_request(time.time() * + MSEC_TO_SEC): + _LOGGER.info( + "Delaying request for %d seconds" % self._throttler.delay_secs) + time.sleep(self._throttler.delay_secs) + self._metrics_collector.throttled_secs.inc(5) + is_throttled_request = True + + if is_throttled_request: + self._metrics_collector.throttled_requests.inc(1) + + try: + req_time = time.time() + response = self._repeater.repeat( + self._caller, request, self._timeout, self._metrics_collector) + self._metrics_collector.responses.inc(1) + self._throttler.throttler.successful_request(req_time * MSEC_TO_SEC) + yield response + except Exception as e: + raise e def teardown(self): + self._metrics_collector.teardown_counter.inc(1) self._caller.__exit__(*sys.exc_info()) diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index 4550745a9c02d..f4841b353b3d1 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -113,6 +113,44 @@ def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: raise UserCodeExecutionException(e) +class EchoHTTPCaller_429(Caller[EchoRequest, EchoResponse]): + """Implements ``Caller`` to call the ``EchoServiceGrpc``'s HTTP handler. + The purpose of ``EchoHTTPCaller`` is to support integration tests. + """ + def __init__(self, url: str): + self.url = url + _HTTP_PATH + + def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: + """Overrides ``Caller``'s call method invoking the + ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning + either a successful ``EchoResponse`` or throwing either a + ``UserCodeExecutionException``, ``UserCodeTimeoutException``, + or a ``UserCodeQuotaException``. + """ + try: + resp = urllib3.request( + "POST", + self.url, + json={ + "id": request.id, "payload": str(request.payload, 'utf-8') + }, + retries=False) + + if resp.status < 300: + resp_body = resp.json() + resp_id = resp_body['id'] + payload = resp_body['payload'] + return EchoResponse(id=resp_id, payload=bytes(payload, 'utf-8')) + + if resp.status == 429: # Too Many Requests + raise UserCodeQuotaException(resp.reason) + else: + raise UserCodeExecutionException(resp.status, resp.reason, request) + + except urllib3.exceptions.HTTPError as e: + raise UserCodeExecutionException(e) + + class EchoHTTPCallerTestIT(unittest.TestCase): options: Union[EchoITOptions, None] = None client: Union[EchoHTTPCaller, None] = None diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index 735d67c63e783..586102aa34437 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -17,7 +17,10 @@ import time import unittest +from google.api_core.exceptions import TooManyRequests + import apache_beam as beam +from apache_beam.io.requestresponse import retry_on_exception from apache_beam.io.requestresponse import Caller from apache_beam.io.requestresponse import RequestResponseIO from apache_beam.io.requestresponse import UserCodeExecutionException @@ -54,6 +57,20 @@ def __call__(self, request: str, *args, **kwargs): raise RuntimeError("Exception expected, not an error.") +class CallerThatRetries(AckCaller): + def __init__(self): + self.count = -1 + + def __call__(self, request: str, *args, **kwargs): + try: + pass + except Exception as e: + raise e + finally: + self.count += 1 + raise TooManyRequests('retries = %d' % self.count) + + class TestCaller(unittest.TestCase): def test_valid_call(self): caller = AckCaller() @@ -83,6 +100,30 @@ def test_call_runtime_error(self): | beam.Create([""]) | RequestResponseIO(caller=caller)) + def test_retry_on_exception(self): + self.assertFalse(retry_on_exception(RuntimeError())) + self.assertTrue(retry_on_exception(TooManyRequests("HTTP 429"))) + + def test_caller_backoff_retry_strategy(self): + caller = CallerThatRetries() + with self.assertRaises(TooManyRequests) as cm: + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create(["sample_request"]) + | RequestResponseIO(caller=caller)) + self.assertRegex(cm.exception.message, 'retries = 2') + + def test_caller_no_retry_strategy(self): + caller = CallerThatRetries() + with self.assertRaises(TooManyRequests) as cm: + with TestPipeline() as test_pipeline: + _ = ( + test_pipeline + | beam.Create(["sample_request"]) + | RequestResponseIO(caller=caller, repeater=None)) + self.assertRegex(cm.exception.message, 'retries = 0') + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index c8ba7436d747b..9eec50280338e 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -72,7 +72,7 @@ class Enrichment(beam.PTransform[beam.PCollection[InputT], enrich elements in a PCollection. **NOTE:** This transform and its implementation are under development and do not provide backward compatibility guarantees. - Uses the :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` + Uses the :class:`apache_beam.transforms.enrichment.EnrichmenwtSourceHandler` to enrich elements by joining the metadata from external source. Processes an input :class:`~apache_beam.pvalue.PCollection` of `beam.Row` by diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py b/sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/__init__.py @@ -0,0 +1,16 @@ +# +# 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 "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 "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. +# diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py new file mode 100644 index 0000000000000..2d13b18dc88ff --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -0,0 +1,104 @@ +# +# 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 "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 "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. +# +import logging +from typing import List +from typing import Optional + +from google.api_core.exceptions import NotFound +from google.cloud import bigtable + +import apache_beam as beam +from apache_beam.transforms.enrichment import EnrichmentSourceHandler + +__all__ = [ + 'EnrichWithBigTable', +] + +_LOGGER = logging.getLogger(__name__) + + +class EnrichWithBigTable(EnrichmentSourceHandler[dict, beam.Row]): + """EnrichWithBigTable is a handler for + :class:`apache_beam.transforms.enrichment.Enrichment` transform to interact + with GCP BigTable. + + Args: + project_id (str): GCP project-id of the BigTable cluster. + instance_id (str): GCP instance-id of the BigTable cluster. + table_id (str): GCP table-id of the BigTable. + row_key (str): unique row key for BigTable + column_family_ids (List(str)) + """ + def __init__( + self, + project_id: str, + instance_id: str, + table_id: str, + row_key: str, + column_family_ids: Optional[List[str]] = None, + column_ids: Optional[List[str]] = None): + self._project_id = project_id + self._instance_id = instance_id + self._table_id = table_id + self._row_key = row_key + self._column_family_ids = column_family_ids + self._column_ids = column_ids + + def __enter__(self): + client = bigtable.Client(project=self._project_id) + instance = client.instance(self._instance_id) + self._table = instance.table(self._table_id) + + def __call__(self, request: dict, *args, **kwargs): + row_key = request[self._row_key].encode() + row = self._table.read_row(row_key) + response_dict = {} + + try: + if self._column_family_ids and self._column_ids: + for column_family_id in self._column_family_ids: + response_dict[column_family_id] = {} + for column_id in self._column_ids: + response_dict[column_family_id][column_id] = row.cells[ + column_family_id][column_id.encode()][0].value.decode('utf-8') + elif self._column_family_ids: + for column_family_id in self._column_family_ids: + response_dict[column_family_id] = {} + for k, v in row.cells[column_family_id].items(): + response_dict[column_family_id][k.decode( + 'utf-8')] = v[0].value.decode('utf-8') + elif self._column_ids: + for cf_id, cf_v in row.cells.items(): + response_dict[cf_id] = {} + for c_id in self._column_ids: + if c_id.encode() in cf_v: + response_dict[cf_id][c_id] = cf_v[c_id.encode()][0].value.decode( + 'utf-8') + else: + for cf_id, cf_v in row.cells.items(): + response_dict[cf_id] = {} + for k, v in cf_v.items(): + response_dict[cf_id][k.decode('utf-8')] = v[0].value.decode('utf-8') + except NotFound: + _LOGGER.warning('request row_key: %s not found') + except Exception as e: + raise e + + return request, response_dict + + def __exit__(self, exc_type, exc_val, exc_tb): + self._table = None diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py new file mode 100644 index 0000000000000..fa7ae44635494 --- /dev/null +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -0,0 +1,101 @@ +# +# 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 "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 "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. +# + +import typing +import unittest + +import apache_beam as beam +from apache_beam.io import WriteToText +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.transforms.enrichment import Enrichment +from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable + + +class _Currency(typing.NamedTuple): + s_id: int + id: str + + +class TestBigTableEnrichment(unittest.TestCase): + def __init__(self): + self.project_id = 'google.com:clouddfe' + self.instance_id = 'beam-test' + self.table_id = 'riteshghorse-test' + self.req = {'s_id': 1, 'id': 'usd'} + self.row_key = 'id' + + def test_enrichment_with_bigtable(self): + column_family_ids = ['test-column'] + column_ids = ['id', 'value'] + bigtable = EnrichWithBigTable( + self.project_id, + self.instance_id, + self.table_id, + self.row_key, + column_family_ids, + column_ids) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create([self.req]) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | 'Write' >> WriteToText('1enrich.txt')) + + def test_enrichment_with_bigtable_no_column_family(self): + column_ids = ['id', 'value'] + bigtable = EnrichWithBigTable( + self.project_id, + self.instance_id, + self.table_id, + self.row_key, + column_ids=column_ids) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create([self.req]) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | 'Write' >> WriteToText('1enrich.txt')) + + def test_enrichment_with_bigtable_no_column_ids(self): + column_family_ids = ['test-column'] + bigtable = EnrichWithBigTable( + self.project_id, + self.instance_id, + self.table_id, + self.row_key, + column_family_ids=column_family_ids) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create([self.req]) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | 'Write' >> WriteToText('2enrich.txt')) + + def test_enrichment_with_bigtable_no_hints(self): + req = {'s_id': 1, 'id': 'usd'} + bigtable = EnrichWithBigTable( + self.project_id, self.instance_id, self.table_id, self.row_key) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create([req]) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | 'Write' >> WriteToText('3enrich.txt')) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_test.py new file mode 100644 index 0000000000000..e69de29bb2d1d From 46e3a6de45234f64ebbbf555ee755b4a5b4934eb Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 11 Jan 2024 17:21:05 -0500 Subject: [PATCH 23/45] update tests for row filter --- .../enrichment_handlers/bigtable.py | 46 +++++------------- .../enrichment_handlers/bigtable_it_test.py | 47 ++++--------------- 2 files changed, 21 insertions(+), 72 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 2d13b18dc88ff..bc8c23b323c55 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -15,11 +15,11 @@ # limitations under the License. # import logging -from typing import List from typing import Optional from google.api_core.exceptions import NotFound from google.cloud import bigtable +from google.cloud.bigtable.row_filters import RowFilter import apache_beam as beam from apache_beam.transforms.enrichment import EnrichmentSourceHandler @@ -41,7 +41,8 @@ class EnrichWithBigTable(EnrichmentSourceHandler[dict, beam.Row]): instance_id (str): GCP instance-id of the BigTable cluster. table_id (str): GCP table-id of the BigTable. row_key (str): unique row key for BigTable - column_family_ids (List(str)) + row_filter: :class:`google.cloud.bigtable.row_filters.RowFilter` to filter + data read with ``read_row()``. """ def __init__( self, @@ -49,14 +50,12 @@ def __init__( instance_id: str, table_id: str, row_key: str, - column_family_ids: Optional[List[str]] = None, - column_ids: Optional[List[str]] = None): + row_filter: Optional[RowFilter] = None): self._project_id = project_id self._instance_id = instance_id self._table_id = table_id self._row_key = row_key - self._column_family_ids = column_family_ids - self._column_ids = column_ids + self._row_filter = row_filter def __enter__(self): client = bigtable.Client(project=self._project_id) @@ -64,35 +63,14 @@ def __enter__(self): self._table = instance.table(self._table_id) def __call__(self, request: dict, *args, **kwargs): - row_key = request[self._row_key].encode() - row = self._table.read_row(row_key) - response_dict = {} - try: - if self._column_family_ids and self._column_ids: - for column_family_id in self._column_family_ids: - response_dict[column_family_id] = {} - for column_id in self._column_ids: - response_dict[column_family_id][column_id] = row.cells[ - column_family_id][column_id.encode()][0].value.decode('utf-8') - elif self._column_family_ids: - for column_family_id in self._column_family_ids: - response_dict[column_family_id] = {} - for k, v in row.cells[column_family_id].items(): - response_dict[column_family_id][k.decode( - 'utf-8')] = v[0].value.decode('utf-8') - elif self._column_ids: - for cf_id, cf_v in row.cells.items(): - response_dict[cf_id] = {} - for c_id in self._column_ids: - if c_id.encode() in cf_v: - response_dict[cf_id][c_id] = cf_v[c_id.encode()][0].value.decode( - 'utf-8') - else: - for cf_id, cf_v in row.cells.items(): - response_dict[cf_id] = {} - for k, v in cf_v.items(): - response_dict[cf_id][k.decode('utf-8')] = v[0].value.decode('utf-8') + row_key = request[self._row_key].encode() + row = self._table.read_row(row_key, filter_=self._row_filter) + response_dict = {} + for cf_id, cf_v in row.cells.items(): + response_dict[cf_id] = {} + for k, v in cf_v.items(): + response_dict[cf_id][k.decode('utf-8')] = v[0].value.decode('utf-8') except NotFound: _LOGGER.warning('request row_key: %s not found') except Exception as e: diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index fa7ae44635494..5d11bd55dcae7 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -18,6 +18,8 @@ import typing import unittest +from google.cloud.bigtable.row_filters import ColumnRangeFilter + import apache_beam as beam from apache_beam.io import WriteToText from apache_beam.testing.test_pipeline import TestPipeline @@ -31,38 +33,17 @@ class _Currency(typing.NamedTuple): class TestBigTableEnrichment(unittest.TestCase): - def __init__(self): + def setUp(self): self.project_id = 'google.com:clouddfe' self.instance_id = 'beam-test' self.table_id = 'riteshghorse-test' self.req = {'s_id': 1, 'id': 'usd'} self.row_key = 'id' + self.column_family_id = 'test-column' def test_enrichment_with_bigtable(self): - column_family_ids = ['test-column'] - column_ids = ['id', 'value'] - bigtable = EnrichWithBigTable( - self.project_id, - self.instance_id, - self.table_id, - self.row_key, - column_family_ids, - column_ids) - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create([self.req]) - | "Enrich W/ BigTable" >> Enrichment(bigtable) - | 'Write' >> WriteToText('1enrich.txt')) - - def test_enrichment_with_bigtable_no_column_family(self): - column_ids = ['id', 'value'] bigtable = EnrichWithBigTable( - self.project_id, - self.instance_id, - self.table_id, - self.row_key, - column_ids=column_ids) + self.project_id, self.instance_id, self.table_id, self.row_key) with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( test_pipeline @@ -70,14 +51,15 @@ def test_enrichment_with_bigtable_no_column_family(self): | "Enrich W/ BigTable" >> Enrichment(bigtable) | 'Write' >> WriteToText('1enrich.txt')) - def test_enrichment_with_bigtable_no_column_ids(self): - column_family_ids = ['test-column'] + def test_enrichment_with_bigtable_row_filter(self): + start_column = 'value'.encode() + column_filter = ColumnRangeFilter(self.column_family_id, start_column) bigtable = EnrichWithBigTable( self.project_id, self.instance_id, self.table_id, self.row_key, - column_family_ids=column_family_ids) + row_filter=column_filter) with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( test_pipeline @@ -85,17 +67,6 @@ def test_enrichment_with_bigtable_no_column_ids(self): | "Enrich W/ BigTable" >> Enrichment(bigtable) | 'Write' >> WriteToText('2enrich.txt')) - def test_enrichment_with_bigtable_no_hints(self): - req = {'s_id': 1, 'id': 'usd'} - bigtable = EnrichWithBigTable( - self.project_id, self.instance_id, self.table_id, self.row_key) - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create([req]) - | "Enrich W/ BigTable" >> Enrichment(bigtable) - | 'Write' >> WriteToText('3enrich.txt')) - if __name__ == '__main__': unittest.main() From 433d5fa248266fe3f270ab9ecb0c954d17f78aa7 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 11 Jan 2024 18:18:39 -0500 Subject: [PATCH 24/45] convert handler types from dict to Row --- .../apache_beam/io/requestresponse_it_test.py | 62 ++++--------------- .../apache_beam/transforms/enrichment.py | 3 +- .../enrichment_handlers/bigtable.py | 9 +-- .../enrichment_handlers/bigtable_it_test.py | 2 +- .../transforms/enrichment_it_test.py | 14 ++--- 5 files changed, 26 insertions(+), 64 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index f4841b353b3d1..e74c77edae0de 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -16,6 +16,7 @@ # import base64 import sys +import typing import unittest from dataclasses import dataclass from typing import Tuple @@ -61,66 +62,27 @@ def _add_argparse_args(cls, parser) -> None: help='The ID for an allocated quota that should exceed.') -# TODO(riteshghorse,damondouglas) replace Echo(Request|Response) with proto -# generated classes from .test-infra/mock-apis: @dataclass -class EchoRequest: +class EchoResponse: id: str payload: bytes -@dataclass -class EchoResponse: +# TODO(riteshghorse,damondouglas) replace Echo(Request|Response) with proto +# generated classes from .test-infra/mock-apis: +class Request(typing.NamedTuple): id: str payload: bytes -class EchoHTTPCaller(Caller[EchoRequest, EchoResponse]): - """Implements ``Caller`` to call the ``EchoServiceGrpc``'s HTTP handler. - The purpose of ``EchoHTTPCaller`` is to support integration tests. - """ - def __init__(self, url: str): - self.url = url + _HTTP_PATH - - def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: - """Overrides ``Caller``'s call method invoking the - ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning - either a successful ``EchoResponse`` or throwing either a - ``UserCodeExecutionException``, ``UserCodeTimeoutException``, - or a ``UserCodeQuotaException``. - """ - try: - resp = urllib3.request( - "POST", - self.url, - json={ - "id": request.id, "payload": str(request.payload, 'utf-8') - }, - retries=False) - - if resp.status < 300: - resp_body = resp.json() - resp_id = resp_body['id'] - payload = resp_body['payload'] - return EchoResponse(id=resp_id, payload=bytes(payload, 'utf-8')) - - if resp.status == 429: # Too Many Requests - raise UserCodeQuotaException(resp.reason) - else: - raise UserCodeExecutionException(resp.status, resp.reason, request) - - except urllib3.exceptions.HTTPError as e: - raise UserCodeExecutionException(e) - - -class EchoHTTPCaller_429(Caller[EchoRequest, EchoResponse]): +class EchoHTTPCaller(Caller[Request, EchoResponse]): """Implements ``Caller`` to call the ``EchoServiceGrpc``'s HTTP handler. The purpose of ``EchoHTTPCaller`` is to support integration tests. """ def __init__(self, url: str): self.url = url + _HTTP_PATH - def __call__(self, request: EchoRequest, *args, **kwargs) -> EchoResponse: + def __call__(self, request: Request, *args, **kwargs) -> EchoResponse: """Overrides ``Caller``'s call method invoking the ``EchoServiceGrpc``'s HTTP handler with an ``EchoRequest``, returning either a successful ``EchoResponse`` or throwing either a @@ -167,7 +129,7 @@ def setUpClass(cls) -> None: def setUp(self) -> None: client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.should_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.should_exceed_quota_id, payload=_PAYLOAD) try: # The following is needed to exceed the API client(req) @@ -186,7 +148,7 @@ def _get_client_and_options(cls) -> Tuple[EchoHTTPCaller, EchoITOptions]: def test_given_valid_request_receives_response(self): client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) response: EchoResponse = client(req) @@ -196,20 +158,20 @@ def test_given_valid_request_receives_response(self): def test_given_exceeded_quota_should_raise(self): client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.should_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.should_exceed_quota_id, payload=_PAYLOAD) self.assertRaises(UserCodeQuotaException, lambda: client(req)) def test_not_found_should_raise(self): client, _ = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id='i-dont-exist-quota-id', payload=_PAYLOAD) + req = Request(id='i-dont-exist-quota-id', payload=_PAYLOAD) self.assertRaisesRegex( UserCodeExecutionException, "Not Found", lambda: client(req)) def test_request_response_io(self): client, options = EchoHTTPCallerTestIT._get_client_and_options() - req = EchoRequest(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) with TestPipeline(is_integration_test=True) as test_pipeline: output = ( test_pipeline diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 9eec50280338e..7026586d2e732 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -103,4 +103,5 @@ def expand(self, caller=self._source_handler, timeout=self._timeout) # EnrichmentSourceHandler returns a tuple of (request,response). - return fetched_data | beam.Map(lambda x: self._join_fn(x[0], x[1])) + return fetched_data | beam.Map( + lambda x: self._join_fn(x[0]._asdict(), x[1]._asdict())) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index bc8c23b323c55..da5111f2ee1fe 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -31,7 +31,7 @@ _LOGGER = logging.getLogger(__name__) -class EnrichWithBigTable(EnrichmentSourceHandler[dict, beam.Row]): +class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): """EnrichWithBigTable is a handler for :class:`apache_beam.transforms.enrichment.Enrichment` transform to interact with GCP BigTable. @@ -62,9 +62,10 @@ def __enter__(self): instance = client.instance(self._instance_id) self._table = instance.table(self._table_id) - def __call__(self, request: dict, *args, **kwargs): + def __call__(self, request: beam.Row, *args, **kwargs): try: - row_key = request[self._row_key].encode() + request_dict = request._asdict() + row_key = request_dict[self._row_key].encode() row = self._table.read_row(row_key, filter_=self._row_filter) response_dict = {} for cf_id, cf_v in row.cells.items(): @@ -76,7 +77,7 @@ def __call__(self, request: dict, *args, **kwargs): except Exception as e: raise e - return request, response_dict + return request, beam.Row(**response_dict) def __exit__(self, exc_type, exc_val, exc_tb): self._table = None diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 5d11bd55dcae7..ca945955db277 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -37,7 +37,7 @@ def setUp(self): self.project_id = 'google.com:clouddfe' self.instance_id = 'beam-test' self.table_id = 'riteshghorse-test' - self.req = {'s_id': 1, 'id': 'usd'} + self.req = beam.Row(s_id=1, id='usd') self.row_key = 'id' self.column_family_id = 'test-column' diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 5efe75ea1b84e..13685d01d883a 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -39,14 +39,14 @@ def _custom_join(left, right): return beam.Row(**right) -class SampleHTTPEnrichment(EnrichmentSourceHandler[dict, beam.Row]): +class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. """ def __init__(self, url: str): self.url = url + '/v1/echo' # append path to the mock API. - def __call__(self, request: dict, *args, **kwargs): + def __call__(self, request: beam.Row, *args, **kwargs): """Overrides ``Caller``'s call method invoking the ``EchoServiceGrpc``'s HTTP handler with an `dict`, returning either a successful ``Tuple[dict,dict]`` or throwing either a @@ -58,7 +58,7 @@ def __call__(self, request: dict, *args, **kwargs): "POST", self.url, json={ - "id": request['id'], "payload": str(request['payload'], 'utf-8') + "id": request.id, "payload": str(request.payload, 'utf-8') }, retries=False) @@ -67,9 +67,7 @@ def __call__(self, request: dict, *args, **kwargs): resp_id = resp_body['id'] payload = resp_body['payload'] return ( - request, { - 'id': resp_id, 'resp_payload': bytes(payload, 'utf-8') - }) + request, beam.Row(id=resp_id, resp_payload=bytes(payload, 'utf-8'))) if resp.status == 429: # Too Many Requests raise UserCodeQuotaException(resp.reason) @@ -120,7 +118,7 @@ def test_http_enrichment(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint with the default cross join.""" client, options = TestEnrichment._get_client_and_options() - req = {'id': options.never_exceed_quota_id, 'payload': _PAYLOAD} + req = beam.Row(id=options.never_exceed_quota_id, payload=_PAYLOAD) fields = ['id', 'payload', 'resp_payload'] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( @@ -133,7 +131,7 @@ def test_http_enrichment_custom_join(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint with a custom join function.""" client, options = TestEnrichment._get_client_and_options() - req = {'id': options.never_exceed_quota_id, 'payload': _PAYLOAD} + req = beam.Row(id=options.never_exceed_quota_id, payload=_PAYLOAD) fields = ['id', 'resp_payload', 'timestamp'] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( From d18d583446770e028243b0b110f7220e83de87d2 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 10:09:56 -0500 Subject: [PATCH 25/45] update tests for bigtable --- .../enrichment_handlers/bigtable.py | 26 ++++-- .../enrichment_handlers/bigtable_it_test.py | 80 ++++++++++++++++--- .../transforms/enrichment_it_test.py | 16 ++-- 3 files changed, 97 insertions(+), 25 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index da5111f2ee1fe..764be6a245bf6 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -15,7 +15,7 @@ # limitations under the License. # import logging -from typing import Optional +from typing import Optional, Tuple from google.api_core.exceptions import NotFound from google.cloud import bigtable @@ -58,16 +58,25 @@ def __init__( self._row_filter = row_filter def __enter__(self): - client = bigtable.Client(project=self._project_id) - instance = client.instance(self._instance_id) - self._table = instance.table(self._table_id) + """connect to the Google BigTable cluster.""" + self.client = bigtable.Client(project=self._project_id) + self.instance = self.client.instance(self._instance_id) + self._table = self.instance.table(self._table_id) - def __call__(self, request: beam.Row, *args, **kwargs): + def __call__(self, request: beam.Row, *args, + **kwargs) -> Tuple[beam.Row, beam.Row]: + """ + Reads a row from the Google BigTable and returns + a `Tuple` of request and response. + + Args: + request: the input `beam.Row` to enrich. + """ + response_dict = {} try: request_dict = request._asdict() - row_key = request_dict[self._row_key].encode() + row_key = str(request_dict[self._row_key]).encode() row = self._table.read_row(row_key, filter_=self._row_filter) - response_dict = {} for cf_id, cf_v in row.cells.items(): response_dict[cf_id] = {} for k, v in cf_v.items(): @@ -80,4 +89,7 @@ def __call__(self, request: beam.Row, *args, **kwargs): return request, beam.Row(**response_dict) def __exit__(self, exc_type, exc_val, exc_tb): + """Clean the instantiated BigTable client.""" + self.client = None + self.instance = None self._table = None diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index ca945955db277..1c511d8d35718 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -15,19 +15,50 @@ # limitations under the License. # -import typing import unittest +from typing import Dict +from typing import List +from typing import NamedTuple from google.cloud.bigtable.row_filters import ColumnRangeFilter import apache_beam as beam -from apache_beam.io import WriteToText from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import BeamAssertException from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable -class _Currency(typing.NamedTuple): +class ValidateResponse(beam.DoFn): + """ValidateResponse validates if a PCollection of `beam.Row` + has the required fields.""" + def __init__( + self, + n_fields: int, + fields: List[str], + enriched_fields: Dict[str, List[str]]): + self.n_fields = n_fields + self._fields = fields + self._enriched_fields = enriched_fields + + def process(self, element: beam.Row, *args, **kwargs): + element_dict = element.as_dict() + if len(element_dict.keys()) != self.n_fields: + raise BeamAssertException( + "Expected %d fields in enriched PCollection:" % self.n_fields) + + for field in self._fields: + if field not in element_dict or element_dict[field] is None: + raise BeamAssertException(f"Expected a not None field: {field}") + + for column_family, columns in self._enriched_fields.items(): + if not all(key in element_dict[column_family] for key in columns): + raise BeamAssertException( + "Response from bigtable should contain a %s column_family with " + "%s keys." % (column_family, columns)) + + +class _Currency(NamedTuple): s_id: int id: str @@ -36,23 +67,44 @@ class TestBigTableEnrichment(unittest.TestCase): def setUp(self): self.project_id = 'google.com:clouddfe' self.instance_id = 'beam-test' - self.table_id = 'riteshghorse-test' - self.req = beam.Row(s_id=1, id='usd') - self.row_key = 'id' - self.column_family_id = 'test-column' + self.table_id = 'riteshghorse-bigtable-test' + self.req = [ + beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), + beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), + beam.Row(sale_id=5, customer_id=5, product_id=4, quantity=2), + beam.Row(sale_id=7, customer_id=7, product_id=1, quantity=1), + ] + self.row_key = 'product_id' + self.column_family_id = 'product' def test_enrichment_with_bigtable(self): + expected_fields = [ + 'sale_id', 'customer_id', 'product_id', 'quantity', 'product' + ] + expected_enriched_fields = { + 'product': ['product_id', 'product_name', 'product_stock'], + } bigtable = EnrichWithBigTable( self.project_id, self.instance_id, self.table_id, self.row_key) with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( test_pipeline - | "Create" >> beam.Create([self.req]) + | "Create" >> beam.Create(self.req) | "Enrich W/ BigTable" >> Enrichment(bigtable) - | 'Write' >> WriteToText('1enrich.txt')) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) def test_enrichment_with_bigtable_row_filter(self): - start_column = 'value'.encode() + expected_fields = [ + 'sale_id', 'customer_id', 'product_id', 'quantity', 'product' + ] + expected_enriched_fields = { + 'product': ['product_name', 'product_stock'], + } + start_column = 'product_name'.encode() column_filter = ColumnRangeFilter(self.column_family_id, start_column) bigtable = EnrichWithBigTable( self.project_id, @@ -63,9 +115,13 @@ def test_enrichment_with_bigtable_row_filter(self): with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( test_pipeline - | "Create" >> beam.Create([self.req]) + | "Create" >> beam.Create(self.req) | "Enrich W/ BigTable" >> Enrichment(bigtable) - | 'Write' >> WriteToText('2enrich.txt')) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 13685d01d883a..5a7daeeab1c26 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -16,6 +16,7 @@ # import time import unittest +from typing import List from typing import Tuple from typing import Union @@ -81,15 +82,16 @@ def __call__(self, request: beam.Row, *args, **kwargs): class ValidateFields(beam.DoFn): """ValidateFields validates if a PCollection of `beam.Row` has certain fields.""" - def __init__(self, fields): + def __init__(self, n_fields: int, fields: List[str]): + self.n_fields = n_fields self._fields = fields def process(self, element: beam.Row, *args, **kwargs): element_dict = element.as_dict() - if len(element_dict.keys()) != 3: + if len(element_dict.keys()) != self.n_fields: raise BeamAssertException( - "Expected three fields in enriched PCollection:" - " id, payload and resp_payload") + "Expected %d fields in enriched PCollection:" + " id, payload and resp_payload" % self.n_fields) for field in self._fields: if field not in element_dict or element_dict[field] is None: @@ -125,7 +127,8 @@ def test_http_enrichment(self): test_pipeline | 'Create PCollection' >> beam.Create([req]) | 'Enrichment Transform' >> Enrichment(client) - | 'Assert Fields' >> beam.ParDo(ValidateFields(fields=fields))) + | 'Assert Fields' >> beam.ParDo( + ValidateFields(len(fields), fields=fields))) def test_http_enrichment_custom_join(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint @@ -138,7 +141,8 @@ def test_http_enrichment_custom_join(self): test_pipeline | 'Create PCollection' >> beam.Create([req]) | 'Enrichment Transform' >> Enrichment(client, join_fn=_custom_join) - | 'Assert Fields' >> beam.ParDo(ValidateFields(fields=fields))) + | 'Assert Fields' >> beam.ParDo( + ValidateFields(len(fields), fields=fields))) if __name__ == '__main__': From c5e792c1e3402e88bcedee20d2c6d6aa6288d59c Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 11:12:51 -0500 Subject: [PATCH 26/45] ran pydocs --- sdks/python/apache_beam/io/requestresponse.py | 32 +++++---- .../apache_beam/transforms/enrichment.py | 2 +- .../enrichment_handlers/bigtable.py | 4 +- .../enrichment_handlers/bigtable_it_test.py | 67 ++++++++++++++++++- 4 files changed, 83 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 57ee20661aaa9..6756d5cc5b82b 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -125,18 +125,17 @@ def repeat( request: RequestT, timeout: float, metrics_collector: Optional[_MetricsCollector]) -> ResponseT: - """ - repeat method is called from the RequestResponseIO when a repeater is - enabled. + """repeat method is called from the RequestResponseIO when + a repeater is enabled. Args: - caller: :class:`apache_beam.io.requestresponse.Caller` object that calls - the API. - request: input request to repeat. - timeout: time to wait for the request to complete. - metrics_collector: (Optional) a - :class:`apache_beam.io.requestresponse._MetricsCollector` object to - collect the metrics for RequestResponseIO. + caller: :class:`apache_beam.io.requestresponse.Caller` object that calls + the API. + request: input request to repeat. + timeout: time to wait for the request to complete. + metrics_collector: (Optional) a + ``:class:`apache_beam.io.requestresponse._MetricsCollector``` object to + collect the metrics for RequestResponseIO. """ pass @@ -186,17 +185,16 @@ def repeat( request: RequestT, timeout: float, metrics_collector: Optional[_MetricsCollector] = None) -> ResponseT: - """ - repeat method is called from the RequestResponseIO when a repeater is - enabled. + """repeat method is called from the RequestResponseIO when + a repeater is enabled. - Args: - caller: :class:`apache_beam.io.requestresponse.Caller` object that calls - the API. + Args: + caller: :class:`apache_beam.io.requestresponse.Caller` object that + calls the API. request: input request to repeat. timeout: time to wait for the request to complete. metrics_collector: (Optional) a - :class:`apache_beam.io.requestresponse._MetricsCollector` object to + ``:class:`apache_beam.io.requestresponse._MetricsCollector``` object to collect the metrics for RequestResponseIO. """ return _execute_request(caller, request, timeout, metrics_collector) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 7026586d2e732..1ac088328920b 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -72,7 +72,7 @@ class Enrichment(beam.PTransform[beam.PCollection[InputT], enrich elements in a PCollection. **NOTE:** This transform and its implementation are under development and do not provide backward compatibility guarantees. - Uses the :class:`apache_beam.transforms.enrichment.EnrichmenwtSourceHandler` + Uses the :class:`apache_beam.transforms.enrichment.EnrichmentSourceHandler` to enrich elements by joining the metadata from external source. Processes an input :class:`~apache_beam.pvalue.PCollection` of `beam.Row` by diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 764be6a245bf6..982ae398f0f64 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -41,8 +41,8 @@ class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): instance_id (str): GCP instance-id of the BigTable cluster. table_id (str): GCP table-id of the BigTable. row_key (str): unique row key for BigTable - row_filter: :class:`google.cloud.bigtable.row_filters.RowFilter` to filter - data read with ``read_row()``. + row_filter: a ``:class:`google.cloud.bigtable.row_filters.RowFilter``` to + filter data read with ``read_row()``. """ def __init__( self, diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 1c511d8d35718..68276dbc3a650 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -16,10 +16,12 @@ # import unittest +from datetime import datetime from typing import Dict from typing import List from typing import NamedTuple +from google.cloud import bigtable from google.cloud.bigtable.row_filters import ColumnRangeFilter import apache_beam as beam @@ -63,11 +65,65 @@ class _Currency(NamedTuple): id: str +def create_rows(table): + product_id = 'product_id' + product_name = 'product_name' + product_stock = 'product_stock' + + column_family_id = "product" + products = [ + { + 'product_id': 1, 'product_name': 'pixel 5', 'product_stock': 2 + }, + { + 'product_id': 2, 'product_name': 'pixel 6', 'product_stock': 4 + }, + { + 'product_id': 3, 'product_name': 'pixel 7', 'product_stock': 20 + }, + { + 'product_id': 4, 'product_name': 'pixel 8', 'product_stock': 10 + }, + { + 'product_id': 5, 'product_name': 'iphone 11', 'product_stock': 3 + }, + { + 'product_id': 6, 'product_name': 'iphone 12', 'product_stock': 7 + }, + { + 'product_id': 7, 'product_name': 'iphone 13', 'product_stock': 8 + }, + { + 'product_id': 8, 'product_name': 'iphone 14', 'product_stock': 3 + }, + ] + + for item in products: + row_key = str(item[product_id]).encode() + row = table.direct_row(row_key) + row.set_cell( + column_family_id, + product_id.encode(), + str(item[product_id]), + timestamp=datetime.datetime.utcnow()) + row.set_cell( + column_family_id, + product_name.encode(), + item[product_name], + timestamp=datetime.datetime.utcnow()) + row.set_cell( + column_family_id, + product_stock.encode(), + str(item[product_stock]), + timestamp=datetime.datetime.utcnow()) + row.commit() + + class TestBigTableEnrichment(unittest.TestCase): def setUp(self): - self.project_id = 'google.com:clouddfe' + self.project_id = 'apache-beam-testing' self.instance_id = 'beam-test' - self.table_id = 'riteshghorse-bigtable-test' + self.table_id = 'bigtable-enrichment-test' self.req = [ beam.Row(sale_id=1, customer_id=1, product_id=1, quantity=1), beam.Row(sale_id=3, customer_id=3, product_id=2, quantity=3), @@ -76,6 +132,13 @@ def setUp(self): ] self.row_key = 'product_id' self.column_family_id = 'product' + client = bigtable.Client(project=self.project_id) + instance = client.instance(self.instance_id) + self.table = instance.table(self.table_id) + create_rows(self.table) + + def tearDown(self) -> None: + self.table = None def test_enrichment_with_bigtable(self): expected_fields = [ From 9285a5bffb6e27bb6bb8995bec74f89e2d74f275 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 11:14:13 -0500 Subject: [PATCH 27/45] ran pydocs --- .../transforms/enrichment_handlers/bigtable_it_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 68276dbc3a650..a3cae1096af2f 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -15,8 +15,8 @@ # limitations under the License. # +import datetime import unittest -from datetime import datetime from typing import Dict from typing import List from typing import NamedTuple From 641bdf7db32d62017b4e9a9cdcf9e6f0ac9d3664 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 11:17:04 -0500 Subject: [PATCH 28/45] mark postcommit --- .../transforms/enrichment_handlers/bigtable_it_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index a3cae1096af2f..1002f46b2f633 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -21,6 +21,7 @@ from typing import List from typing import NamedTuple +import pytest from google.cloud import bigtable from google.cloud.bigtable.row_filters import ColumnRangeFilter @@ -119,6 +120,7 @@ def create_rows(table): row.commit() +@pytest.mark.it_postcommit class TestBigTableEnrichment(unittest.TestCase): def setUp(self): self.project_id = 'apache-beam-testing' From c36a21e5df978bce228d2c64e0276860d5893376 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 12:07:51 -0500 Subject: [PATCH 29/45] remove _test file, fix import --- .../apache_beam/transforms/enrichment_handlers/bigtable.py | 4 ++-- .../transforms/enrichment_handlers/bigtable_it_test.py | 4 ++-- .../transforms/enrichment_handlers/bigtable_test.py | 0 3 files changed, 4 insertions(+), 4 deletions(-) delete mode 100644 sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_test.py diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 982ae398f0f64..1319bf3895237 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -18,7 +18,7 @@ from typing import Optional, Tuple from google.api_core.exceptions import NotFound -from google.cloud import bigtable +from google.cloud.bigtable import Client from google.cloud.bigtable.row_filters import RowFilter import apache_beam as beam @@ -59,7 +59,7 @@ def __init__( def __enter__(self): """connect to the Google BigTable cluster.""" - self.client = bigtable.Client(project=self._project_id) + self.client = Client(project=self._project_id) self.instance = self.client.instance(self._instance_id) self._table = self.instance.table(self._table_id) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 1002f46b2f633..5beb3e62f869c 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -22,7 +22,7 @@ from typing import NamedTuple import pytest -from google.cloud import bigtable +from google.cloud.bigtable import Client from google.cloud.bigtable.row_filters import ColumnRangeFilter import apache_beam as beam @@ -134,7 +134,7 @@ def setUp(self): ] self.row_key = 'product_id' self.column_family_id = 'product' - client = bigtable.Client(project=self.project_id) + client = Client(project=self.project_id) instance = client.instance(self.instance_id) self.table = instance.table(self.table_id) create_rows(self.table) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_test.py deleted file mode 100644 index e69de29bb2d1d..0000000000000 From 3989c16fbae1cc9c2ffa5709a18edb18abfca5b7 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 12:15:37 -0500 Subject: [PATCH 30/45] enable postcommit --- .github/trigger_files/beam_PostCommit_Python.json | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Python.json diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json new file mode 100644 index 0000000000000..e69de29bb2d1d From 7102acd49e01828d26c47b00fe7a828a53999852 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Fri, 12 Jan 2024 12:58:33 -0500 Subject: [PATCH 31/45] add more tests --- .../apache_beam/io/requestresponse_test.py | 2 +- .../enrichment_handlers/bigtable.py | 14 ++--- .../enrichment_handlers/bigtable_it_test.py | 52 ++++++++++++++++++- .../transforms/enrichment_it_test.py | 14 +++-- 4 files changed, 69 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index 586102aa34437..c7a2f27455e51 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -20,11 +20,11 @@ from google.api_core.exceptions import TooManyRequests import apache_beam as beam -from apache_beam.io.requestresponse import retry_on_exception from apache_beam.io.requestresponse import Caller from apache_beam.io.requestresponse import RequestResponseIO from apache_beam.io.requestresponse import UserCodeExecutionException from apache_beam.io.requestresponse import UserCodeTimeoutException +from apache_beam.io.requestresponse import retry_on_exception from apache_beam.testing.test_pipeline import TestPipeline diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 1319bf3895237..b44f9535beb05 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -15,7 +15,7 @@ # limitations under the License. # import logging -from typing import Optional, Tuple +from typing import Optional from google.api_core.exceptions import NotFound from google.cloud.bigtable import Client @@ -63,8 +63,7 @@ def __enter__(self): self.instance = self.client.instance(self._instance_id) self._table = self.instance.table(self._table_id) - def __call__(self, request: beam.Row, *args, - **kwargs) -> Tuple[beam.Row, beam.Row]: + def __call__(self, request: beam.Row, *args, **kwargs): """ Reads a row from the Google BigTable and returns a `Tuple` of request and response. @@ -77,10 +76,11 @@ def __call__(self, request: beam.Row, *args, request_dict = request._asdict() row_key = str(request_dict[self._row_key]).encode() row = self._table.read_row(row_key, filter_=self._row_filter) - for cf_id, cf_v in row.cells.items(): - response_dict[cf_id] = {} - for k, v in cf_v.items(): - response_dict[cf_id][k.decode('utf-8')] = v[0].value.decode('utf-8') + if row: + for cf_id, cf_v in row.cells.items(): + response_dict[cf_id] = {} + for k, v in cf_v.items(): + response_dict[cf_id][k.decode('utf-8')] = v[0].value.decode('utf-8') except NotFound: _LOGGER.warning('request row_key: %s not found') except Exception as e: diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 5beb3e62f869c..bf53ce1d6ab05 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -55,7 +55,8 @@ def process(self, element: beam.Row, *args, **kwargs): raise BeamAssertException(f"Expected a not None field: {field}") for column_family, columns in self._enriched_fields.items(): - if not all(key in element_dict[column_family] for key in columns): + if (len(element_dict[column_family]) != len(columns) or + not all(key in element_dict[column_family] for key in columns)): raise BeamAssertException( "Response from bigtable should contain a %s column_family with " "%s keys." % (column_family, columns)) @@ -188,6 +189,55 @@ def test_enrichment_with_bigtable_row_filter(self): expected_fields, expected_enriched_fields))) + def test_enrichment_with_bigtable_no_enrichment(self): + expected_fields = ['sale_id', 'customer_id', 'product_id', 'quantity'] + expected_enriched_fields = {} + bigtable = EnrichWithBigTable( + self.project_id, self.instance_id, self.table_id, self.row_key) + # row_key which is product_id=11 doesn't exist, so the enriched field + # won't be added. Hence, the response is same as the request. + req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + + def test_enrichment_with_bigtable_bad_row_filter(self): + # in case of a bad column filter, that is, incorrect column_family_id and + # columns, no enrichment is done. If the column_family is correct but not + # column names then all columns in that column_family are returned. + expected_fields = [ + 'sale_id', + 'customer_id', + 'product_id', + 'quantity', + ] + expected_enriched_fields = {} + start_column = 'car_name'.encode() + column_filter = ColumnRangeFilter('car_name', start_column) + bigtable = EnrichWithBigTable( + self.project_id, + self.instance_id, + self.table_id, + self.row_key, + row_filter=column_filter) + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable) + | "Validate Response" >> beam.ParDo( + ValidateResponse( + len(expected_fields), + expected_fields, + expected_enriched_fields))) + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 5a7daeeab1c26..ccc5a95e12e07 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -17,6 +17,7 @@ import time import unittest from typing import List +from typing import NamedTuple from typing import Tuple from typing import Union @@ -34,20 +35,25 @@ from apache_beam.transforms.enrichment import EnrichmentSourceHandler +class Request(NamedTuple): + id: str + payload: bytes + + def _custom_join(left, right): """custom_join returns the id and resp_payload along with a timestamp""" right['timestamp'] = time.time() return beam.Row(**right) -class SampleHTTPEnrichment(EnrichmentSourceHandler[beam.Row, beam.Row]): +class SampleHTTPEnrichment(EnrichmentSourceHandler[Request, beam.Row]): """Implements ``EnrichmentSourceHandler`` to call the ``EchoServiceGrpc``'s HTTP handler. """ def __init__(self, url: str): self.url = url + '/v1/echo' # append path to the mock API. - def __call__(self, request: beam.Row, *args, **kwargs): + def __call__(self, request: Request, *args, **kwargs): """Overrides ``Caller``'s call method invoking the ``EchoServiceGrpc``'s HTTP handler with an `dict`, returning either a successful ``Tuple[dict,dict]`` or throwing either a @@ -120,7 +126,7 @@ def test_http_enrichment(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint with the default cross join.""" client, options = TestEnrichment._get_client_and_options() - req = beam.Row(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) fields = ['id', 'payload', 'resp_payload'] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( @@ -134,7 +140,7 @@ def test_http_enrichment_custom_join(self): """Tests Enrichment Transform against the Mock-API HTTP endpoint with a custom join function.""" client, options = TestEnrichment._get_client_and_options() - req = beam.Row(id=options.never_exceed_quota_id, payload=_PAYLOAD) + req = Request(id=options.never_exceed_quota_id, payload=_PAYLOAD) fields = ['id', 'resp_payload', 'timestamp'] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( From 87e32bb5d5a6f3bd52082f88cdf6bd9e3714d71e Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 15 Jan 2024 20:39:28 -0500 Subject: [PATCH 32/45] skip tests when dependencies are not installed --- .../apache_beam/io/requestresponse_it_test.py | 15 ++++++++++----- .../apache_beam/io/requestresponse_test.py | 15 ++++++++++----- .../transforms/enrichment_handlers/bigtable.py | 4 +++- .../enrichment_handlers/bigtable_it_test.py | 11 +++++++---- .../transforms/enrichment_it_test.py | 17 +++++++++++------ .../apache_beam/transforms/enrichment_test.py | 7 ++++++- 6 files changed, 47 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index e74c77edae0de..80a1ddabc1526 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -25,12 +25,17 @@ import urllib3 import apache_beam as beam -from apache_beam.io.requestresponse import Caller -from apache_beam.io.requestresponse import RequestResponseIO -from apache_beam.io.requestresponse import UserCodeExecutionException -from apache_beam.io.requestresponse import UserCodeQuotaException -from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.options.pipeline_options import PipelineOptions + +# pylint: disable=ungrouped-imports +try: + from apache_beam.io.requestresponse import Caller + from apache_beam.io.requestresponse import RequestResponseIO + from apache_beam.io.requestresponse import UserCodeExecutionException + from apache_beam.io.requestresponse import UserCodeQuotaException +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') _HTTP_PATH = '/v1/echo' _PAYLOAD = base64.b64encode(bytes('payload', 'utf-8')) diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index c7a2f27455e51..7618add61cf6b 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -20,13 +20,18 @@ from google.api_core.exceptions import TooManyRequests import apache_beam as beam -from apache_beam.io.requestresponse import Caller -from apache_beam.io.requestresponse import RequestResponseIO -from apache_beam.io.requestresponse import UserCodeExecutionException -from apache_beam.io.requestresponse import UserCodeTimeoutException -from apache_beam.io.requestresponse import retry_on_exception from apache_beam.testing.test_pipeline import TestPipeline +# pylint: disable=ungrouped-imports +try: + from apache_beam.io.requestresponse import Caller + from apache_beam.io.requestresponse import RequestResponseIO + from apache_beam.io.requestresponse import UserCodeExecutionException + from apache_beam.io.requestresponse import UserCodeTimeoutException + from apache_beam.io.requestresponse import retry_on_exception +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') + class AckCaller(Caller[str, str]): """AckCaller acknowledges the incoming request by returning a diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index b44f9535beb05..2fee18eac33ea 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -15,6 +15,8 @@ # limitations under the License. # import logging +from typing import Any +from typing import Dict from typing import Optional from google.api_core.exceptions import NotFound @@ -71,7 +73,7 @@ def __call__(self, request: beam.Row, *args, **kwargs): Args: request: the input `beam.Row` to enrich. """ - response_dict = {} + response_dict: Dict[str, Any] = {} try: request_dict = request._asdict() row_key = str(request_dict[self._row_key]).encode() diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index bf53ce1d6ab05..37320f276da6c 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -22,14 +22,17 @@ from typing import NamedTuple import pytest -from google.cloud.bigtable import Client -from google.cloud.bigtable.row_filters import ColumnRangeFilter import apache_beam as beam from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import BeamAssertException -from apache_beam.transforms.enrichment import Enrichment -from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable + +# pylint: disable=ungrouped-imports +try: + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable +except ImportError: + raise unittest.SkipTest('GCP BigTable dependencies are not installed.') class ValidateResponse(beam.DoFn): diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index ccc5a95e12e07..50099ec0ec89c 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -25,14 +25,19 @@ import urllib3 import apache_beam as beam -from apache_beam.io.requestresponse import UserCodeExecutionException -from apache_beam.io.requestresponse import UserCodeQuotaException -from apache_beam.io.requestresponse_it_test import _PAYLOAD -from apache_beam.io.requestresponse_it_test import EchoITOptions from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import BeamAssertException -from apache_beam.transforms.enrichment import Enrichment -from apache_beam.transforms.enrichment import EnrichmentSourceHandler + +# pylint: disable=ungrouped-imports +try: + from apache_beam.io.requestresponse import UserCodeExecutionException + from apache_beam.io.requestresponse import UserCodeQuotaException + from apache_beam.io.requestresponse_it_test import _PAYLOAD + from apache_beam.io.requestresponse_it_test import EchoITOptions + from apache_beam.transforms.enrichment import Enrichment + from apache_beam.transforms.enrichment import EnrichmentSourceHandler +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') class Request(NamedTuple): diff --git a/sdks/python/apache_beam/transforms/enrichment_test.py b/sdks/python/apache_beam/transforms/enrichment_test.py index 74545bb548a74..23b5f1828c15c 100644 --- a/sdks/python/apache_beam/transforms/enrichment_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_test.py @@ -19,7 +19,12 @@ import unittest import apache_beam as beam -from apache_beam.transforms.enrichment import cross_join + +# pylint: disable=ungrouped-imports +try: + from apache_beam.transforms.enrichment import cross_join +except ImportError: + raise unittest.SkipTest('RequestResponseIO dependencies are not installed.') class TestEnrichmentTransform(unittest.TestCase): From 57efa52be9ca1509442e5000679f0cdebd1426af Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Mon, 15 Jan 2024 21:36:36 -0500 Subject: [PATCH 33/45] add deleted imports from last commit --- .../transforms/enrichment_handlers/bigtable_it_test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 37320f276da6c..cfafa5b675e91 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -22,6 +22,8 @@ from typing import NamedTuple import pytest +from google.cloud.bigtable import Client +from google.cloud.bigtable.row_filters import ColumnRangeFilter import apache_beam as beam from apache_beam.testing.test_pipeline import TestPipeline @@ -125,6 +127,7 @@ def create_rows(table): @pytest.mark.it_postcommit +@unittest.skipIf() class TestBigTableEnrichment(unittest.TestCase): def setUp(self): self.project_id = 'apache-beam-testing' From 282c6081dc9876e5837c523079d6e85759bbdfa1 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 16 Jan 2024 09:50:49 -0500 Subject: [PATCH 34/45] add skip test condition --- .../transforms/enrichment_handlers/bigtable_it_test.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index cfafa5b675e91..2692c86e11402 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -22,8 +22,6 @@ from typing import NamedTuple import pytest -from google.cloud.bigtable import Client -from google.cloud.bigtable.row_filters import ColumnRangeFilter import apache_beam as beam from apache_beam.testing.test_pipeline import TestPipeline @@ -31,9 +29,12 @@ # pylint: disable=ungrouped-imports try: + from google.cloud.bigtable import Client + from google.cloud.bigtable.row_filters import ColumnRangeFilter from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable except ImportError: + Client = None raise unittest.SkipTest('GCP BigTable dependencies are not installed.') @@ -127,7 +128,10 @@ def create_rows(table): @pytest.mark.it_postcommit -@unittest.skipIf() +@unittest.skipIf( + Client is None, + 'BigTable dependencies are not installed, ' + 'skipping BigTable Enrichment test.') class TestBigTableEnrichment(unittest.TestCase): def setUp(self): self.project_id = 'apache-beam-testing' From deecdbc8f9b905f8ada8544a6b6a82ec7d13fa5f Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 16 Jan 2024 10:24:07 -0500 Subject: [PATCH 35/45] fix import order, add TooManyRequests to try-catch --- sdks/python/apache_beam/io/requestresponse_it_test.py | 2 +- sdks/python/apache_beam/io/requestresponse_test.py | 3 +-- .../transforms/enrichment_handlers/bigtable_it_test.py | 5 ----- 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse_it_test.py b/sdks/python/apache_beam/io/requestresponse_it_test.py index 80a1ddabc1526..396347c58d163 100644 --- a/sdks/python/apache_beam/io/requestresponse_it_test.py +++ b/sdks/python/apache_beam/io/requestresponse_it_test.py @@ -25,8 +25,8 @@ import urllib3 import apache_beam as beam -from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.testing.test_pipeline import TestPipeline # pylint: disable=ungrouped-imports try: diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index 7618add61cf6b..cacd0c22bbf54 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -17,13 +17,12 @@ import time import unittest -from google.api_core.exceptions import TooManyRequests - import apache_beam as beam from apache_beam.testing.test_pipeline import TestPipeline # pylint: disable=ungrouped-imports try: + from google.api_core.exceptions import TooManyRequests from apache_beam.io.requestresponse import Caller from apache_beam.io.requestresponse import RequestResponseIO from apache_beam.io.requestresponse import UserCodeExecutionException diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 2692c86e11402..5374eebcb00b9 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -34,7 +34,6 @@ from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable except ImportError: - Client = None raise unittest.SkipTest('GCP BigTable dependencies are not installed.') @@ -128,10 +127,6 @@ def create_rows(table): @pytest.mark.it_postcommit -@unittest.skipIf( - Client is None, - 'BigTable dependencies are not installed, ' - 'skipping BigTable Enrichment test.') class TestBigTableEnrichment(unittest.TestCase): def setUp(self): self.project_id = 'apache-beam-testing' From 253633e4b477a9d35dd11d68c28f8c615d026980 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 16 Jan 2024 15:19:54 -0500 Subject: [PATCH 36/45] make throttler, repeater non-optional --- sdks/python/apache_beam/io/requestresponse.py | 10 +++++----- sdks/python/apache_beam/transforms/enrichment.py | 10 +++++++++- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 6756d5cc5b82b..1b7200c26d666 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -150,7 +150,7 @@ def _execute_request( try: return future.result(timeout=timeout) except TooManyRequests as e: - _LOGGER.warning( + _LOGGER.info( 'request could not be completed. got code %i from the service.', e.code) raise e @@ -264,10 +264,10 @@ def __init__( caller: Caller[RequestT, ResponseT], timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, - repeater: Optional[Repeater] = ExponentialBackOffRepeater(), + repeater: Repeater = ExponentialBackOffRepeater(), cache_reader: Optional[CacheReader] = None, cache_writer: Optional[CacheWriter] = None, - throttler: Optional[PreCallThrottler] = DefaultThrottler(), + throttler: PreCallThrottler = DefaultThrottler(), ): """ Instantiates a RequestResponseIO transform. @@ -343,8 +343,8 @@ def __init__( caller: Caller[RequestT, ResponseT], timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, should_backoff: Optional[ShouldBackOff] = None, - repeater: Optional[Repeater] = None, - throttler: Optional[PreCallThrottler] = None, + repeater: Repeater = None, + throttler: PreCallThrottler = None, ): self._caller = caller self._timeout = timeout diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 1ac088328920b..c4056ddedc6d8 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # - +import logging from typing import Any from typing import Callable from typing import Dict @@ -37,6 +37,8 @@ JoinFn = Callable[[Dict[str, Any], Dict[str, Any]], beam.Row] +_LOGGER = logging.getLogger(__name__) + def cross_join(left: Dict[str, Any], right: Dict[str, Any]) -> beam.Row: """cross_join performs a cross join on two `dict` objects. @@ -54,6 +56,12 @@ def cross_join(left: Dict[str, Any], right: Dict[str, Any]) -> beam.Row: if k not in left: # Don't override the values in left. left[k] = v + elif left[k] != v: + _LOGGER.warning( + '%s exists in the input row as well the row fetched ' + 'from API but have different values. Using the input ' + 'value, you can override this behavior by passing a ' + 'custom `join_fn`.' % k) return beam.Row(**left) From 932fae3f91f9b11527af71324503ce31061b004b Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Tue, 16 Jan 2024 22:41:38 -0500 Subject: [PATCH 37/45] add exception level and tests --- .../apache_beam/transforms/enrichment.py | 7 +- .../enrichment_handlers/bigtable.py | 49 ++++++++- .../enrichment_handlers/bigtable_it_test.py | 102 +++++++++++++----- 3 files changed, 121 insertions(+), 37 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index c4056ddedc6d8..f373b6eb32f8c 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -59,9 +59,10 @@ def cross_join(left: Dict[str, Any], right: Dict[str, Any]) -> beam.Row: elif left[k] != v: _LOGGER.warning( '%s exists in the input row as well the row fetched ' - 'from API but have different values. Using the input ' - 'value, you can override this behavior by passing a ' - 'custom `join_fn`.' % k) + 'from API but have different values - %s and %s. Using the input ' + 'value (%s) for the enriched row. You can override this behavior by ' + 'passing a custom `join_fn` to Enrichment transform.' % + (k, left[k], v, left[k])) return beam.Row(**left) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 2fee18eac33ea..d2f5fb07920ae 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -15,6 +15,7 @@ # limitations under the License. # import logging +from enum import Enum from typing import Any from typing import Dict from typing import Optional @@ -28,11 +29,27 @@ __all__ = [ 'EnrichWithBigTable', + 'ExceptionLevel', ] _LOGGER = logging.getLogger(__name__) +class ExceptionLevel(Enum): + """ExceptionLevel defines the exception level options to either + log a warning, or raise an exception, or do nothing when a BigTable query + returns an empty row. + + Members: + - WARNING_ONLY: Log a warning for exception without raising it. + - RAISE: Raise the exception. + - QUIET: Neither log nor raise the exception. + """ + WARNING_ONLY = 0 + RAISE = 1 + QUIET = 2 + + class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): """EnrichWithBigTable is a handler for :class:`apache_beam.transforms.enrichment.Enrichment` transform to interact @@ -42,9 +59,14 @@ class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): project_id (str): GCP project-id of the BigTable cluster. instance_id (str): GCP instance-id of the BigTable cluster. table_id (str): GCP table-id of the BigTable. - row_key (str): unique row key for BigTable + row_key (str): unique row-key field name from the input `beam.Row` object + to use as `row_key` for BigTable querying. row_filter: a ``:class:`google.cloud.bigtable.row_filters.RowFilter``` to filter data read with ``read_row()``. + exception_level: a `enum.Enum` value from + ``apache_beam.transforms.enrichment_handlers.bigtable.ExceptionLevel`` + to set the level when an empty row is returned from the BigTable query. + Defaults to ``ExceptionLevel.QUIET``. """ def __init__( self, @@ -52,12 +74,15 @@ def __init__( instance_id: str, table_id: str, row_key: str, - row_filter: Optional[RowFilter] = None): + row_filter: Optional[RowFilter] = None, + exception_level: ExceptionLevel = ExceptionLevel.QUIET, + ): self._project_id = project_id self._instance_id = instance_id self._table_id = table_id self._row_key = row_key self._row_filter = row_filter + self._exception_level = exception_level def __enter__(self): """connect to the Google BigTable cluster.""" @@ -67,13 +92,14 @@ def __enter__(self): def __call__(self, request: beam.Row, *args, **kwargs): """ - Reads a row from the Google BigTable and returns + Reads a row from the GCP BigTable and returns a `Tuple` of request and response. Args: request: the input `beam.Row` to enrich. """ response_dict: Dict[str, Any] = {} + row_key: str = "" try: request_dict = request._asdict() row_key = str(request_dict[self._row_key]).encode() @@ -82,9 +108,22 @@ def __call__(self, request: beam.Row, *args, **kwargs): for cf_id, cf_v in row.cells.items(): response_dict[cf_id] = {} for k, v in cf_v.items(): - response_dict[cf_id][k.decode('utf-8')] = v[0].value.decode('utf-8') + response_dict[cf_id][k.decode('utf-8')] = \ + v[0].value.decode('utf-8') + elif self._exception_level == ExceptionLevel.WARNING_ONLY: + _LOGGER.warning( + 'no matching row found for row_key: %s ' + 'with row_filter: %s' % (row_key, self._row_filter)) + elif self._exception_level == ExceptionLevel.RAISE: + raise ValueError( + 'no matching row found for row_key: %s ' + 'with row_filter=%s' % (row_key, self._row_filter)) + except KeyError: + raise KeyError('row_key %s not found in input PCollection.' % row_key) except NotFound: - _LOGGER.warning('request row_key: %s not found') + raise NotFound( + 'GCP BigTable cluster `%s:%s:%s` not found.' % + (self._project_id, self._instance_id, self._table_id)) except Exception as e: raise e diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index 5374eebcb00b9..bd10e1b1ccaf8 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -29,10 +29,12 @@ # pylint: disable=ungrouped-imports try: + from google.api_core.exceptions import NotFound from google.cloud.bigtable import Client from google.cloud.bigtable.row_filters import ColumnRangeFilter from apache_beam.transforms.enrichment import Enrichment from apache_beam.transforms.enrichment_handlers.bigtable import EnrichWithBigTable + from apache_beam.transforms.enrichment_handlers.bigtable import ExceptionLevel except ImportError: raise unittest.SkipTest('GCP BigTable dependencies are not installed.') @@ -156,7 +158,10 @@ def test_enrichment_with_bigtable(self): 'product': ['product_id', 'product_name', 'product_stock'], } bigtable = EnrichWithBigTable( - self.project_id, self.instance_id, self.table_id, self.row_key) + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key) with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( test_pipeline @@ -178,10 +183,10 @@ def test_enrichment_with_bigtable_row_filter(self): start_column = 'product_name'.encode() column_filter = ColumnRangeFilter(self.column_family_id, start_column) bigtable = EnrichWithBigTable( - self.project_id, - self.instance_id, - self.table_id, - self.row_key, + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, row_filter=column_filter) with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( @@ -195,12 +200,15 @@ def test_enrichment_with_bigtable_row_filter(self): expected_enriched_fields))) def test_enrichment_with_bigtable_no_enrichment(self): + # row_key which is product_id=11 doesn't exist, so the enriched field + # won't be added. Hence, the response is same as the request. expected_fields = ['sale_id', 'customer_id', 'product_id', 'quantity'] expected_enriched_fields = {} bigtable = EnrichWithBigTable( - self.project_id, self.instance_id, self.table_id, self.row_key) - # row_key which is product_id=11 doesn't exist, so the enriched field - # won't be added. Hence, the response is same as the request. + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key) req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] with TestPipeline(is_integration_test=True) as test_pipeline: _ = ( @@ -217,31 +225,67 @@ def test_enrichment_with_bigtable_bad_row_filter(self): # in case of a bad column filter, that is, incorrect column_family_id and # columns, no enrichment is done. If the column_family is correct but not # column names then all columns in that column_family are returned. - expected_fields = [ - 'sale_id', - 'customer_id', - 'product_id', - 'quantity', - ] - expected_enriched_fields = {} start_column = 'car_name'.encode() column_filter = ColumnRangeFilter('car_name', start_column) bigtable = EnrichWithBigTable( - self.project_id, - self.instance_id, - self.table_id, - self.row_key, + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, row_filter=column_filter) - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ BigTable" >> Enrichment(bigtable) - | "Validate Response" >> beam.ParDo( - ValidateResponse( - len(expected_fields), - expected_fields, - expected_enriched_fields))) + with self.assertRaises(NotFound): + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + + def test_enrichment_with_bigtable_raises_key_error(self): + """raises a `KeyError` when the row_key doesn't exist in + the input PCollection.""" + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key='car_name') + with self.assertRaises(KeyError): + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + + def test_enrichment_with_bigtable_raises_not_found(self): + """raises a `NotFound` exception when the GCP BigTable Cluster + doesn't exist.""" + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id='invalid_table', + row_key=self.row_key) + with self.assertRaises(NotFound): + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + + def test_enrichment_with_bigtable_exception_level(self): + """raises a `NotFound` exception when the GCP BigTable Cluster + doesn't exist.""" + bigtable = EnrichWithBigTable( + project_id=self.project_id, + instance_id=self.instance_id, + table_id=self.table_id, + row_key=self.row_key, + exception_level=ExceptionLevel.RAISE) + req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] + with self.assertRaises(ValueError): + with TestPipeline(is_integration_test=True) as test_pipeline: + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) if __name__ == '__main__': From cf88d6fe7132a37716b4082bd551e6d6c83c8e6a Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 17 Jan 2024 09:21:25 -0500 Subject: [PATCH 38/45] correct pydoc statement --- .../transforms/enrichment_handlers/bigtable_it_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index bd10e1b1ccaf8..a3c97ab2bf97f 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -271,8 +271,8 @@ def test_enrichment_with_bigtable_raises_not_found(self): | "Enrich W/ BigTable" >> Enrichment(bigtable)) def test_enrichment_with_bigtable_exception_level(self): - """raises a `NotFound` exception when the GCP BigTable Cluster - doesn't exist.""" + """raises a `ValueError` exception when the GCP BigTable query returns + an empty row.""" bigtable = EnrichWithBigTable( project_id=self.project_id, instance_id=self.instance_id, From 5b702d2517308942c79e6fc0667bf7859dc2cf06 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 17 Jan 2024 11:41:51 -0500 Subject: [PATCH 39/45] add throttle tests --- sdks/python/apache_beam/io/requestresponse.py | 8 +++--- .../apache_beam/io/requestresponse_test.py | 25 ++++++++++++++++++- .../apache_beam/transforms/enrichment.py | 19 +++++++++++--- .../enrichment_handlers/bigtable.py | 23 +++++++++-------- 4 files changed, 56 insertions(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 1b7200c26d666..7effa31154c04 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -278,14 +278,14 @@ def __init__( timeout (float): timeout value in seconds to wait for response from API. should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponse.Repeater): (Optional) - provides methods to repeat requests to API. + repeater (~apache_beam.io.requestresponse.Repeater): provides methods to + repeat requests to API. cache_reader (~apache_beam.io.requestresponse.CacheReader): (Optional) provides methods to read external cache. cache_writer (~apache_beam.io.requestresponse.CacheWriter): (Optional) provides methods to write to external cache. throttler (~apache_beam.io.requestresponse.PreCallThrottler): - (Optional) provides methods to pre-throttle a request. + provides methods to pre-throttle a request. """ self._caller = caller self._timeout = timeout @@ -387,7 +387,7 @@ def process(self, request: RequestT, *args, **kwargs): _LOGGER.info( "Delaying request for %d seconds" % self._throttler.delay_secs) time.sleep(self._throttler.delay_secs) - self._metrics_collector.throttled_secs.inc(5) + self._metrics_collector.throttled_secs.inc(self._throttler.delay_secs) is_throttled_request = True if is_throttled_request: diff --git a/sdks/python/apache_beam/io/requestresponse_test.py b/sdks/python/apache_beam/io/requestresponse_test.py index cacd0c22bbf54..6d807c2a8eb83 100644 --- a/sdks/python/apache_beam/io/requestresponse_test.py +++ b/sdks/python/apache_beam/io/requestresponse_test.py @@ -23,7 +23,7 @@ # pylint: disable=ungrouped-imports try: from google.api_core.exceptions import TooManyRequests - from apache_beam.io.requestresponse import Caller + from apache_beam.io.requestresponse import Caller, DefaultThrottler from apache_beam.io.requestresponse import RequestResponseIO from apache_beam.io.requestresponse import UserCodeExecutionException from apache_beam.io.requestresponse import UserCodeTimeoutException @@ -128,6 +128,29 @@ def test_caller_no_retry_strategy(self): | RequestResponseIO(caller=caller, repeater=None)) self.assertRegex(cm.exception.message, 'retries = 0') + def test_default_throttler(self): + caller = CallerWithTimeout() + throttler = DefaultThrottler( + window_ms=10000, bucket_ms=5000, overload_ratio=1) + # manually override the number of received requests for testing. + throttler.throttler._all_requests.add(time.time() * 1000, 100) + test_pipeline = TestPipeline() + _ = ( + test_pipeline + | beam.Create(['sample_request']) + | RequestResponseIO(caller=caller, throttler=throttler)) + result = test_pipeline.run() + result.wait_until_finish() + metrics = result.metrics().query( + beam.metrics.MetricsFilter().with_name('throttled_requests')) + self.assertEqual(metrics['counters'][0].committed, 1) + metrics = result.metrics().query( + beam.metrics.MetricsFilter().with_name('cumulativeThrottlingSeconds')) + self.assertGreater(metrics['counters'][0].committed, 0) + metrics = result.metrics().query( + beam.metrics.MetricsFilter().with_name('responses')) + self.assertEqual(metrics['counters'][0].committed, 1) + if __name__ == '__main__': unittest.main() diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index f373b6eb32f8c..b5b5e6adaf33f 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -22,7 +22,8 @@ from typing import TypeVar import apache_beam as beam -from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS +from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS, ExponentialBackOffRepeater, Repeater, DefaultThrottler, \ + PreCallThrottler from apache_beam.io.requestresponse import Caller from apache_beam.io.requestresponse import RequestResponseIO @@ -96,20 +97,32 @@ class Enrichment(beam.PTransform[beam.PCollection[InputT], join_fn: A lambda function to join original element with lookup metadata. Defaults to `CROSS_JOIN`. timeout: (Optional) timeout for source requests. Defaults to 30 seconds. + repeater (~apache_beam.io.requestresponse.Repeater): provides methods to + repeat requests to API. + throttler (~apache_beam.io.requestresponse.PreCallThrottler): + provides methods to pre-throttle a request. """ def __init__( self, source_handler: EnrichmentSourceHandler, join_fn: JoinFn = cross_join, - timeout: Optional[float] = DEFAULT_TIMEOUT_SECS): + timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, + repeater: Repeater = ExponentialBackOffRepeater, + throttler: PreCallThrottler = DefaultThrottler(), + ): self._source_handler = source_handler self._join_fn = join_fn self._timeout = timeout + self._repeater = repeater + self._throttler = throttler def expand(self, input_row: beam.PCollection[InputT]) -> beam.PCollection[OutputT]: fetched_data = input_row | RequestResponseIO( - caller=self._source_handler, timeout=self._timeout) + caller=self._source_handler, + timeout=self._timeout, + repeater=self._repeater, + throttler=self._throttler) # EnrichmentSourceHandler returns a tuple of (request,response). return fetched_data | beam.Map( diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index d2f5fb07920ae..5883db111a496 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -41,12 +41,12 @@ class ExceptionLevel(Enum): returns an empty row. Members: - - WARNING_ONLY: Log a warning for exception without raising it. - RAISE: Raise the exception. + - WARN: Log a warning for exception without raising it. - QUIET: Neither log nor raise the exception. """ - WARNING_ONLY = 0 - RAISE = 1 + RAISE = 0 + WARN = 1 QUIET = 2 @@ -66,7 +66,7 @@ class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): exception_level: a `enum.Enum` value from ``apache_beam.transforms.enrichment_handlers.bigtable.ExceptionLevel`` to set the level when an empty row is returned from the BigTable query. - Defaults to ``ExceptionLevel.QUIET``. + Defaults to ``ExceptionLevel.WARN``. """ def __init__( self, @@ -75,7 +75,7 @@ def __init__( table_id: str, row_key: str, row_filter: Optional[RowFilter] = None, - exception_level: ExceptionLevel = ExceptionLevel.QUIET, + exception_level: ExceptionLevel = ExceptionLevel.WARN, ): self._project_id = project_id self._instance_id = instance_id @@ -99,10 +99,11 @@ def __call__(self, request: beam.Row, *args, **kwargs): request: the input `beam.Row` to enrich. """ response_dict: Dict[str, Any] = {} - row_key: str = "" + row_key_str: str = "" try: request_dict = request._asdict() - row_key = str(request_dict[self._row_key]).encode() + row_key_str = str(request_dict[self._row_key]) + row_key = row_key_str.encode() row = self._table.read_row(row_key, filter_=self._row_filter) if row: for cf_id, cf_v in row.cells.items(): @@ -110,16 +111,16 @@ def __call__(self, request: beam.Row, *args, **kwargs): for k, v in cf_v.items(): response_dict[cf_id][k.decode('utf-8')] = \ v[0].value.decode('utf-8') - elif self._exception_level == ExceptionLevel.WARNING_ONLY: + elif self._exception_level == ExceptionLevel.WARN: _LOGGER.warning( 'no matching row found for row_key: %s ' - 'with row_filter: %s' % (row_key, self._row_filter)) + 'with row_filter: %s' % (row_key_str, self._row_filter)) elif self._exception_level == ExceptionLevel.RAISE: raise ValueError( 'no matching row found for row_key: %s ' - 'with row_filter=%s' % (row_key, self._row_filter)) + 'with row_filter=%s' % (row_key_str, self._row_filter)) except KeyError: - raise KeyError('row_key %s not found in input PCollection.' % row_key) + raise KeyError('row_key %s not found in input PCollection.' % row_key_str) except NotFound: raise NotFound( 'GCP BigTable cluster `%s:%s:%s` not found.' % From 18d95397f886cbac933cf2d89af7d3b037a29551 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Wed, 17 Jan 2024 21:59:54 -0500 Subject: [PATCH 40/45] add bigtable improvements --- .../apache_beam/transforms/enrichment.py | 9 +++++--- .../enrichment_handlers/bigtable.py | 23 +++++++++++++++---- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index b5b5e6adaf33f..0e1d4eea70e9c 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -22,9 +22,12 @@ from typing import TypeVar import apache_beam as beam -from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS, ExponentialBackOffRepeater, Repeater, DefaultThrottler, \ - PreCallThrottler +from apache_beam.io.requestresponse import DEFAULT_TIMEOUT_SECS from apache_beam.io.requestresponse import Caller +from apache_beam.io.requestresponse import DefaultThrottler +from apache_beam.io.requestresponse import ExponentialBackOffRepeater +from apache_beam.io.requestresponse import PreCallThrottler +from apache_beam.io.requestresponse import Repeater from apache_beam.io.requestresponse import RequestResponseIO __all__ = [ @@ -107,7 +110,7 @@ def __init__( source_handler: EnrichmentSourceHandler, join_fn: JoinFn = cross_join, timeout: Optional[float] = DEFAULT_TIMEOUT_SECS, - repeater: Repeater = ExponentialBackOffRepeater, + repeater: Repeater = ExponentialBackOffRepeater(), throttler: PreCallThrottler = DefaultThrottler(), ): self._source_handler = source_handler diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 5883db111a496..eb4843e96c261 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -21,7 +21,9 @@ from typing import Optional from google.api_core.exceptions import NotFound +from google.cloud import bigtable from google.cloud.bigtable import Client +from google.cloud.bigtable.row_filters import CellsColumnLimitFilter from google.cloud.bigtable.row_filters import RowFilter import apache_beam as beam @@ -63,6 +65,10 @@ class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): to use as `row_key` for BigTable querying. row_filter: a ``:class:`google.cloud.bigtable.row_filters.RowFilter``` to filter data read with ``read_row()``. + Defaults to `CellsColumnLimitFilter(1)`. + app_profile_id (str): App profile ID to use for BigTable. + encoding (str): encoding type to convert the string to bytes and vice-versa + from BigTable. Default is `utf-8`. exception_level: a `enum.Enum` value from ``apache_beam.transforms.enrichment_handlers.bigtable.ExceptionLevel`` to set the level when an empty row is returned from the BigTable query. @@ -74,7 +80,9 @@ def __init__( instance_id: str, table_id: str, row_key: str, - row_filter: Optional[RowFilter] = None, + row_filter: Optional[RowFilter] = CellsColumnLimitFilter(1), + app_profile_id: str = None, + encoding: str = 'utf-8', exception_level: ExceptionLevel = ExceptionLevel.WARN, ): self._project_id = project_id @@ -82,13 +90,18 @@ def __init__( self._table_id = table_id self._row_key = row_key self._row_filter = row_filter + self._app_profile_id = app_profile_id + self._encoding = encoding self._exception_level = exception_level def __enter__(self): """connect to the Google BigTable cluster.""" self.client = Client(project=self._project_id) self.instance = self.client.instance(self._instance_id) - self._table = self.instance.table(self._table_id) + self._table = bigtable.table.Table( + table_id=self._table_id, + instance=self.instance, + app_profile_id=self._app_profile_id) def __call__(self, request: beam.Row, *args, **kwargs): """ @@ -103,14 +116,14 @@ def __call__(self, request: beam.Row, *args, **kwargs): try: request_dict = request._asdict() row_key_str = str(request_dict[self._row_key]) - row_key = row_key_str.encode() + row_key = row_key_str.encode(self._encoding) row = self._table.read_row(row_key, filter_=self._row_filter) if row: for cf_id, cf_v in row.cells.items(): response_dict[cf_id] = {} for k, v in cf_v.items(): - response_dict[cf_id][k.decode('utf-8')] = \ - v[0].value.decode('utf-8') + response_dict[cf_id][k.decode(self._encoding)] = \ + v[0].value.decode(self._encoding) elif self._exception_level == ExceptionLevel.WARN: _LOGGER.warning( 'no matching row found for row_key: %s ' From 6e251ced252f51543356fcb546a5432e2844aeb4 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 18 Jan 2024 09:28:11 -0500 Subject: [PATCH 41/45] default app_profile_id --- .../apache_beam/transforms/enrichment_handlers/bigtable.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index eb4843e96c261..03dd7568197c8 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -81,7 +81,7 @@ def __init__( table_id: str, row_key: str, row_filter: Optional[RowFilter] = CellsColumnLimitFilter(1), - app_profile_id: str = None, + app_profile_id: str = "", encoding: str = 'utf-8', exception_level: ExceptionLevel = ExceptionLevel.WARN, ): From 20b8ba6d1b8db7a781f4fa21f712fd5ed09b71ef Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 18 Jan 2024 11:15:24 -0500 Subject: [PATCH 42/45] add documentation, ignore None assignment --- sdks/python/apache_beam/io/requestresponse.py | 11 ++++++++--- sdks/python/apache_beam/transforms/enrichment.py | 11 ++++++++--- .../transforms/enrichment_handlers/bigtable.py | 3 ++- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/io/requestresponse.py b/sdks/python/apache_beam/io/requestresponse.py index 7effa31154c04..63ec7061d3e5a 100644 --- a/sdks/python/apache_beam/io/requestresponse.py +++ b/sdks/python/apache_beam/io/requestresponse.py @@ -278,14 +278,19 @@ def __init__( timeout (float): timeout value in seconds to wait for response from API. should_backoff (~apache_beam.io.requestresponse.ShouldBackOff): (Optional) provides methods for backoff. - repeater (~apache_beam.io.requestresponse.Repeater): provides methods to - repeat requests to API. + repeater (~apache_beam.io.requestresponse.Repeater): provides method to + repeat failed requests to API due to service errors. Defaults to + :class:`apache_beam.io.requestresponse.ExponentialBackOffRepeater` to + repeat requests with exponential backoff. cache_reader (~apache_beam.io.requestresponse.CacheReader): (Optional) provides methods to read external cache. cache_writer (~apache_beam.io.requestresponse.CacheWriter): (Optional) provides methods to write to external cache. throttler (~apache_beam.io.requestresponse.PreCallThrottler): - provides methods to pre-throttle a request. + provides methods to pre-throttle a request. Defaults to + :class:`apache_beam.io.requestresponse.DefaultThrottler` for + client-side adaptive throttling using + :class:`apache_beam.io.components.adaptive_throttler.AdaptiveThrottler` """ self._caller = caller self._timeout = timeout diff --git a/sdks/python/apache_beam/transforms/enrichment.py b/sdks/python/apache_beam/transforms/enrichment.py index 0e1d4eea70e9c..a2f961be64373 100644 --- a/sdks/python/apache_beam/transforms/enrichment.py +++ b/sdks/python/apache_beam/transforms/enrichment.py @@ -100,10 +100,15 @@ class Enrichment(beam.PTransform[beam.PCollection[InputT], join_fn: A lambda function to join original element with lookup metadata. Defaults to `CROSS_JOIN`. timeout: (Optional) timeout for source requests. Defaults to 30 seconds. - repeater (~apache_beam.io.requestresponse.Repeater): provides methods to - repeat requests to API. + repeater (~apache_beam.io.requestresponse.Repeater): provides method to + repeat failed requests to API due to service errors. Defaults to + :class:`apache_beam.io.requestresponse.ExponentialBackOffRepeater` to + repeat requests with exponential backoff. throttler (~apache_beam.io.requestresponse.PreCallThrottler): - provides methods to pre-throttle a request. + provides methods to pre-throttle a request. Defaults to + :class:`apache_beam.io.requestresponse.DefaultThrottler` for + client-side adaptive throttling using + :class:`apache_beam.io.components.adaptive_throttler.AdaptiveThrottler`. """ def __init__( self, diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py index 03dd7568197c8..86ff2f3b8e7f6 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable.py @@ -67,6 +67,7 @@ class EnrichWithBigTable(EnrichmentSourceHandler[beam.Row, beam.Row]): filter data read with ``read_row()``. Defaults to `CellsColumnLimitFilter(1)`. app_profile_id (str): App profile ID to use for BigTable. + See https://cloud.google.com/bigtable/docs/app-profiles for more details. encoding (str): encoding type to convert the string to bytes and vice-versa from BigTable. Default is `utf-8`. exception_level: a `enum.Enum` value from @@ -81,7 +82,7 @@ def __init__( table_id: str, row_key: str, row_filter: Optional[RowFilter] = CellsColumnLimitFilter(1), - app_profile_id: str = "", + app_profile_id: str = None, # type: ignore[assignment] encoding: str = 'utf-8', exception_level: ExceptionLevel = ExceptionLevel.WARN, ): From 27974b8cfeec621745949dfbb250219ac4d17a5b Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 18 Jan 2024 11:20:01 -0500 Subject: [PATCH 43/45] add to changes.md --- CHANGES.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index b791dd806d4a1..ed40ffcb04af3 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -66,7 +66,7 @@ ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* [Enrichment Transform](https://s.apache.org/enrichment-transform) along with GCP BigTable handler added to Python SDK ([#30001](https://github.com/apache/beam/pull/30001)). ## Breaking Changes From 7c9a03ca6e87eb1a045bed325936b9a1a664ed9e Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 18 Jan 2024 14:40:43 -0500 Subject: [PATCH 44/45] change test structure that throws exception, skip http test for now --- .../enrichment_handlers/bigtable_it_test.py | 48 +++++++++++-------- .../transforms/enrichment_it_test.py | 4 +- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py index a3c97ab2bf97f..dd48c8e5ef4d0 100644 --- a/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_handlers/bigtable_it_test.py @@ -234,11 +234,13 @@ def test_enrichment_with_bigtable_bad_row_filter(self): row_key=self.row_key, row_filter=column_filter) with self.assertRaises(NotFound): - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ BigTable" >> Enrichment(bigtable)) + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() def test_enrichment_with_bigtable_raises_key_error(self): """raises a `KeyError` when the row_key doesn't exist in @@ -249,11 +251,13 @@ def test_enrichment_with_bigtable_raises_key_error(self): table_id=self.table_id, row_key='car_name') with self.assertRaises(KeyError): - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ BigTable" >> Enrichment(bigtable)) + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() def test_enrichment_with_bigtable_raises_not_found(self): """raises a `NotFound` exception when the GCP BigTable Cluster @@ -264,11 +268,13 @@ def test_enrichment_with_bigtable_raises_not_found(self): table_id='invalid_table', row_key=self.row_key) with self.assertRaises(NotFound): - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(self.req) - | "Enrich W/ BigTable" >> Enrichment(bigtable)) + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(self.req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() def test_enrichment_with_bigtable_exception_level(self): """raises a `ValueError` exception when the GCP BigTable query returns @@ -281,11 +287,13 @@ def test_enrichment_with_bigtable_exception_level(self): exception_level=ExceptionLevel.RAISE) req = [beam.Row(sale_id=1, customer_id=1, product_id=11, quantity=1)] with self.assertRaises(ValueError): - with TestPipeline(is_integration_test=True) as test_pipeline: - _ = ( - test_pipeline - | "Create" >> beam.Create(req) - | "Enrich W/ BigTable" >> Enrichment(bigtable)) + test_pipeline = beam.Pipeline() + _ = ( + test_pipeline + | "Create" >> beam.Create(req) + | "Enrich W/ BigTable" >> Enrichment(bigtable)) + res = test_pipeline.run() + res.wait_until_finish() if __name__ == '__main__': diff --git a/sdks/python/apache_beam/transforms/enrichment_it_test.py b/sdks/python/apache_beam/transforms/enrichment_it_test.py index 50099ec0ec89c..89842cb18be02 100644 --- a/sdks/python/apache_beam/transforms/enrichment_it_test.py +++ b/sdks/python/apache_beam/transforms/enrichment_it_test.py @@ -117,7 +117,9 @@ class TestEnrichment(unittest.TestCase): @classmethod def setUpClass(cls) -> None: cls.options = EchoITOptions() - http_endpoint_address = 'http://10.138.0.32:8080' # endpoint of mock api + http_endpoint_address = cls.options.http_endpoint_address + if not http_endpoint_address or http_endpoint_address == '': + raise unittest.SkipTest('HTTP_ENDPOINT_ADDRESS is required.') cls.client = SampleHTTPEnrichment(http_endpoint_address) @classmethod From 5a626e317555b3c56cd2becd7e617c5bace71a71 Mon Sep 17 00:00:00 2001 From: riteshghorse Date: Thu, 18 Jan 2024 17:14:35 -0500 Subject: [PATCH 45/45] drop postcommit trigger file --- .github/trigger_files/beam_PostCommit_Python.json | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 .github/trigger_files/beam_PostCommit_Python.json diff --git a/.github/trigger_files/beam_PostCommit_Python.json b/.github/trigger_files/beam_PostCommit_Python.json deleted file mode 100644 index e69de29bb2d1d..0000000000000