| { |
| "cells": [ |
| { |
| "cell_type": "code", |
| "execution_count": 1, |
| "metadata": { |
| "cellView": "form", |
| "id": "paYiulysGrwR" |
| }, |
| "outputs": [], |
| "source": [ |
| "# @title ###### Licensed to the Apache Software Foundation (ASF), Version 2.0 (the \"License\")\n", |
| "\n", |
| "# Licensed to the Apache Software Foundation (ASF) under one\n", |
| "# or more contributor license agreements. See the NOTICE file\n", |
| "# distributed with this work for additional information\n", |
| "# regarding copyright ownership. The ASF licenses this file\n", |
| "# to you under the Apache License, Version 2.0 (the\n", |
| "# \"License\"); you may not use this file except in compliance\n", |
| "# with the License. You may obtain a copy of the License at\n", |
| "#\n", |
| "# http://www.apache.org/licenses/LICENSE-2.0\n", |
| "#\n", |
| "# Unless required by applicable law or agreed to in writing,\n", |
| "# software distributed under the License is distributed on an\n", |
| "# \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n", |
| "# KIND, either express or implied. See the License for the\n", |
| "# specific language governing permissions and limitations\n", |
| "# under the License" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "0UGzzndTBPWQ" |
| }, |
| "source": [ |
| "# Remote inference in Apache Beam\n", |
| "\n", |
| "<table align=\"left\">\n", |
| " <td>\n", |
| " <a target=\"_blank\" href=\"https://colab.research.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb\"><img src=\"https://raw.githubusercontent.com/google/or-tools/main/tools/colab_32px.png\" />Run in Google Colab</a>\n", |
| " </td>\n", |
| " <td>\n", |
| " <a target=\"_blank\" href=\"https://github.com/apache/beam/blob/master/examples/notebooks/beam-ml/custom_remote_inference.ipynb\"><img src=\"https://raw.githubusercontent.com/google/or-tools/main/tools/github_32px.png\" />View source on GitHub</a>\n", |
| " </td>\n", |
| "</table>\n" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "GNbarEZsalS2" |
| }, |
| "source": [ |
| "This example demonstrates how to implement a custom inference call in Apache Beam by using the Google Cloud Vision API.\n", |
| "\n", |
| "The prefered way to run inference in Apache Beam is by using the [RunInference API](https://beam.apache.org/documentation/sdks/python-machine-learning/).\n", |
| "The RunInference API enables you to run models as part of your pipeline in a way that is optimized for machine learning inference.\n", |
| "To reduce the number of steps in your pipeline, RunInference supports features like batching. For more infomation about the RunInference API, review the [RunInference API](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.html#apache_beam.ml.inference.RunInference).\n", |
| "\n", |
| "This notebook creates a custom model handler to make remote inference calls by using the Cloud Vision API. To make remote inference calls to Vertex AI, use the [Vertex AI model handler JSON](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/ml/inference/vertex_ai_inference.py).\n", |
| "\n", |
| "**Note:** All images are licensed CC-BY. Creators are listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "GNbarEZsalS1" |
| }, |
| "source": [ |
| "## Run the Cloud Vision API\n", |
| "\n", |
| "You can use the Cloud Vision API to retrieve labels that describe an image.\n", |
| "For example, the following image shows a cat with possible labels." |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "q-jVQn3maZ81" |
| }, |
| "source": [ |
| "" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "4io1vzkzF683" |
| }, |
| "source": [ |
| "To run the Google Cloud Vision API on a large set of images, Apache Beam is the ideal tool to handle the workflow.\n", |
| "This example demonstates how to retrieve image labels with this API on a small set of images.\n", |
| "\n", |
| "The example follows these steps:\n", |
| "* Read the images.\n", |
| "* Send the images to an external API to run inference by using the `RunInference PTransform`.\n", |
| "* Postprocess the results of your API.\n", |
| "\n", |
| "**Caution:** Be aware of API quotas and the heavy load you might incur on your external API. Verify that your pipeline and API are configured correctly for your use case.\n", |
| "\n", |
| "To optimize the calls to the external API, limit the parallel calls to the external remote API by [configuring pipeline options](https://beam.apache.org/documentation/programming-guide/#configuring-pipeline-options).\n", |
| "In Apache Beam, each runner provides options to handle the parallelism. The following list includes two examples:\n", |
| "* With the [Direct Runner](https://beam.apache.org/documentation/runners/direct/), use the `direct_num_workers` pipeline option.\n", |
| "* With the [Google Cloud Dataflow Runner](https://beam.apache.org/documentation/runners/dataflow/), use the `max_num_workers` pipeline option.\n", |
| "\n", |
| "For information about other runners, see the [Beam capability matrix](https://beam.apache.org/documentation/runners/capability-matrix/)" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "FAawWOaiIYaS" |
| }, |
| "source": [ |
| "## Before you begin" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "XhpKOxINrIqz" |
| }, |
| "source": [ |
| "Download and install the dependencies." |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": null, |
| "metadata": { |
| "id": "bA7MLR8OptJw" |
| }, |
| "outputs": [], |
| "source": [ |
| "!pip install --upgrade pip\n", |
| "!pip install protobuf==3.19.4\n", |
| "!pip install apache-beam[interactive,gcp]>=2.65.0\n", |
| "!pip install google-cloud-vision==3.1.1\n", |
| "!pip install requests\n", |
| "\n", |
| "# To use the newly installed version, restart the runtime.\n", |
| "exit()" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "C-RVR2eprc0r" |
| }, |
| "source": [ |
| "To use the Cloud Vision API, authenticate with Google Cloud." |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": null, |
| "metadata": { |
| "id": "qGDJCbxgTprh" |
| }, |
| "outputs": [], |
| "source": [ |
| "# Follow the steps to configure your Google Cloup setup.\n", |
| "!gcloud init" |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": null, |
| "metadata": { |
| "id": "74acX7AlT91N" |
| }, |
| "outputs": [], |
| "source": [ |
| "!gcloud auth application-default login" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "mL4MaHm_XOVd" |
| }, |
| "source": [ |
| "## Run remote inference on Cloud Vision API\n", |
| "\n", |
| "This section shows how to run remote inference on the Cloud Vision API.\n", |
| "\n", |
| "Download and install Apache Beam and the required modules." |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": 5, |
| "metadata": { |
| "id": "gE0go8CpnTy3" |
| }, |
| "outputs": [], |
| "source": [ |
| "import io\n", |
| "import requests\n", |
| "\n", |
| "from google.cloud import vision\n", |
| "from google.cloud.vision_v1.types import Feature\n", |
| "import apache_beam as beam\n", |
| "from apache_beam.ml.inference.base import RemoteModelHandler\n", |
| "from apache_beam.ml.inference.base import RunInference\n" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "09k08IYlLmON" |
| }, |
| "source": [ |
| "This example uses images from the [MSCoco dataset](https://cocodataset.org/#explore) as a list of image URLs.\n", |
| "This data is used as the pipeline input." |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": 6, |
| "metadata": { |
| "id": "_89eN_1QeYEd" |
| }, |
| "outputs": [], |
| "source": [ |
| "image_urls = [\n", |
| " \"http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg\",\n", |
| " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", |
| " \"http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg\",\n", |
| " \"http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg\",\n", |
| " \"http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg\",\n", |
| " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", |
| " \"http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg\",\n", |
| "]\n", |
| "\n", |
| "def read_image(image_url):\n", |
| " \"\"\"Read image from url and return image_url, image bytes\"\"\"\n", |
| " response = requests.get(image_url)\n", |
| " image_bytes = io.BytesIO(response.content).read()\n", |
| " return image_url, image_bytes" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "HLy7VKJhLrmT" |
| }, |
| "source": [ |
| "### Create a custom model handler\n", |
| "\n", |
| "In order to implement remote inference, create a custom model handler. Use the `run_inference` method to implement the model call and to return its results.\n", |
| "\n", |
| "When you run remote inference, prepare to encounter, identify, and handle failure as gracefully as possible. We recommend using the following techniques:\n", |
| "\n", |
| "* **Exponential backoff:** Retry failed remote calls with exponentially growing pauses between retries. Using exponential backoff ensures that failures don't lead to an overwhelming number of retries in quick succession. The `RemoteModelHandler` base class handles this logic, with the `retry_fn` argument determining which errors are retryable. For this example we will always retry. \n", |
| "\n", |
| "* **Dead-letter queues:** Route failed inferences to a separate `PCollection` without failing the whole transform. Continue execution without failing the job (batch jobs' default behavior) or retrying indefinitely (streaming jobs' default behavior). This is provided through the `with_exception_handling()` [option](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.RunInference.with_exception_handling) for RunInference. This produces tagged outputs for the failed inferences which can be handled separately from successful ones. You can then run custom pipeline logic on the dead-letter (unprocessed messages) queue to log the failure, send an alert, and push the failed message to temporary storage so that it can eventually be reprocessed." |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": 7, |
| "metadata": { |
| "id": "LnaisJ_JiY_Q" |
| }, |
| "outputs": [], |
| "source": [ |
| "def _always_retry(exception: Exception) -> bool:\n", |
| " return True\n", |
| "\n", |
| "class CloudVisionModelHandler(RemoteModelHandler):\n", |
| " def __init__(self):\n", |
| " \"\"\"DoFn that accepts a batch of images as bytearray\n", |
| " and sends that batch to the Cloud Vision API for remote inference\n", |
| " \"\"\"\n", |
| " super().__init__(namespace=\"CloudVisionModelHandler\", retry_filter=_always_retry)\n", |
| " def create_client(self):\n", |
| " \"\"\"Initiate the Google Vision API client.\"\"\"\n", |
| " client = vision.ImageAnnotatorClient()\n", |
| " return client\n", |
| "\n", |
| " def request(self, batch, model, inference_args):\n", |
| " feature = Feature()\n", |
| " feature.type_ = Feature.Type.LABEL_DETECTION\n", |
| "\n", |
| " # The list of image_urls\n", |
| " image_urls = [image_url for (image_url, image_bytes) in batch]\n", |
| "\n", |
| " # Create a batch request for all images in the batch.\n", |
| " images = [vision.Image(content=image_bytes) for (image_url, image_bytes) in batch]\n", |
| " image_requests = [vision.AnnotateImageRequest(image=image, features=[feature]) for image in images]\n", |
| " batch_image_request = vision.BatchAnnotateImagesRequest(requests=image_requests)\n", |
| "\n", |
| " # Send the batch request to the remote endpoint.\n", |
| " responses = model.batch_annotate_images(request=batch_image_request).responses\n", |
| "\n", |
| " return list(zip(image_urls, responses))\n" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "lHJuyHhvL0-a" |
| }, |
| "source": [ |
| "### Manage batching\n", |
| "\n", |
| "When you run inference with your model, either in Apache Beam or in an external API, batch your input to increase the efficiency of the model execution.\n", |
| "The `RunInference PTransform` automatically manages batching by using the `BatchElements` transform to dynamically group elements together into batches based on the throughput of the pipeline.\n", |
| "\n", |
| "If you are designing your own API endpoint, make sure that it can handle batches.\n", |
| "\n" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "4sXHwZk9Url2" |
| }, |
| "source": [ |
| "### Create the pipeline\n", |
| "\n", |
| "This section demonstrates how to chain the pipeline steps together to complete the following tasks:\n", |
| "\n", |
| "* Read data.\n", |
| "\n", |
| "* Transform the data to fit the model input.\n", |
| "\n", |
| "* Run inference with a custom Cloud Vision model handler.\n", |
| "\n", |
| "* Process and display the results." |
| ] |
| }, |
| { |
| "cell_type": "code", |
| "execution_count": 9, |
| "metadata": { |
| "colab": { |
| "base_uri": "https://localhost:8080/" |
| }, |
| "id": "LLg0OTvNkqo4", |
| "outputId": "3eee9ae4-f4cb-49e5-e03b-9af6aaef8805" |
| }, |
| "outputs": [ |
| { |
| "name": "stdout", |
| "output_type": "stream", |
| "text": [ |
| "('http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg', label_annotations {\n", |
| " mid: \"/m/04_sv\"\n", |
| " description: \"Motorcycle\"\n", |
| " score: 0.9922548\n", |
| " topicality: 0.14033242\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/01prls\"\n", |
| " description: \"Land vehicle\"\n", |
| " score: 0.99086833\n", |
| " topicality: 0.0029524593\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0768fx\"\n", |
| " description: \"Automotive lighting\"\n", |
| " score: 0.9853215\n", |
| " topicality: 0.002913047\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/07yv9\"\n", |
| " description: \"Vehicle\"\n", |
| " score: 0.98517245\n", |
| " topicality: 0.010408105\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/043g5f\"\n", |
| " description: \"Fuel tank\"\n", |
| " score: 0.9823826\n", |
| " topicality: 0.01933147\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/012f08\"\n", |
| " description: \"Motor vehicle\"\n", |
| " score: 0.97732854\n", |
| " topicality: 0.0009314301\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0h9mv\"\n", |
| " description: \"Tire\"\n", |
| " score: 0.9735299\n", |
| " topicality: 0.0020883244\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/083wq\"\n", |
| " description: \"Wheel\"\n", |
| " score: 0.9715105\n", |
| " topicality: 0.0028435893\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0h8pb3l\"\n", |
| " description: \"Automotive Tire\"\n", |
| " score: 0.96993804\n", |
| " topicality: 5.827098e-05\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0h8ls87\"\n", |
| " description: \"Automotive Exterior\"\n", |
| " score: 0.9641536\n", |
| " topicality: 0.00045098987\n", |
| "}\n", |
| ")\n", |
| "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", |
| " mid: \"/m/02w3_ws\"\n", |
| " description: \"Personal care\"\n", |
| " score: 0.853392\n", |
| " topicality: 0.00013828959\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/02pkr5\"\n", |
| " description: \"Plumbing fixture\"\n", |
| " score: 0.8383083\n", |
| " topicality: 0.012253191\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0b_zf\"\n", |
| " description: \"Plumbing\"\n", |
| " score: 0.726803\n", |
| " topicality: 0.016276756\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/01j2bj\"\n", |
| " description: \"Bathroom\"\n", |
| " score: 0.72486097\n", |
| " topicality: 0.35419264\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/02jz0l\"\n", |
| " description: \"Tap\"\n", |
| " score: 0.6317307\n", |
| " topicality: 0.00705197\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0130jx\"\n", |
| " description: \"Sink\"\n", |
| " score: 0.5732167\n", |
| " topicality: 0.07520393\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/054_l\"\n", |
| " description: \"Mirror\"\n", |
| " score: 0.5680867\n", |
| " topicality: 0.08497098\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0h8lr5r\"\n", |
| " description: \"Bathroom Sink\"\n", |
| " score: 0.557554\n", |
| " topicality: 0.007725588\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/03jvk\"\n", |
| " description: \"Household hardware\"\n", |
| " score: 0.5140049\n", |
| " topicality: 0.00064662547\n", |
| "}\n", |
| ")\n", |
| "('http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg', error {\n", |
| " code: 3\n", |
| " message: \"Bad image data.\"\n", |
| "}\n", |
| ")\n", |
| "('http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg', error {\n", |
| " code: 3\n", |
| " message: \"Bad image data.\"\n", |
| "}\n", |
| ")\n", |
| "('http://farm6.staticflickr.com/5207/5304302785_7b5f763190_z.jpg', error {\n", |
| " code: 3\n", |
| " message: \"Bad image data.\"\n", |
| "}\n", |
| ")\n", |
| "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", |
| " mid: \"/m/02w3_ws\"\n", |
| " description: \"Personal care\"\n", |
| " score: 0.853392\n", |
| " topicality: 0.00013828959\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/02pkr5\"\n", |
| " description: \"Plumbing fixture\"\n", |
| " score: 0.8383083\n", |
| " topicality: 0.012253191\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0b_zf\"\n", |
| " description: \"Plumbing\"\n", |
| " score: 0.726803\n", |
| " topicality: 0.016276756\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/01j2bj\"\n", |
| " description: \"Bathroom\"\n", |
| " score: 0.72486097\n", |
| " topicality: 0.35419264\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/02jz0l\"\n", |
| " description: \"Tap\"\n", |
| " score: 0.6317307\n", |
| " topicality: 0.00705197\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0130jx\"\n", |
| " description: \"Sink\"\n", |
| " score: 0.5732167\n", |
| " topicality: 0.07520393\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/054_l\"\n", |
| " description: \"Mirror\"\n", |
| " score: 0.5680867\n", |
| " topicality: 0.08497098\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0h8lr5r\"\n", |
| " description: \"Bathroom Sink\"\n", |
| " score: 0.557554\n", |
| " topicality: 0.007725588\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/03jvk\"\n", |
| " description: \"Household hardware\"\n", |
| " score: 0.5140049\n", |
| " topicality: 0.00064662547\n", |
| "}\n", |
| ")\n", |
| "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", |
| " mid: \"/m/02w3_ws\"\n", |
| " description: \"Personal care\"\n", |
| " score: 0.853392\n", |
| " topicality: 0.00013828959\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/02pkr5\"\n", |
| " description: \"Plumbing fixture\"\n", |
| " score: 0.8383083\n", |
| " topicality: 0.012253191\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0b_zf\"\n", |
| " description: \"Plumbing\"\n", |
| " score: 0.726803\n", |
| " topicality: 0.016276756\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/01j2bj\"\n", |
| " description: \"Bathroom\"\n", |
| " score: 0.72486097\n", |
| " topicality: 0.35419264\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/02jz0l\"\n", |
| " description: \"Tap\"\n", |
| " score: 0.6317307\n", |
| " topicality: 0.00705197\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0130jx\"\n", |
| " description: \"Sink\"\n", |
| " score: 0.5732167\n", |
| " topicality: 0.07520393\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/054_l\"\n", |
| " description: \"Mirror\"\n", |
| " score: 0.5680867\n", |
| " topicality: 0.08497098\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/0h8lr5r\"\n", |
| " description: \"Bathroom Sink\"\n", |
| " score: 0.557554\n", |
| " topicality: 0.007725588\n", |
| "}\n", |
| "label_annotations {\n", |
| " mid: \"/m/03jvk\"\n", |
| " description: \"Household hardware\"\n", |
| " score: 0.5140049\n", |
| " topicality: 0.00064662547\n", |
| "}\n", |
| ")\n" |
| ] |
| } |
| ], |
| "source": [ |
| "with beam.Pipeline() as pipeline:\n", |
| " main, failed = (pipeline | \"Create inputs\" >> beam.Create(image_urls)\n", |
| " | \"Read images\" >> beam.Map(read_image)\n", |
| " | \"Inference\" >> RunInference(model_handler=CloudVisionModelHandler()).with_exception_handling()\n", |
| " )\n", |
| " _ = main | \"Print image_url and annotation\" >> beam.Map(print)\n", |
| " _ = failed.failed_inferences | \"Print failed inferences\" >> beam.Map(print)" |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "7gwn5bF1XaDm" |
| }, |
| "source": [ |
| "## Monitor the pipeline\n", |
| "\n", |
| "Because monitoring can provide insight into the status and health of the application, consider monitoring and measuring pipeline performance.\n", |
| "For information about the available tracking metrics, see [RunInference Metrics](https://beam.apache.org/documentation/ml/runinference-metrics/)." |
| ] |
| }, |
| { |
| "cell_type": "markdown", |
| "metadata": { |
| "id": "TcBFS0rluusJ" |
| }, |
| "source": [] |
| } |
| ], |
| "metadata": { |
| "colab": { |
| "provenance": [] |
| }, |
| "kernelspec": { |
| "display_name": "Python 3", |
| "language": "python", |
| "name": "python3" |
| }, |
| "language_info": { |
| "name": "python", |
| "version": "3.10.7 (main, Dec 7 2022, 13:34:16) [Clang 14.0.0 (clang-1400.0.29.102)]" |
| }, |
| "vscode": { |
| "interpreter": { |
| "hash": "40c55305dca37c951f6b497e2e996ca59c449c4502b9f8a4515c118ec923845d" |
| } |
| } |
| }, |
| "nbformat": 4, |
| "nbformat_minor": 0 |
| } |