This is an automated email from the ASF dual-hosted git repository.

cvandermerwe pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new cecc2a6ae0c Add image embedding support to ml/rag (#37628)
cecc2a6ae0c is described below

commit cecc2a6ae0cdb86aba7648ad8a07e4086279beea
Author: claudevdm <[email protected]>
AuthorDate: Wed Feb 18 16:21:02 2026 -0500

    Add image embedding support to ml/rag (#37628)
    
    * image embeddings.
    
    * comments.
    
    * lint.
    
    * Add pillow to default requires.
    
    * update images
    
    * lint
    
    * mypy
    
    ---------
    
    Co-authored-by: Claude <[email protected]>
---
 .../apache_beam/ml/inference/gemini_inference.py   |   5 +-
 .../apache_beam/ml/rag/embeddings/base_test.py     |  37 ++++++
 .../apache_beam/ml/rag/embeddings/huggingface.py   | 140 +++++++++++++++++++--
 .../ml/rag/embeddings/huggingface_test.py          | 135 ++++++++++++++++++++
 .../apache_beam/ml/rag/embeddings/vertex_ai.py     | 122 ++++++++++++++++--
 .../ml/rag/embeddings/vertex_ai_test.py            | 128 +++++++++++++++++++
 .../apache_beam/ml/rag/ingestion/bigquery.py       |  34 ++---
 .../apache_beam/ml/rag/ingestion/mysql_common.py   |   4 +-
 .../ml/rag/ingestion/postgres_common.py            |   4 +-
 .../python/apache_beam/ml/rag/ingestion/spanner.py |   4 +-
 sdks/python/apache_beam/ml/rag/types.py            |  48 ++++++-
 sdks/python/apache_beam/ml/transforms/base_test.py |   4 +-
 .../ml/transforms/embeddings/huggingface_test.py   |   2 +-
 .../transforms/embeddings/tensorflow_hub_test.py   |   2 +-
 .../container/ml/py310/base_image_requirements.txt |  88 ++++++-------
 .../container/ml/py310/gpu_image_requirements.txt  | 120 +++++++++---------
 .../container/ml/py311/base_image_requirements.txt |  90 ++++++-------
 .../container/ml/py311/gpu_image_requirements.txt  | 120 +++++++++---------
 .../container/ml/py312/base_image_requirements.txt |  87 +++++++------
 .../container/ml/py312/gpu_image_requirements.txt  | 116 ++++++++---------
 .../container/ml/py313/base_image_requirements.txt |  93 +++++++-------
 .../container/py310/base_image_requirements.txt    |  73 +++++------
 .../container/py311/base_image_requirements.txt    |  75 +++++------
 .../container/py312/base_image_requirements.txt    |  72 ++++++-----
 .../container/py313/base_image_requirements.txt    |  78 ++++++------
 sdks/python/setup.py                               |   2 +-
 26 files changed, 1138 insertions(+), 545 deletions(-)

diff --git a/sdks/python/apache_beam/ml/inference/gemini_inference.py 
b/sdks/python/apache_beam/ml/inference/gemini_inference.py
index 8b6d0edfa99..2ba220d0162 100644
--- a/sdks/python/apache_beam/ml/inference/gemini_inference.py
+++ b/sdks/python/apache_beam/ml/inference/gemini_inference.py
@@ -22,6 +22,7 @@ from collections.abc import Sequence
 from typing import Any
 from typing import Optional
 from typing import Union
+from typing import cast
 
 from google import genai
 from google.genai import errors
@@ -73,7 +74,7 @@ def generate_from_string(
       call.
   """
   return model.models.generate_content(
-      model=model_name, contents=batch, **inference_args)
+      model=model_name, contents=cast(Any, batch), **inference_args)
 
 
 def generate_image_from_strings_and_images(
@@ -96,7 +97,7 @@ def generate_image_from_strings_and_images(
       call.
   """
   return model.models.generate_content(
-      model=model_name, contents=batch, **inference_args)
+      model=model_name, contents=cast(Any, batch), **inference_args)
 
 
 class GeminiModelHandler(RemoteModelHandler[Any, PredictionResult,
diff --git a/sdks/python/apache_beam/ml/rag/embeddings/base_test.py 
b/sdks/python/apache_beam/ml/rag/embeddings/base_test.py
index aacdf6004ee..61e211e415c 100644
--- a/sdks/python/apache_beam/ml/rag/embeddings/base_test.py
+++ b/sdks/python/apache_beam/ml/rag/embeddings/base_test.py
@@ -19,6 +19,7 @@ import unittest
 from apache_beam.ml.rag.embeddings.base import create_text_adapter
 from apache_beam.ml.rag.types import Chunk
 from apache_beam.ml.rag.types import Content
+from apache_beam.ml.rag.types import EmbeddableItem
 from apache_beam.ml.rag.types import Embedding
 
 
@@ -89,5 +90,41 @@ class RAGBaseEmbeddingsTest(unittest.TestCase):
     self.assertListEqual(embeddings, expected)
 
 
+class ImageEmbeddableItemTest(unittest.TestCase):
+  def test_from_image_str(self):
+    item = EmbeddableItem.from_image('gs://bucket/img.jpg', id='img1')
+    self.assertEqual(item.content.image, 'gs://bucket/img.jpg')
+    self.assertIsNone(item.content.text)
+    self.assertEqual(item.id, 'img1')
+
+  def test_from_image_bytes(self):
+    data = b'\x89PNG\r\n'
+    item = EmbeddableItem.from_image(data, id='img2')
+    self.assertEqual(item.content.image, data)
+    self.assertIsNone(item.content.text)
+
+  def test_from_image_with_metadata(self):
+    item = EmbeddableItem.from_image(
+        'path/to/img.jpg', id='img3', metadata={'source': 'camera'})
+    self.assertEqual(item.metadata, {'source': 'camera'})
+    self.assertEqual(item.content.image, 'path/to/img.jpg')
+
+
+class ContentStringTest(unittest.TestCase):
+  def test_text_content(self):
+    item = EmbeddableItem(content=Content(text="hello"), id="1")
+    self.assertEqual(item.content_string, "hello")
+
+  def test_image_uri_content(self):
+    item = EmbeddableItem.from_image('gs://bucket/img.jpg', id='img1')
+    self.assertEqual(item.content_string, 'gs://bucket/img.jpg')
+
+  def test_image_bytes_raises(self):
+    item = EmbeddableItem.from_image(b'\x89PNG\r\n', id='img2')
+    with self.assertRaisesRegex(ValueError,
+                                "EmbeddableItem does not contain.*"):
+      item.content_string
+
+
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/ml/rag/embeddings/huggingface.py 
b/sdks/python/apache_beam/ml/rag/embeddings/huggingface.py
index 8cf9298849c..e17a5d66fe9 100644
--- a/sdks/python/apache_beam/ml/rag/embeddings/huggingface.py
+++ b/sdks/python/apache_beam/ml/rag/embeddings/huggingface.py
@@ -16,13 +16,19 @@
 
 """RAG-specific embedding implementations using HuggingFace models."""
 
+import io
+from collections.abc import Sequence
 from typing import Optional
 
 import apache_beam as beam
+from apache_beam.io.filesystems import FileSystems
 from apache_beam.ml.inference.base import RunInference
+from apache_beam.ml.rag.embeddings.base import _add_embedding_fn
 from apache_beam.ml.rag.embeddings.base import create_text_adapter
 from apache_beam.ml.rag.types import EmbeddableItem
 from apache_beam.ml.transforms.base import EmbeddingsManager
+from apache_beam.ml.transforms.base import EmbeddingTypeAdapter
+from apache_beam.ml.transforms.base import _ImageEmbeddingHandler
 from apache_beam.ml.transforms.base import _TextEmbeddingHandler
 from apache_beam.ml.transforms.embeddings.huggingface import 
_SentenceTransformerModelHandler
 
@@ -31,19 +37,35 @@ try:
 except ImportError:
   SentenceTransformer = None
 
+try:
+  from PIL import Image as PILImage
+except ImportError:
+  PILImage = None  # type: ignore[assignment]
+
 
 class HuggingfaceTextEmbeddings(EmbeddingsManager):
   def __init__(
       self, model_name: str, *, max_seq_length: Optional[int] = None, 
**kwargs):
-    """Utilizes huggingface SentenceTransformer embeddings for RAG pipeline.
-
-        Args:
-            model_name: Name of the sentence-transformers model to use
-            max_seq_length: Maximum sequence length for the model
-            **kwargs: Additional arguments passed to 
-            :class:`~apache_beam.ml.transforms.base.EmbeddingsManager`
-            constructor including ModelHandler arguments
-        """
+    """HuggingFace text embeddings for RAG pipelines.
+
+    Args:
+        model_name: Name of the sentence-transformers model to use.
+        max_seq_length: Maximum sequence length for the model.
+        **kwargs: Additional arguments passed to
+            :class:`~apache_beam.ml.transforms.base.EmbeddingsManager`,
+            including:
+
+            - ``load_model_args``: dict passed to
+              ``SentenceTransformer()`` constructor
+              (e.g. ``device``, ``cache_folder``).
+            - ``min_batch_size`` / ``max_batch_size``:
+              Control batching for inference.
+            - ``large_model``: If True, share the model
+              across processes to reduce memory usage.
+            - ``inference_args``: dict passed to
+              ``model.encode()``
+              (e.g. ``normalize_embeddings``).
+    """
     if not SentenceTransformer:
       raise ImportError(
           "sentence-transformers is required to use "
@@ -73,3 +95,103 @@ class HuggingfaceTextEmbeddings(EmbeddingsManager):
     return RunInference(
         model_handler=_TextEmbeddingHandler(self),
         inference_args=self.inference_args).with_output_types(EmbeddableItem)
+
+
+def _extract_images(items: Sequence[EmbeddableItem]) -> list:
+  """Extract images from items and convert to PIL.Image objects.
+
+  Supports raw bytes, local file paths, and remote URIs
+  (e.g. gs://, s3://) via Beam's FileSystems.
+  """
+  images = []
+  for item in items:
+    if not item.content.image:
+      raise ValueError(
+          "Expected image content in "
+          f"{type(item).__name__} {item.id}, "
+          "got None")
+    img_data = item.content.image
+    if isinstance(img_data, bytes):
+      img = PILImage.open(io.BytesIO(img_data))
+    else:
+      with FileSystems.open(img_data, 'rb') as f:
+        img = PILImage.open(f)
+        img.load()
+    images.append(img.convert('RGB'))
+  return images
+
+
+def _create_hf_image_adapter(
+) -> EmbeddingTypeAdapter[EmbeddableItem, EmbeddableItem]:
+  """Creates adapter for HuggingFace image embedding.
+
+  Extracts content.image from EmbeddableItems and converts
+  to PIL.Image objects. Supports both raw bytes and file paths.
+
+  Returns:
+      EmbeddingTypeAdapter for HuggingFace image embedding.
+  """
+  return EmbeddingTypeAdapter(
+      input_fn=_extract_images, output_fn=_add_embedding_fn)
+
+
+class HuggingfaceImageEmbeddings(EmbeddingsManager):
+  def __init__(
+      self, model_name: str, *, max_seq_length: Optional[int] = None, 
**kwargs):
+    """HuggingFace image embeddings for RAG pipelines.
+
+    Generates embeddings for images using sentence-transformers
+    models that support image input (e.g. clip-ViT-B-32).
+
+    Args:
+        model_name: Name of the sentence-transformers model.
+            Must be an image-text model. See
+            
https://www.sbert.net/docs/sentence_transformer/pretrained_models.html#image-text-models
+        max_seq_length: Maximum sequence length for the model
+            if applicable.
+        **kwargs: Additional arguments passed to
+            :class:`~apache_beam.ml.transforms.base.EmbeddingsManager`,
+            including:
+
+            - ``load_model_args``: dict passed to
+              ``SentenceTransformer()`` constructor
+              (e.g. ``device``, ``cache_folder``,
+              ``trust_remote_code``).
+            - ``min_batch_size`` / ``max_batch_size``:
+              Control batching for inference.
+            - ``large_model``: If True, share the model
+              across processes to reduce memory usage.
+            - ``inference_args``: dict passed to
+              ``model.encode()``
+              (e.g. ``normalize_embeddings``).
+    """
+    if not SentenceTransformer:
+      raise ImportError(
+          "sentence-transformers is required to use "
+          "HuggingfaceImageEmbeddings. "
+          "Please install it with `pip install sentence-transformers`.")
+    if not PILImage:
+      raise ImportError(
+          "Pillow is required to use HuggingfaceImageEmbeddings. "
+          "Please install it with `pip install pillow`.")
+    super().__init__(type_adapter=_create_hf_image_adapter(), **kwargs)
+    self.model_name = model_name
+    self.max_seq_length = max_seq_length
+    self.model_class = SentenceTransformer
+
+  def get_model_handler(self):
+    """Returns model handler configured with RAG adapter."""
+    return _SentenceTransformerModelHandler(
+        model_class=self.model_class,
+        max_seq_length=self.max_seq_length,
+        model_name=self.model_name,
+        load_model_args=self.load_model_args,
+        min_batch_size=self.min_batch_size,
+        max_batch_size=self.max_batch_size,
+        large_model=self.large_model)
+
+  def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform:
+    """Returns PTransform for image embedding."""
+    return RunInference(
+        model_handler=_ImageEmbeddingHandler(self),
+        inference_args=self.inference_args).with_output_types(EmbeddableItem)
diff --git a/sdks/python/apache_beam/ml/rag/embeddings/huggingface_test.py 
b/sdks/python/apache_beam/ml/rag/embeddings/huggingface_test.py
index f0b9316dcee..65c21771524 100644
--- a/sdks/python/apache_beam/ml/rag/embeddings/huggingface_test.py
+++ b/sdks/python/apache_beam/ml/rag/embeddings/huggingface_test.py
@@ -16,6 +16,8 @@
 
 """Tests for apache_beam.ml.rag.embeddings.huggingface."""
 
+import io
+import os
 import shutil
 import tempfile
 import unittest
@@ -23,9 +25,12 @@ import unittest
 import pytest
 
 import apache_beam as beam
+from apache_beam.ml.rag.embeddings.huggingface import 
HuggingfaceImageEmbeddings
 from apache_beam.ml.rag.embeddings.huggingface import HuggingfaceTextEmbeddings
+from apache_beam.ml.rag.embeddings.huggingface import _create_hf_image_adapter
 from apache_beam.ml.rag.types import Chunk
 from apache_beam.ml.rag.types import Content
+from apache_beam.ml.rag.types import EmbeddableItem
 from apache_beam.ml.rag.types import Embedding
 from apache_beam.ml.transforms.base import MLTransform
 from apache_beam.testing.test_pipeline import TestPipeline
@@ -38,6 +43,11 @@ try:
   SENTENCE_TRANSFORMERS_AVAILABLE = True
 except ImportError:
   SENTENCE_TRANSFORMERS_AVAILABLE = False
+try:
+  from PIL import Image
+  PIL_AVAILABLE = True
+except ImportError:
+  PIL_AVAILABLE = False
 
 
 def chunk_approximately_equals(expected, actual):
@@ -108,5 +118,130 @@ class HuggingfaceTextEmbeddingsTest(unittest.TestCase):
           embeddings, equal_to(expected, equals_fn=chunk_approximately_equals))
 
 
[email protected]_transformers
[email protected](
+    not SENTENCE_TRANSFORMERS_AVAILABLE or not PIL_AVAILABLE,
+    "sentence-transformers or PIL not available")
+class HuggingfaceImageAdapterTest(unittest.TestCase):
+  def test_image_adapter_missing_content(self):
+    adapter = _create_hf_image_adapter()
+    item = EmbeddableItem(content=Content(), id="no_img")
+    with self.assertRaisesRegex(ValueError, "Expected image content"):
+      adapter.input_fn([item])
+
+  def test_image_adapter_bytes_input(self):
+    adapter = _create_hf_image_adapter()
+    png_bytes = _create_png_bytes()
+    item = EmbeddableItem.from_image(png_bytes, id='img1')
+    images = adapter.input_fn([item])
+    self.assertEqual(len(images), 1)
+
+  def test_image_adapter_path_input(self):
+    adapter = _create_hf_image_adapter()
+    png_bytes = _create_png_bytes()
+    tmpdir = tempfile.mkdtemp()
+    try:
+      img_path = os.path.join(tmpdir, 'test.png')
+      with open(img_path, 'wb') as f:
+        f.write(png_bytes)
+      item = EmbeddableItem.from_image(img_path, id='img2')
+      images = adapter.input_fn([item])
+      self.assertEqual(len(images), 1)
+    finally:
+      shutil.rmtree(tmpdir)
+
+  def test_image_adapter_output(self):
+    adapter = _create_hf_image_adapter()
+    items = [EmbeddableItem.from_image(b'\x89PNG', id='img1')]
+    mock_embeddings = [[0.1, 0.2, 0.3]]
+    result = adapter.output_fn(items, mock_embeddings)
+    self.assertEqual(len(result), 1)
+    self.assertEqual(result[0].embedding.dense_embedding, [0.1, 0.2, 0.3])
+
+
[email protected]_transformers
[email protected](
+    not SENTENCE_TRANSFORMERS_AVAILABLE or not PIL_AVAILABLE,
+    "sentence-transformers or PIL not available")
+class HuggingfaceImageEmbeddingsTest(unittest.TestCase):
+  def setUp(self):
+    self.artifact_location = tempfile.mkdtemp(prefix='hf_image_')
+
+  def tearDown(self) -> None:
+    shutil.rmtree(self.artifact_location)
+
+  def test_image_embedding_pipeline(self):
+    png_bytes = _create_png_bytes()
+    test_items = [
+        EmbeddableItem.from_image(
+            png_bytes, id="img1", metadata={"source": "test"}),
+    ]
+
+    # clip-ViT-B-32 produces 512-dimensional embeddings
+    expected = [
+        EmbeddableItem(
+            id="img1",
+            embedding=Embedding(dense_embedding=[0.0] * 512),
+            metadata={"source": "test"},
+            content=Content(image=png_bytes)),
+    ]
+
+    embedder = HuggingfaceImageEmbeddings(model_name="clip-ViT-B-32")
+
+    with TestPipeline() as p:
+      embeddings = (
+          p
+          | beam.Create(test_items)
+          | MLTransform(write_artifact_location=(
+              self.artifact_location)).with_transform(embedder))
+
+      assert_that(
+          embeddings, equal_to(expected, equals_fn=chunk_approximately_equals))
+
+  def test_image_embedding_pipeline_from_path(self):
+    png_bytes = _create_png_bytes()
+    img_path = os.path.join(self.artifact_location, 'test_img.png')
+    with open(img_path, 'wb') as f:
+      f.write(png_bytes)
+
+    test_items = [
+        EmbeddableItem.from_image(
+            img_path, id="img1", metadata={"source": "test"}),
+    ]
+
+    expected = [
+        EmbeddableItem(
+            id="img1",
+            embedding=Embedding(dense_embedding=[0.0] * 512),
+            metadata={"source": "test"},
+            content=Content(image=img_path)),
+    ]
+
+    artifact_location = tempfile.mkdtemp(prefix='hf_image_path_')
+    embedder = HuggingfaceImageEmbeddings(model_name="clip-ViT-B-32")
+    try:
+      with TestPipeline() as p:
+        embeddings = (
+            p
+            | beam.Create(test_items)
+            | MLTransform(write_artifact_location=(
+                artifact_location)).with_transform(embedder))
+
+        assert_that(
+            embeddings,
+            equal_to(expected, equals_fn=chunk_approximately_equals))
+    finally:
+      shutil.rmtree(artifact_location)
+
+
+def _create_png_bytes():
+  """Create a small valid RGB PNG image."""
+  from PIL import Image
+  img = Image.new('RGB', (10, 10), color=(128, 64, 32))
+  buf = io.BytesIO()
+  img.save(buf, format='PNG')
+  return buf.getvalue()
+
+
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai.py 
b/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai.py
index 8ef98223e69..f71984fd567 100644
--- a/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai.py
+++ b/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai.py
@@ -21,21 +21,27 @@
 
 """RAG-specific embedding implementations using Vertex AI models."""
 
+from collections.abc import Sequence
 from typing import Optional
 
 from google.auth.credentials import Credentials
 
 import apache_beam as beam
 from apache_beam.ml.inference.base import RunInference
+from apache_beam.ml.rag.embeddings.base import _add_embedding_fn
 from apache_beam.ml.rag.embeddings.base import create_text_adapter
 from apache_beam.ml.rag.types import EmbeddableItem
 from apache_beam.ml.transforms.base import EmbeddingsManager
+from apache_beam.ml.transforms.base import EmbeddingTypeAdapter
+from apache_beam.ml.transforms.base import _ImageEmbeddingHandler
 from apache_beam.ml.transforms.base import _TextEmbeddingHandler
 from apache_beam.ml.transforms.embeddings.vertex_ai import DEFAULT_TASK_TYPE
+from apache_beam.ml.transforms.embeddings.vertex_ai import 
_VertexAIImageEmbeddingHandler
 from apache_beam.ml.transforms.embeddings.vertex_ai import 
_VertexAITextEmbeddingHandler
 
 try:
   import vertexai
+  from vertexai.vision_models import Image
 except ImportError:
   vertexai = None  # type: ignore[assignment]
 
@@ -52,18 +58,18 @@ class VertexAITextEmbeddings(EmbeddingsManager):
       credentials: Optional[Credentials] = None,
       **kwargs):
     """Utilizes Vertex AI text embeddings for semantic search and RAG
-        pipelines.
-        
-        Args:
-            model_name: Name of the Vertex AI text embedding model
-            title: Optional title for the text content
-            task_type: Task type for embeddings (default: RETRIEVAL_DOCUMENT)
-            project: GCP project ID
-            location: GCP location
-            credentials: Optional GCP credentials
-            **kwargs: Additional arguments passed to EmbeddingsManager 
including
-            ModelHandler inference_args.
-        """
+    pipelines.
+
+    Args:
+        model_name: Name of the Vertex AI text embedding model
+        title: Optional title for the text content
+        task_type: Task type for embeddings (default: RETRIEVAL_DOCUMENT)
+        project: GCP project ID
+        location: GCP location
+        credentials: Optional GCP credentials
+        **kwargs: Additional arguments passed to
+            :class:`~apache_beam.ml.transforms.base.EmbeddingsManager`.
+    """
     if not vertexai:
       raise ImportError(
           "vertexai is required to use VertexAITextEmbeddings. "
@@ -96,3 +102,95 @@ class VertexAITextEmbeddings(EmbeddingsManager):
     return RunInference(
         model_handler=_TextEmbeddingHandler(self),
         inference_args=self.inference_args).with_output_types(EmbeddableItem)
+
+
+def _extract_images(items: Sequence[EmbeddableItem]) -> list:
+  """Extract images from items and convert to vertexai Image objects."""
+  images = []
+  for item in items:
+    if not item.content.image:
+      raise ValueError(
+          "Expected image content in "
+          f"{type(item).__name__} {item.id}, "
+          "got None")
+    img_data = item.content.image
+    if isinstance(img_data, bytes):
+      images.append(Image(image_bytes=img_data))
+    else:
+      images.append(Image.load_from_file(img_data))
+  return images
+
+
+def _create_image_adapter(
+) -> EmbeddingTypeAdapter[EmbeddableItem, EmbeddableItem]:
+  """Creates adapter for Vertex AI image embedding.
+
+  Extracts content.image from EmbeddableItems and converts
+  to vertexai.vision_models.Image objects. Supports both
+  raw bytes and file paths/URIs.
+
+  Returns:
+      EmbeddingTypeAdapter for Vertex AI image embedding.
+  """
+  return EmbeddingTypeAdapter(
+      input_fn=_extract_images, output_fn=_add_embedding_fn)
+
+
+class VertexAIImageEmbeddings(EmbeddingsManager):
+  def __init__(
+      self,
+      model_name: str,
+      *,
+      dimension: Optional[int] = None,
+      project: Optional[str] = None,
+      location: Optional[str] = None,
+      credentials: Optional[Credentials] = None,
+      **kwargs):
+    """Vertex AI image embeddings for RAG pipelines.
+
+    Generates embeddings for images using Vertex AI
+    multimodal embedding models.
+
+    Args:
+        model_name: Name of the Vertex AI model.
+        dimension: Embedding dimension. Must be one of
+            128, 256, 512, or 1408.
+        project: GCP project ID.
+        location: GCP location.
+        credentials: Optional GCP credentials.
+        **kwargs: Additional arguments passed to
+            :class:`~apache_beam.ml.transforms.base.EmbeddingsManager`.
+    """
+    if not vertexai:
+      raise ImportError(
+          "vertexai is required to use "
+          "VertexAIImageEmbeddings. "
+          "Please install it with "
+          "`pip install google-cloud-aiplatform`")
+
+    if dimension is not None and dimension not in (128, 256, 512, 1408):
+      raise ValueError("dimension must be one of "
+                       "128, 256, 512, or 1408")
+
+    super().__init__(type_adapter=_create_image_adapter(), **kwargs)
+    self.model_name = model_name
+    self.dimension = dimension
+    self.project = project
+    self.location = location
+    self.credentials = credentials
+
+  def get_model_handler(self):
+    """Returns model handler for image embedding."""
+    return _VertexAIImageEmbeddingHandler(
+        model_name=self.model_name,
+        dimension=self.dimension,
+        project=self.project,
+        location=self.location,
+        credentials=self.credentials,
+    )
+
+  def get_ptransform_for_processing(self, **kwargs) -> beam.PTransform:
+    """Returns PTransform for image embedding."""
+    return RunInference(
+        model_handler=_ImageEmbeddingHandler(self),
+        inference_args=self.inference_args).with_output_types(EmbeddableItem)
diff --git a/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai_test.py 
b/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai_test.py
index 435475ffb33..1a479c8e7db 100644
--- a/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai_test.py
+++ b/sdks/python/apache_beam/ml/rag/embeddings/vertex_ai_test.py
@@ -16,13 +16,17 @@
 
 """Tests for apache_beam.ml.rag.embeddings.vertex_ai."""
 
+import os
 import shutil
+import struct
 import tempfile
 import unittest
+import zlib
 
 import apache_beam as beam
 from apache_beam.ml.rag.types import Chunk
 from apache_beam.ml.rag.types import Content
+from apache_beam.ml.rag.types import EmbeddableItem
 from apache_beam.ml.rag.types import Embedding
 from apache_beam.ml.transforms.base import MLTransform
 from apache_beam.testing.test_pipeline import TestPipeline
@@ -33,7 +37,9 @@ from apache_beam.testing.util import equal_to
 try:
   import vertexai  # pylint: disable=unused-import
 
+  from apache_beam.ml.rag.embeddings.vertex_ai import VertexAIImageEmbeddings
   from apache_beam.ml.rag.embeddings.vertex_ai import VertexAITextEmbeddings
+  from apache_beam.ml.rag.embeddings.vertex_ai import _create_image_adapter
   VERTEX_AI_AVAILABLE = True
 except ImportError:
   VERTEX_AI_AVAILABLE = False
@@ -107,5 +113,127 @@ class VertexAITextEmbeddingsTest(unittest.TestCase):
           embeddings, equal_to(expected, equals_fn=chunk_approximately_equals))
 
 
[email protected](
+    not VERTEX_AI_AVAILABLE, "Vertex AI dependencies not available")
+class VertexAIImageAdapterTest(unittest.TestCase):
+  def test_image_adapter_missing_content(self):
+    adapter = _create_image_adapter()
+    item = EmbeddableItem(content=Content(), id="no_img")
+    with self.assertRaisesRegex(ValueError, "Expected image content"):
+      adapter.input_fn([item])
+
+  def test_image_adapter_str_input(self):
+    adapter = _create_image_adapter()
+    item = EmbeddableItem.from_image('gs://bucket/img.jpg', id='img1')
+    images = adapter.input_fn([item])
+    self.assertEqual(len(images), 1)
+
+  def test_image_adapter_bytes_input(self):
+    adapter = _create_image_adapter()
+    png_bytes = _create_png_bytes()
+    item = EmbeddableItem.from_image(png_bytes, id='img2')
+    images = adapter.input_fn([item])
+    self.assertEqual(len(images), 1)
+
+  def test_image_adapter_output(self):
+    adapter = _create_image_adapter()
+    items = [
+        EmbeddableItem.from_image('gs://bucket/img.jpg', id='img1'),
+    ]
+    mock_embeddings = [[0.1, 0.2, 0.3]]
+    result = adapter.output_fn(items, mock_embeddings)
+    self.assertEqual(len(result), 1)
+    self.assertEqual(result[0].embedding.dense_embedding, [0.1, 0.2, 0.3])
+
+
[email protected](
+    not VERTEX_AI_AVAILABLE, "Vertex AI dependencies not available")
+class VertexAIImageEmbeddingsTest(unittest.TestCase):
+  def setUp(self):
+    self.artifact_location = tempfile.mkdtemp(prefix='vertex_ai_img_')
+
+  def tearDown(self) -> None:
+    shutil.rmtree(self.artifact_location)
+
+  def test_image_embedding_pipeline(self):
+    png_bytes = _create_png_bytes()
+
+    test_items = [
+        EmbeddableItem.from_image(
+            png_bytes, id="img1", metadata={"source": "test"}),
+    ]
+
+    # Default dimension for multimodal is 1408
+    expected = [
+        EmbeddableItem(
+            id="img1",
+            embedding=Embedding(dense_embedding=[0.0] * 1408),
+            metadata={"source": "test"},
+            content=Content(image=png_bytes)),
+    ]
+
+    embedder = VertexAIImageEmbeddings(model_name="multimodalembedding@001")
+    with TestPipeline() as p:
+      embeddings = (
+          p
+          | beam.Create(test_items)
+          | MLTransform(write_artifact_location=(
+              self.artifact_location)).with_transform(embedder))
+
+      assert_that(
+          embeddings, equal_to(expected, equals_fn=chunk_approximately_equals))
+
+  def test_image_embedding_pipeline_from_path(self):
+    png_bytes = _create_png_bytes()
+    img_path = os.path.join(self.artifact_location, 'test_img.png')
+    with open(img_path, 'wb') as f:
+      f.write(png_bytes)
+
+    test_items = [
+        EmbeddableItem.from_image(
+            img_path, id="img1", metadata={"source": "test"}),
+    ]
+
+    expected = [
+        EmbeddableItem(
+            id="img1",
+            embedding=Embedding(dense_embedding=[0.0] * 1408),
+            metadata={"source": "test"},
+            content=Content(image=img_path)),
+    ]
+
+    artifact_location = tempfile.mkdtemp(prefix='vertex_ai_img_path_')
+    embedder = VertexAIImageEmbeddings(model_name="multimodalembedding@001")
+    try:
+      with TestPipeline() as p:
+        embeddings = (
+            p
+            | beam.Create(test_items)
+            | MLTransform(write_artifact_location=(
+                artifact_location)).with_transform(embedder))
+
+        assert_that(
+            embeddings,
+            equal_to(expected, equals_fn=chunk_approximately_equals))
+    finally:
+      shutil.rmtree(artifact_location)
+
+
+def _create_png_bytes():
+  """Create a minimal valid 1x1 RGB PNG image (no PIL dependency)."""
+  png_sig = b'\x89PNG\r\n\x1a\n'
+  ihdr_data = struct.pack('>IIBBBBB', 1, 1, 8, 2, 0, 0, 0)
+  ihdr_crc = zlib.crc32(b'IHDR' + ihdr_data)
+  ihdr = struct.pack('>I', 13) + b'IHDR' + ihdr_data + struct.pack(
+      '>I', ihdr_crc)
+  raw = zlib.compress(b'\x00\x00\x00\x00')
+  idat_crc = zlib.crc32(b'IDAT' + raw)
+  idat = struct.pack('>I', len(raw)) + b'IDAT' + raw + struct.pack(
+      '>I', idat_crc)
+  iend_crc = zlib.crc32(b'IEND')
+  iend = struct.pack('>I', 0) + b'IEND' + struct.pack('>I', iend_crc)
+  return png_sig + ihdr + idat + iend
+
+
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/ml/rag/ingestion/bigquery.py 
b/sdks/python/apache_beam/ml/rag/ingestion/bigquery.py
index e955c7856f5..af170992b09 100644
--- a/sdks/python/apache_beam/ml/rag/ingestion/bigquery.py
+++ b/sdks/python/apache_beam/ml/rag/ingestion/bigquery.py
@@ -54,11 +54,11 @@ class SchemaConfig:
           Dict[str, Any] with keys matching
           schema fields.
           Example:
-          >>> def embeddable_to_dict(chunk: EmbeddableItem) -> Dict[str, Any]:
+          >>> def embeddable_to_dict(item: EmbeddableItem) -> Dict[str, Any]:
           ...     return {
-          ...         'id': chunk.id,
-          ...         'embedding': chunk.embedding.dense_embedding,
-          ...         'custom_field': chunk.metadata.get('custom_field')
+          ...         'id': item.id,
+          ...         'embedding': item.embedding.dense_embedding,
+          ...         'custom_field': item.metadata.get('custom_field')
           ...     }
   """
   schema: Dict
@@ -89,10 +89,10 @@ class BigQueryVectorWriterConfig(VectorDatabaseWriteConfig):
       ...       {'name': 'source_url', 'type': 'STRING'}
       ...     ]
       ...   },
-      ...   embeddable_to_dict_fn=lambda chunk: {
-      ...       'id': chunk.id,
-      ...       'embedding': chunk.embedding.dense_embedding,
-      ...       'source_url': chunk.metadata.get('url')
+      ...   embeddable_to_dict_fn=lambda item: {
+      ...       'id': item.id,
+      ...       'embedding': item.embedding.dense_embedding,
+      ...       'source_url': item.metadata.get('url')
       ...   }
       ... )
       >>> config = BigQueryVectorWriterConfig(
@@ -123,16 +123,16 @@ class 
BigQueryVectorWriterConfig(VectorDatabaseWriteConfig):
     return _WriteToBigQueryVectorDatabase(self)
 
 
-def _default_embeddable_to_dict_fn(chunk: EmbeddableItem):
-  if chunk.embedding is None or chunk.embedding.dense_embedding is None:
-    raise ValueError("chunk must contain dense embedding")
+def _default_embeddable_to_dict_fn(item: EmbeddableItem):
+  if item.embedding is None or item.embedding.dense_embedding is None:
+    raise ValueError("EmbeddableItem must contain dense embedding")
   return {
-      'id': chunk.id,
-      'embedding': chunk.embedding.dense_embedding,
-      'content': chunk.content.text,
+      'id': item.id,
+      'embedding': item.embedding.dense_embedding,
+      'content': item.content_string,
       'metadata': [{
           "key": k, "value": str(v)
-      } for k, v in chunk.metadata.items()]
+      } for k, v in item.metadata.items()]
   }
 
 
@@ -174,8 +174,8 @@ class _WriteToBigQueryVectorDatabase(beam.PTransform):
         pcoll
         | "EmbeddableItem to dict" >> beam.Map(embeddable_to_dict_fn)
         | "EmbeddableItem dict to schema'd row" >> beam.Map(
-            lambda chunk_dict: beam_row_from_dict(
-                row=chunk_dict, schema=schema)).with_output_types(
+            lambda embeddable_item_dict: beam_row_from_dict(
+                row=embeddable_item_dict, schema=schema)).with_output_types(
                     RowTypeConstraint.from_fields(
                         get_beam_typehints_from_tableschema(schema)))
         | "Write to BigQuery" >> beam.managed.Write(
diff --git a/sdks/python/apache_beam/ml/rag/ingestion/mysql_common.py 
b/sdks/python/apache_beam/ml/rag/ingestion/mysql_common.py
index 6fb88b637ed..829e95de9f2 100644
--- a/sdks/python/apache_beam/ml/rag/ingestion/mysql_common.py
+++ b/sdks/python/apache_beam/ml/rag/ingestion/mysql_common.py
@@ -215,9 +215,7 @@ class ColumnSpecsBuilder:
           ... )
       """
     def value_fn(chunk: EmbeddableItem) -> Any:
-      if chunk.content.text is None:
-        raise ValueError(f'Expected chunk to contain content. {chunk}')
-      value = chunk.content.text
+      value = chunk.content_string
       return convert_fn(value) if convert_fn else value
 
     self._specs.append(
diff --git a/sdks/python/apache_beam/ml/rag/ingestion/postgres_common.py 
b/sdks/python/apache_beam/ml/rag/ingestion/postgres_common.py
index d789c25bb09..4aa08fc7c49 100644
--- a/sdks/python/apache_beam/ml/rag/ingestion/postgres_common.py
+++ b/sdks/python/apache_beam/ml/rag/ingestion/postgres_common.py
@@ -230,9 +230,7 @@ class ColumnSpecsBuilder:
           ... )
       """
     def value_fn(chunk: EmbeddableItem) -> Any:
-      if chunk.content.text is None:
-        raise ValueError(f'Expected chunk to contain content. {chunk}')
-      value = chunk.content.text
+      value = chunk.content_string
       return convert_fn(value) if convert_fn else value
 
     self._specs.append(
diff --git a/sdks/python/apache_beam/ml/rag/ingestion/spanner.py 
b/sdks/python/apache_beam/ml/rag/ingestion/spanner.py
index 49314ee226c..8e108759721 100644
--- a/sdks/python/apache_beam/ml/rag/ingestion/spanner.py
+++ b/sdks/python/apache_beam/ml/rag/ingestion/spanner.py
@@ -267,9 +267,7 @@ class SpannerColumnSpecsBuilder:
         ... )
     """
     def extract_fn(embeddable: EmbeddableItem) -> str:
-      if embeddable.content.text is None:
-        raise ValueError(f'EmbeddableItem must contain content: {embeddable}')
-      return embeddable.content.text
+      return embeddable.content_string
 
     self._specs.append(
         SpannerColumnSpec(
diff --git a/sdks/python/apache_beam/ml/rag/types.py 
b/sdks/python/apache_beam/ml/rag/types.py
index 6a08bb66051..0de93a35306 100644
--- a/sdks/python/apache_beam/ml/rag/types.py
+++ b/sdks/python/apache_beam/ml/rag/types.py
@@ -36,6 +36,7 @@ from typing import Dict
 from typing import List
 from typing import Optional
 from typing import Tuple
+from typing import Union
 
 
 @dataclass
@@ -44,8 +45,11 @@ class Content:
 
   Args:
       text: Text content to be embedded.
+      image: Image as bytes or path/URI
+          (e.g., 'gs://bucket/img.jpg').
   """
   text: Optional[str] = None
+  image: Optional[Union[bytes, str]] = None
 
 
 @dataclass
@@ -73,8 +77,13 @@ class EmbeddableItem:
           item = EmbeddableItem.from_text(
               "hello world", metadata={'src': 'doc'})
 
+      Image (via factory):
+          item = EmbeddableItem.from_image(
+              'gs://bucket/img.jpg')
+
       Text (direct, equivalent to old Chunk usage):
-          item = EmbeddableItem(content=Content(text="hello"), index=3)
+          item = EmbeddableItem(
+              content=Content(text="hello"), index=3)
 
   Args:
       content: The content to embed.
@@ -113,6 +122,28 @@ class EmbeddableItem:
         metadata=metadata or {},
     )
 
+  @classmethod
+  def from_image(
+      cls,
+      image: Union[bytes, str],
+      *,
+      id: Optional[str] = None,
+      metadata: Optional[Dict[str, Any]] = None,
+  ) -> 'EmbeddableItem':
+    """Create an EmbeddableItem with image content.
+
+    Args:
+        image: Image bytes or path/URI (e.g. GCS path)
+        id: Unique identifier (auto-generated if not
+            provided)
+        metadata: Additional metadata
+    """
+    return cls(
+        content=Content(image=image),
+        id=id or str(uuid.uuid4()),
+        metadata=metadata or {},
+    )
+
   @property
   def dense_embedding(self) -> Optional[List[float]]:
     return self.embedding.dense_embedding if self.embedding else None
@@ -121,6 +152,21 @@ class EmbeddableItem:
   def sparse_embedding(self) -> Optional[Tuple[List[int], List[float]]]:
     return self.embedding.sparse_embedding if self.embedding else None
 
+  @property
+  def content_string(self) -> str:
+    """Returns storable string content for ingestion.
+
+    Falls back through content fields in priority order:
+    text > image URI.
+    """
+    if self.content.text is not None:
+      return self.content.text
+    if isinstance(self.content.image, str):
+      return self.content.image
+    raise ValueError(
+        f'EmbeddableItem does not contain storable string content'
+        f' (text or image URI). {self}')
+
 
 # Backward compatibility alias. Existing code using Chunk continues to work
 # unchanged since Chunk IS EmbeddableItem.
diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py 
b/sdks/python/apache_beam/ml/transforms/base_test.py
index 190381cc2f3..64d0afe955d 100644
--- a/sdks/python/apache_beam/ml/transforms/base_test.py
+++ b/sdks/python/apache_beam/ml/transforms/base_test.py
@@ -51,8 +51,8 @@ try:
   import PIL
   from PIL.Image import Image as PIL_Image
 except ImportError:
-  PIL = None
-  PIL_Image = Any
+  PIL = None  # type: ignore[assignment]
+  PIL_Image = Any  # type: ignore[misc, assignment]
 
 try:
 
diff --git 
a/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py 
b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py
index a2358c54478..5e56dc71054 100644
--- a/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py
+++ b/sdks/python/apache_beam/ml/transforms/embeddings/huggingface_test.py
@@ -53,7 +53,7 @@ except ImportError:
 try:
   from PIL import Image
 except ImportError:
-  Image = None
+  Image = None  # type: ignore[assignment]
 
 _HF_TOKEN = os.environ.get('HF_INFERENCE_TOKEN')
 test_query = "This is a test"
diff --git 
a/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py 
b/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py
index 0a4f8c8275c..b9a14ddb8c5 100644
--- a/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py
+++ b/sdks/python/apache_beam/ml/transforms/embeddings/tensorflow_hub_test.py
@@ -52,7 +52,7 @@ try:
   from apache_beam.ml.transforms.embeddings.tensorflow_hub import 
TensorflowHubImageEmbeddings
 except ImportError:
   TensorflowHubImageEmbeddings = None  # type: ignore
-  Image = None
+  Image = None  # type: ignore[assignment]
 
 
 @unittest.skipIf(
diff --git a/sdks/python/container/ml/py310/base_image_requirements.txt 
b/sdks/python/container/ml/py310/base_image_requirements.txt
index 673e63f0ac9..9817ce616a6 100644
--- a/sdks/python/container/ml/py310/base_image_requirements.txt
+++ b/sdks/python/container/ml/py310/base_image_requirements.txt
@@ -21,7 +21,8 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -33,19 +34,21 @@ astunparse==1.6.3
 async-timeout==5.0.1
 attrs==25.4.0
 backports.tarfile==1.2.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b7
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -57,44 +60,44 @@ exceptiongroup==1.3.1
 execnet==2.1.2
 fastavro==1.12.1
 fasteners==0.20
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -107,9 +110,9 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -121,15 +124,14 @@ jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
 joblib==1.5.3
-Js2Py==0.74
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.12.0
+keras==3.12.1
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 libclang==18.1.1
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
@@ -139,7 +141,7 @@ mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
 mpmath==1.3.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.4.2
 nltk==3.9.2
@@ -152,34 +154,34 @@ opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pillow==12.1.0
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
+proto-plus==1.27.1
+protobuf==5.29.6
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
-pyjsparser==2.7.1
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -187,13 +189,14 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
+rich==14.3.2
 rpds-py==0.30.0
 rsa==4.9.1
 safetensors==0.7.0
@@ -201,12 +204,12 @@ scikit-learn==1.7.2
 scipy==1.15.3
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 sympy==1.14.0
@@ -216,25 +219,24 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tokenizers==0.21.4
 tomli==2.4.0
 torch==2.8.0+cpu
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
-tzlocal==5.3.1
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/ml/py310/gpu_image_requirements.txt 
b/sdks/python/container/ml/py310/gpu_image_requirements.txt
index 553de91f9ef..f43dd2745ab 100644
--- a/sdks/python/container/ml/py310/gpu_image_requirements.txt
+++ b/sdks/python/container/ml/py310/gpu_image_requirements.txt
@@ -21,7 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -41,7 +41,7 @@ betterproto==2.0.0b7
 blake3==1.0.8
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 cbor2==5.8.0
 certifi==2026.1.4
 cffi==2.0.0
@@ -51,8 +51,10 @@ cloud-sql-python-connector==1.20.0
 cloudpickle==3.1.2
 compressed-tensors==0.10.2
 crcmod==1.7
-cryptography==46.0.3
-cupy-cuda12x==13.6.0
+cryptography==46.0.5
+cuda-bindings==12.9.4
+cuda-pathfinder==1.3.4
+cupy-cuda12x==14.0.0
 Cython==3.2.4
 depyf==0.19.0
 dill==0.3.1.1
@@ -61,57 +63,56 @@ distro==1.9.0
 dnspython==2.8.0
 docker==7.1.0
 docstring_parser==0.17.0
-einops==0.8.1
+einops==0.8.2
 email-validator==2.3.0
 envoy-data-plane==0.2.6
 exceptiongroup==1.3.1
 execnet==2.1.2
-fastapi==0.128.0
-fastapi-cli==0.0.20
-fastapi-cloud-cli==0.11.0
+fastapi==0.129.0
+fastapi-cli==0.0.23
+fastapi-cloud-cli==0.13.0
 fastar==0.8.0
 fastavro==1.12.1
 fasteners==0.20
-fastrlock==0.8.3
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 gguf==0.17.1
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -124,10 +125,10 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httptools==0.7.1
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -139,12 +140,12 @@ jaraco.context==6.1.0
 jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
-jiter==0.12.0
+jiter==0.13.0
 joblib==1.5.3
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.12.0
+keras==3.12.1
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 lark==1.2.2
@@ -152,12 +153,12 @@ libclang==18.1.1
 llguidance==0.7.30
 llvmlite==0.44.0
 lm-format-enforcer==0.10.12
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
 milvus-lite==2.5.1
-mistral_common==1.8.8
+mistral_common==1.9.1
 ml_dtypes==0.5.4
 mmh3==5.2.0
 mock==5.2.0
@@ -165,7 +166,7 @@ more-itertools==10.8.0
 mpmath==1.3.0
 msgpack==1.1.2
 msgspec==0.20.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.4.2
 ninja==1.13.0
@@ -185,56 +186,55 @@ nvidia-cusparse-cu12==12.5.4.2
 nvidia-cusparselt-cu12==0.6.3
 nvidia-nccl-cu12==2.26.2
 nvidia-nvjitlink-cu12==12.6.85
-nvidia-nvshmem-cu12==3.3.20
+nvidia-nvshmem-cu12==3.4.5
 nvidia-nvtx-cu12==12.6.77
 oauth2client==4.1.3
 objsize==0.7.1
 openai==1.107.1
 openai-harmony==0.0.8
-opencv-python-headless==4.13.0.90
+opencv-python-headless==4.13.0.92
 opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
+oracledb==3.4.2
+orjson==3.11.7
 outlines_core==0.2.10
-overrides==7.7.0
-packaging==25.0
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 partial-json-parser==0.2.1.1.post7
 pg8000==1.31.5
-pillow==12.1.0
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
 prometheus-fastapi-instrumentator==7.1.0
 prometheus_client==0.24.1
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
-psutil==7.2.1
+proto-plus==1.27.1
+protobuf==5.29.6
+psutil==7.2.2
 psycopg2-binary==2.9.11
 py-cpuinfo==9.0.0
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
 pybase64==1.4.3
-pycountry==24.6.1
-pycparser==2.23
+pycountry==26.2.16
+pycparser==3.0
 pydantic==2.12.5
 pydantic-extra-types==2.11.0
-pydantic-settings==2.12.0
+pydantic-settings==2.13.0
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -242,18 +242,18 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-json-logger==4.0.0
-python-multipart==0.0.21
+python-multipart==0.0.22
 python-tds==1.17.1
 pytz==2025.2
 PyYAML==6.0.3
 pyzmq==27.1.0
-ray==2.53.0
+ray==2.54.0
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
-rich-toolkit==0.17.1
+rich==14.3.2
+rich-toolkit==0.19.4
 rignore==0.7.6
 rpds-py==0.30.0
 rsa==4.9.1
@@ -263,9 +263,9 @@ scipy==1.15.3
 scramp==1.4.8
 SecretStorage==3.5.0
 sentencepiece==0.2.1
-sentry-sdk==2.50.0
+sentry-sdk==2.53.0
 setproctitle==1.3.7
-setuptools==80.9.0
+setuptools==82.0.0
 shellingham==1.5.4
 six==1.17.0
 sniffio==1.3.1
@@ -273,10 +273,10 @@ sortedcontainers==2.4.0
 soundfile==0.13.1
 soupsieve==2.8.3
 soxr==1.0.0
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
-starlette==0.50.0
+starlette==0.52.1
 sympy==1.14.0
 tenacity==8.5.0
 tensorboard==2.20.0
@@ -284,7 +284,7 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tiktoken==0.12.0
 tokenizers==0.21.4
@@ -292,25 +292,25 @@ tomli==2.4.0
 torch==2.7.1
 torchaudio==2.7.1
 torchvision==0.22.1
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 triton==3.3.1
-typer==0.21.1
+typer==0.24.0
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
-uvicorn==0.40.0
+uvicorn==0.41.0
 uvloop==0.22.1
 virtualenv-clone==0.5.7
 vllm==0.10.1.1
 watchfiles==1.1.1
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 xformers==0.0.31
 xgrammar==0.1.21
 yarl==1.22.0
diff --git a/sdks/python/container/ml/py311/base_image_requirements.txt 
b/sdks/python/container/ml/py311/base_image_requirements.txt
index cfdf71ce77b..571cdb28c76 100644
--- a/sdks/python/container/ml/py311/base_image_requirements.txt
+++ b/sdks/python/container/ml/py311/base_image_requirements.txt
@@ -21,7 +21,8 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -32,19 +33,21 @@ asn1crypto==1.5.1
 astunparse==1.6.3
 attrs==25.4.0
 backports.tarfile==1.2.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b7
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -55,44 +58,44 @@ envoy-data-plane==0.2.6
 execnet==2.1.2
 fastavro==1.12.1
 fasteners==0.20
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -105,9 +108,9 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -119,15 +122,14 @@ jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
 joblib==1.5.3
-Js2Py==0.74
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.13.1
+keras==3.13.2
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 libclang==18.1.1
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
@@ -137,11 +139,11 @@ mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
 mpmath==1.3.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.6.1
 nltk==3.9.2
-numpy==2.4.1
+numpy==2.4.2
 oauth2client==4.1.3
 objsize==0.7.1
 opentelemetry-api==1.39.1
@@ -150,34 +152,34 @@ opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pillow==12.1.0
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
+proto-plus==1.27.1
+protobuf==5.29.6
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
-pyjsparser==2.7.1
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -185,13 +187,14 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
+rich==14.3.2
 rpds-py==0.30.0
 rsa==4.9.1
 safetensors==0.7.0
@@ -199,12 +202,12 @@ scikit-learn==1.7.2
 scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 sympy==1.14.0
@@ -214,24 +217,23 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tokenizers==0.21.4
 torch==2.8.0+cpu
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
-tzlocal==5.3.1
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/ml/py311/gpu_image_requirements.txt 
b/sdks/python/container/ml/py311/gpu_image_requirements.txt
index 313cad9147b..f44a8ff9be0 100644
--- a/sdks/python/container/ml/py311/gpu_image_requirements.txt
+++ b/sdks/python/container/ml/py311/gpu_image_requirements.txt
@@ -21,7 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -40,7 +40,7 @@ betterproto==2.0.0b7
 blake3==1.0.8
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 cbor2==5.8.0
 certifi==2026.1.4
 cffi==2.0.0
@@ -50,8 +50,10 @@ cloud-sql-python-connector==1.20.0
 cloudpickle==3.1.2
 compressed-tensors==0.10.2
 crcmod==1.7
-cryptography==46.0.3
-cupy-cuda12x==13.6.0
+cryptography==46.0.5
+cuda-bindings==12.9.4
+cuda-pathfinder==1.3.4
+cupy-cuda12x==14.0.0
 Cython==3.2.4
 depyf==0.19.0
 dill==0.3.1.1
@@ -60,56 +62,55 @@ distro==1.9.0
 dnspython==2.8.0
 docker==7.1.0
 docstring_parser==0.17.0
-einops==0.8.1
+einops==0.8.2
 email-validator==2.3.0
 envoy-data-plane==0.2.6
 execnet==2.1.2
-fastapi==0.128.0
-fastapi-cli==0.0.20
-fastapi-cloud-cli==0.11.0
+fastapi==0.129.0
+fastapi-cli==0.0.23
+fastapi-cloud-cli==0.13.0
 fastar==0.8.0
 fastavro==1.12.1
 fasteners==0.20
-fastrlock==0.8.3
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 gguf==0.17.1
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -122,10 +123,10 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httptools==0.7.1
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -137,12 +138,12 @@ jaraco.context==6.1.0
 jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
-jiter==0.12.0
+jiter==0.13.0
 joblib==1.5.3
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.13.1
+keras==3.13.2
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 lark==1.2.2
@@ -150,12 +151,12 @@ libclang==18.1.1
 llguidance==0.7.30
 llvmlite==0.44.0
 lm-format-enforcer==0.10.12
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
 milvus-lite==2.5.1
-mistral_common==1.8.8
+mistral_common==1.9.1
 ml_dtypes==0.5.4
 mmh3==5.2.0
 mock==5.2.0
@@ -163,7 +164,7 @@ more-itertools==10.8.0
 mpmath==1.3.0
 msgpack==1.1.2
 msgspec==0.20.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.6.1
 ninja==1.13.0
@@ -183,56 +184,55 @@ nvidia-cusparse-cu12==12.5.4.2
 nvidia-cusparselt-cu12==0.6.3
 nvidia-nccl-cu12==2.26.2
 nvidia-nvjitlink-cu12==12.6.85
-nvidia-nvshmem-cu12==3.3.20
+nvidia-nvshmem-cu12==3.4.5
 nvidia-nvtx-cu12==12.6.77
 oauth2client==4.1.3
 objsize==0.7.1
 openai==1.107.1
 openai-harmony==0.0.8
-opencv-python-headless==4.13.0.90
+opencv-python-headless==4.13.0.92
 opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
+oracledb==3.4.2
+orjson==3.11.7
 outlines_core==0.2.10
-overrides==7.7.0
-packaging==25.0
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 partial-json-parser==0.2.1.1.post7
 pg8000==1.31.5
-pillow==12.1.0
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
 prometheus-fastapi-instrumentator==7.1.0
 prometheus_client==0.24.1
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
-psutil==7.2.1
+proto-plus==1.27.1
+protobuf==5.29.6
+psutil==7.2.2
 psycopg2-binary==2.9.11
 py-cpuinfo==9.0.0
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
 pybase64==1.4.3
-pycountry==24.6.1
-pycparser==2.23
+pycountry==26.2.16
+pycparser==3.0
 pydantic==2.12.5
 pydantic-extra-types==2.11.0
-pydantic-settings==2.12.0
+pydantic-settings==2.13.0
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -240,18 +240,18 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-json-logger==4.0.0
-python-multipart==0.0.21
+python-multipart==0.0.22
 python-tds==1.17.1
 pytz==2025.2
 PyYAML==6.0.3
 pyzmq==27.1.0
-ray==2.53.0
+ray==2.54.0
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
-rich-toolkit==0.17.1
+rich==14.3.2
+rich-toolkit==0.19.4
 rignore==0.7.6
 rpds-py==0.30.0
 rsa==4.9.1
@@ -261,9 +261,9 @@ scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
 sentencepiece==0.2.1
-sentry-sdk==2.50.0
+sentry-sdk==2.53.0
 setproctitle==1.3.7
-setuptools==80.9.0
+setuptools==82.0.0
 shellingham==1.5.4
 six==1.17.0
 sniffio==1.3.1
@@ -271,10 +271,10 @@ sortedcontainers==2.4.0
 soundfile==0.13.1
 soupsieve==2.8.3
 soxr==1.0.0
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
-starlette==0.50.0
+starlette==0.52.1
 sympy==1.14.0
 tenacity==8.5.0
 tensorboard==2.20.0
@@ -282,32 +282,32 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tiktoken==0.12.0
 tokenizers==0.21.4
 torch==2.7.1
 torchaudio==2.7.1
 torchvision==0.22.1
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 triton==3.3.1
-typer==0.21.1
+typer==0.24.0
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
-uvicorn==0.40.0
+uvicorn==0.41.0
 uvloop==0.22.1
 virtualenv-clone==0.5.7
 vllm==0.10.1.1
 watchfiles==1.1.1
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 xformers==0.0.31
 xgrammar==0.1.21
 yarl==1.22.0
diff --git a/sdks/python/container/ml/py312/base_image_requirements.txt 
b/sdks/python/container/ml/py312/base_image_requirements.txt
index da889c0526d..fdc80fd0ccb 100644
--- a/sdks/python/container/ml/py312/base_image_requirements.txt
+++ b/sdks/python/container/ml/py312/base_image_requirements.txt
@@ -21,7 +21,8 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -31,19 +32,21 @@ anyio==4.12.1
 asn1crypto==1.5.1
 astunparse==1.6.3
 attrs==25.4.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b7
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -54,44 +57,44 @@ envoy-data-plane==0.2.6
 execnet==2.1.2
 fastavro==1.12.1
 fasteners==0.20
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -104,9 +107,9 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -121,11 +124,11 @@ joblib==1.5.3
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.13.1
+keras==3.13.2
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 libclang==18.1.1
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
@@ -135,11 +138,11 @@ mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
 mpmath==1.3.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.6.1
 nltk==3.9.2
-numpy==2.4.1
+numpy==2.4.2
 oauth2client==4.1.3
 objsize==0.7.1
 opentelemetry-api==1.39.1
@@ -148,25 +151,26 @@ opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pillow==12.1.0
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
+proto-plus==1.27.1
+protobuf==5.29.6
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
@@ -174,7 +178,7 @@ PyHamcrest==2.1.0
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -182,13 +186,14 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
+rich==14.3.2
 rpds-py==0.30.0
 rsa==4.9.1
 safetensors==0.7.0
@@ -196,12 +201,12 @@ scikit-learn==1.7.2
 scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 sympy==1.14.0
@@ -211,11 +216,11 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tokenizers==0.21.4
 torch==2.8.0+cpu
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 typing-inspection==0.4.2
 typing_extensions==4.15.0
@@ -226,8 +231,8 @@ urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/ml/py312/gpu_image_requirements.txt 
b/sdks/python/container/ml/py312/gpu_image_requirements.txt
index 21e0db7540c..a9b13f834d1 100644
--- a/sdks/python/container/ml/py312/gpu_image_requirements.txt
+++ b/sdks/python/container/ml/py312/gpu_image_requirements.txt
@@ -21,7 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -39,7 +39,7 @@ betterproto==2.0.0b7
 blake3==1.0.8
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 cbor2==5.8.0
 certifi==2026.1.4
 cffi==2.0.0
@@ -49,8 +49,10 @@ cloud-sql-python-connector==1.20.0
 cloudpickle==3.1.2
 compressed-tensors==0.10.2
 crcmod==1.7
-cryptography==46.0.3
-cupy-cuda12x==13.6.0
+cryptography==46.0.5
+cuda-bindings==12.9.4
+cuda-pathfinder==1.3.4
+cupy-cuda12x==14.0.0
 Cython==3.2.4
 depyf==0.19.0
 dill==0.3.1.1
@@ -59,56 +61,55 @@ distro==1.9.0
 dnspython==2.8.0
 docker==7.1.0
 docstring_parser==0.17.0
-einops==0.8.1
+einops==0.8.2
 email-validator==2.3.0
 envoy-data-plane==0.2.6
 execnet==2.1.2
-fastapi==0.128.0
-fastapi-cli==0.0.20
-fastapi-cloud-cli==0.11.0
+fastapi==0.129.0
+fastapi-cli==0.0.23
+fastapi-cloud-cli==0.13.0
 fastar==0.8.0
 fastavro==1.12.1
 fasteners==0.20
-fastrlock==0.8.3
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 gguf==0.17.1
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -121,10 +122,10 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httptools==0.7.1
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -136,12 +137,12 @@ jaraco.context==6.1.0
 jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
-jiter==0.12.0
+jiter==0.13.0
 joblib==1.5.3
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.13.1
+keras==3.13.2
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 lark==1.2.2
@@ -149,12 +150,12 @@ libclang==18.1.1
 llguidance==0.7.30
 llvmlite==0.44.0
 lm-format-enforcer==0.10.12
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
 milvus-lite==2.5.1
-mistral_common==1.8.8
+mistral_common==1.9.1
 ml_dtypes==0.5.4
 mmh3==5.2.0
 mock==5.2.0
@@ -162,7 +163,7 @@ more-itertools==10.8.0
 mpmath==1.3.0
 msgpack==1.1.2
 msgspec==0.20.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.6.1
 ninja==1.13.0
@@ -182,56 +183,55 @@ nvidia-cusparse-cu12==12.5.4.2
 nvidia-cusparselt-cu12==0.6.3
 nvidia-nccl-cu12==2.26.2
 nvidia-nvjitlink-cu12==12.6.85
-nvidia-nvshmem-cu12==3.3.20
+nvidia-nvshmem-cu12==3.4.5
 nvidia-nvtx-cu12==12.6.77
 oauth2client==4.1.3
 objsize==0.7.1
 openai==1.107.1
 openai-harmony==0.0.8
-opencv-python-headless==4.13.0.90
+opencv-python-headless==4.13.0.92
 opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
+oracledb==3.4.2
+orjson==3.11.7
 outlines_core==0.2.10
-overrides==7.7.0
-packaging==25.0
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 partial-json-parser==0.2.1.1.post7
 pg8000==1.31.5
-pillow==12.1.0
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
 prometheus-fastapi-instrumentator==7.1.0
 prometheus_client==0.24.1
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
-psutil==7.2.1
+proto-plus==1.27.1
+protobuf==5.29.6
+psutil==7.2.2
 psycopg2-binary==2.9.11
 py-cpuinfo==9.0.0
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
 pybase64==1.4.3
-pycountry==24.6.1
-pycparser==2.23
+pycountry==26.2.16
+pycparser==3.0
 pydantic==2.12.5
 pydantic-extra-types==2.11.0
-pydantic-settings==2.12.0
+pydantic-settings==2.13.0
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -244,13 +244,13 @@ python-tds==1.17.1
 pytz==2025.2
 PyYAML==6.0.3
 pyzmq==27.1.0
-ray==2.53.0
+ray==2.54.0
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
-rich-toolkit==0.17.1
+rich==14.3.2
+rich-toolkit==0.19.4
 rignore==0.7.6
 rpds-py==0.30.0
 rsa==4.9.1
@@ -260,7 +260,7 @@ scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
 sentencepiece==0.2.1
-sentry-sdk==2.50.0
+sentry-sdk==2.53.0
 setproctitle==1.3.7
 setuptools==79.0.1
 shellingham==1.5.4
@@ -270,10 +270,10 @@ sortedcontainers==2.4.0
 soundfile==0.13.1
 soupsieve==2.8.3
 soxr==1.0.0
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
-starlette==0.50.0
+starlette==0.52.1
 sympy==1.14.0
 tenacity==8.5.0
 tensorboard==2.20.0
@@ -281,32 +281,32 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tiktoken==0.12.0
 tokenizers==0.21.4
 torch==2.7.1
 torchaudio==2.7.1
 torchvision==0.22.1
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 triton==3.3.1
-typer==0.21.1
+typer==0.24.0
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
-uvicorn==0.40.0
+uvicorn==0.41.0
 uvloop==0.22.1
 virtualenv-clone==0.5.7
 vllm==0.10.1.1
 watchfiles==1.1.1
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 xformers==0.0.31
 xgrammar==0.1.21
 yarl==1.22.0
diff --git a/sdks/python/container/ml/py313/base_image_requirements.txt 
b/sdks/python/container/ml/py313/base_image_requirements.txt
index e1d53b9b894..517110ec815 100644
--- a/sdks/python/container/ml/py313/base_image_requirements.txt
+++ b/sdks/python/container/ml/py313/base_image_requirements.txt
@@ -21,7 +21,8 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
-absl-py==2.3.1
+absl-py==2.4.0
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -31,19 +32,21 @@ anyio==4.12.1
 asn1crypto==1.5.1
 astunparse==1.6.3
 attrs==25.4.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b6
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -54,47 +57,47 @@ envoy_data_plane==1.0.3
 execnet==2.1.2
 fastavro==1.12.1
 fasteners==0.20
-filelock==3.20.3
+filelock==3.24.2
 flatbuffers==25.12.19
 freezegun==1.5.5
 frozenlist==1.8.0
-fsspec==2026.1.0
+fsspec==2026.2.0
 future==1.0.0
 gast==0.7.0
 google-api-core==2.29.0
 google-apitools==0.5.35
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
-google-cloud-pubsub==2.34.0
+google-cloud-monitoring==2.29.1
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-pasta==0.2.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
-grpcio==1.76.0
-grpcio-status==1.76.0
-grpcio-tools==1.76.0
+grpcio==1.78.0
+grpcio-status==1.78.0
+grpcio-tools==1.78.0
 grpclib==0.4.9
 guppy3==3.1.6
 h11==0.16.0
@@ -103,9 +106,9 @@ h5py==3.15.1
 hf-xet==1.2.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
-huggingface-hub==0.36.0
+huggingface_hub==0.36.2
 hyperframe==6.1.0
 hypothesis==6.148.3
 idna==3.11
@@ -120,11 +123,11 @@ joblib==1.5.3
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
-keras==3.13.1
+keras==3.13.2
 keyring==25.7.0
 keyrings.google-artifactregistry-auth==1.1.2
 libclang==18.1.1
-Markdown==3.10
+Markdown==3.10.2
 markdown-it-py==4.0.0
 MarkupSafe==3.0.3
 mdurl==0.1.2
@@ -133,11 +136,11 @@ mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
 mpmath==1.3.0
-multidict==6.7.0
+multidict==6.7.1
 namex==0.1.0
 networkx==3.6.1
 nltk==3.9.2
-numpy==2.4.1
+numpy==2.4.2
 oauth2client==4.1.3
 objsize==0.7.1
 opentelemetry-api==1.39.1
@@ -146,33 +149,34 @@ opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
 opt_einsum==3.4.0
 optree==0.18.0
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pillow==12.1.0
-pip==26.0
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==6.33.4
+proto-plus==1.27.1
+protobuf==6.33.5
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
-pymilvus==2.6.6
+pymilvus==2.6.9
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -180,13 +184,14 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
 regex==2026.1.15
 requests==2.32.5
 requests-mock==1.12.1
-rich==14.2.0
+rich==14.3.2
 rpds-py==0.30.0
 rsa==4.9.1
 safetensors==0.7.0
@@ -194,12 +199,12 @@ scikit-learn==1.7.2
 scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 sympy==1.14.0
@@ -209,11 +214,11 @@ tensorboard-data-server==0.7.2
 tensorflow==2.20.0
 tensorflow-cpu-aws==2.20.0;platform_machine=="aarch64"
 termcolor==3.3.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tokenizers==0.21.4
 torch==2.8.0+cpu
-tqdm==4.67.1
+tqdm==4.67.3
 transformers==4.55.4
 typing-inspection==0.4.2
 typing_extensions==4.15.0
@@ -222,8 +227,8 @@ urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
 Werkzeug==3.1.5
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/py310/base_image_requirements.txt 
b/sdks/python/container/py310/base_image_requirements.txt
index 61c827e0963..1f78a4935f7 100644
--- a/sdks/python/container/py310/base_image_requirements.txt
+++ b/sdks/python/container/py310/base_image_requirements.txt
@@ -21,6 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -31,19 +32,21 @@ asn1crypto==1.5.1
 async-timeout==5.0.1
 attrs==25.4.0
 backports.tarfile==1.2.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b7
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -59,35 +62,35 @@ freezegun==1.5.5
 frozenlist==1.8.0
 future==1.0.0
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -98,7 +101,7 @@ h11==0.16.0
 h2==4.3.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
 hyperframe==6.1.0
 hypothesis==6.148.3
@@ -111,7 +114,6 @@ jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
 joblib==1.5.3
-Js2Py==0.74
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
@@ -122,7 +124,7 @@ milvus-lite==2.5.1
 mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
-multidict==6.7.0
+multidict==6.7.1
 nltk==3.9.2
 numpy==2.2.6
 oauth2client==4.1.3
@@ -131,33 +133,34 @@ opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
+proto-plus==1.27.1
+protobuf==5.29.6
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
-pyjsparser==2.7.1
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -165,6 +168,7 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
@@ -177,30 +181,29 @@ scikit-learn==1.7.2
 scipy==1.15.3
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 tenacity==8.5.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
 tomli==2.4.0
-tqdm==4.67.1
+tqdm==4.67.3
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
-tzlocal==5.3.1
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/py311/base_image_requirements.txt 
b/sdks/python/container/py311/base_image_requirements.txt
index 7be8ec58069..e41f72af87e 100644
--- a/sdks/python/container/py311/base_image_requirements.txt
+++ b/sdks/python/container/py311/base_image_requirements.txt
@@ -21,6 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -30,19 +31,21 @@ anyio==4.12.1
 asn1crypto==1.5.1
 attrs==25.4.0
 backports.tarfile==1.2.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b7
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -57,35 +60,35 @@ freezegun==1.5.5
 frozenlist==1.8.0
 future==1.0.0
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -96,7 +99,7 @@ h11==0.16.0
 h2==4.3.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
 hyperframe==6.1.0
 hypothesis==6.148.3
@@ -109,7 +112,6 @@ jaraco.functools==4.4.0
 jeepney==0.9.0
 Jinja2==3.1.6
 joblib==1.5.3
-Js2Py==0.74
 jsonpickle==3.4.2
 jsonschema==4.26.0
 jsonschema-specifications==2025.9.1
@@ -120,42 +122,43 @@ milvus-lite==2.5.1
 mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
-multidict==6.7.0
+multidict==6.7.1
 nltk==3.9.2
-numpy==2.4.1
+numpy==2.4.2
 oauth2client==4.1.3
 objsize==0.7.1
 opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
+proto-plus==1.27.1
+protobuf==5.29.6
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
-pyjsparser==2.7.1
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -163,6 +166,7 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
@@ -175,29 +179,28 @@ scikit-learn==1.7.2
 scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 tenacity==8.5.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
-tqdm==4.67.1
+tqdm==4.67.3
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
-tzlocal==5.3.1
 ujson==5.11.0
 uritemplate==4.2.0
 urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/py312/base_image_requirements.txt 
b/sdks/python/container/py312/base_image_requirements.txt
index f2c9ce2786f..2f7820d01d6 100644
--- a/sdks/python/container/py312/base_image_requirements.txt
+++ b/sdks/python/container/py312/base_image_requirements.txt
@@ -21,6 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -29,19 +30,21 @@ annotated-types==0.7.0
 anyio==4.12.1
 asn1crypto==1.5.1
 attrs==25.4.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b7
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -56,35 +59,35 @@ freezegun==1.5.5
 frozenlist==1.8.0
 future==1.0.0
 google-api-core==2.29.0
-google-api-python-client==2.188.0
+google-api-python-client==2.190.0
 google-apitools==0.5.31
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
+google-cloud-monitoring==2.29.1
 google-cloud-profiler==4.1.0
-google-cloud-pubsub==2.34.0
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
 grpcio==1.65.5
@@ -95,7 +98,7 @@ h11==0.16.0
 h2==4.3.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
 hyperframe==6.1.0
 hypothesis==6.148.3
@@ -118,33 +121,35 @@ milvus-lite==2.5.1
 mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
-multidict==6.7.0
+multidict==6.7.1
 nltk==3.9.2
-numpy==2.4.1
+numpy==2.4.2
 oauth2client==4.1.3
 objsize==0.7.1
 opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==5.29.5
+proto-plus==1.27.1
+protobuf==5.29.6
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
@@ -152,7 +157,7 @@ PyHamcrest==2.1.0
 pymilvus==2.5.18
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -160,6 +165,7 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
@@ -172,18 +178,18 @@ scikit-learn==1.7.2
 scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 tenacity==8.5.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
-tqdm==4.67.1
+tqdm==4.67.3
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
@@ -192,8 +198,8 @@ uritemplate==4.2.0
 urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/container/py313/base_image_requirements.txt 
b/sdks/python/container/py313/base_image_requirements.txt
index 6a919da4bc9..223e962c585 100644
--- a/sdks/python/container/py313/base_image_requirements.txt
+++ b/sdks/python/container/py313/base_image_requirements.txt
@@ -21,6 +21,7 @@
 # https://s.apache.org/beam-python-dev-wiki
 # Reach out to a committer if you need help.
 
+aiodns==4.0.0
 aiofiles==25.1.0
 aiohappyeyeballs==2.6.1
 aiohttp==3.13.3
@@ -29,19 +30,21 @@ annotated-types==0.7.0
 anyio==4.12.1
 asn1crypto==1.5.1
 attrs==25.4.0
+backports.zstd==1.3.0
 beartype==0.22.9
 beautifulsoup4==4.14.3
 betterproto==2.0.0b6
+brotli==1.2.0
 bs4==0.0.2
 build==1.4.0
-cachetools==6.2.4
+cachetools==6.2.6
 certifi==2026.1.4
 cffi==2.0.0
 charset-normalizer==3.4.4
 click==8.3.1
 cloud-sql-python-connector==1.20.0
 crcmod==1.7
-cryptography==46.0.3
+cryptography==46.0.5
 Cython==3.2.4
 dill==0.3.1.1
 distro==1.9.0
@@ -57,44 +60,44 @@ frozenlist==1.8.0
 future==1.0.0
 google-api-core==2.29.0
 google-apitools==0.5.35
-google-auth==2.47.0
+google-auth==2.49.0.dev0
 google-auth-httplib2==0.2.1
-google-cloud-aiplatform==1.133.0
-google-cloud-bigquery==3.40.0
-google-cloud-bigquery-storage==2.36.0
+google-cloud-aiplatform==1.138.0
+google-cloud-bigquery==3.40.1
+google-cloud-bigquery-storage==2.36.1
 google-cloud-bigtable==2.35.0
 google-cloud-build==3.35.0
 google-cloud-core==2.5.0
 google-cloud-datastore==2.23.0
 google-cloud-dlp==3.34.0
-google-cloud-kms==3.9.0
+google-cloud-kms==3.10.0
 google-cloud-language==2.19.0
-google-cloud-monitoring==2.29.0
-google-cloud-pubsub==2.34.0
+google-cloud-monitoring==2.29.1
+google-cloud-pubsub==2.35.0
 google-cloud-recommendations-ai==0.10.18
 google-cloud-resource-manager==1.16.0
 google-cloud-secret-manager==2.26.0
-google-cloud-spanner==3.62.0
+google-cloud-spanner==3.63.0
 google-cloud-storage==2.19.0
 google-cloud-videointelligence==2.18.0
-google-cloud-vision==3.12.0
+google-cloud-vision==3.12.1
 google-crc32c==1.8.0
-google-genai==1.59.0
+google-genai==1.63.0
 google-resumable-media==2.8.0
 googleapis-common-protos==1.72.0
-greenlet==3.3.0
+greenlet==3.3.1
 grpc-google-iam-v1==0.14.3
 grpc-interceptor==0.15.4
-grpcio==1.76.0
-grpcio-status==1.76.0
-grpcio-tools==1.76.0
+grpcio==1.78.0
+grpcio-status==1.78.0
+grpcio-tools==1.78.0
 grpclib==0.4.9
 guppy3==3.1.6
 h11==0.16.0
 h2==4.3.0
 hpack==4.1.0
 httpcore==1.0.9
-httplib2==0.31.1
+httplib2==0.31.2
 httpx==0.28.1
 hyperframe==6.1.0
 hypothesis==6.148.3
@@ -116,41 +119,43 @@ MarkupSafe==3.0.3
 mmh3==5.2.0
 mock==5.2.0
 more-itertools==10.8.0
-multidict==6.7.0
+multidict==6.7.1
 nltk==3.9.2
-numpy==2.4.1
+numpy==2.4.2
 oauth2client==4.1.3
 objsize==0.7.1
 opentelemetry-api==1.39.1
 opentelemetry-resourcedetector-gcp==1.11.0a0
 opentelemetry-sdk==1.39.1
 opentelemetry-semantic-conventions==0.60b1
-oracledb==3.4.1
-orjson==3.11.5
-overrides==7.7.0
-packaging==25.0
+oracledb==3.4.2
+orjson==3.11.7
+packaging==26.0
 pandas==2.2.3
 parameterized==0.9.0
 pg8000==1.31.5
-pip==25.3
+pillow==12.1.1
+pip==26.0.1
 pluggy==1.6.0
+pminit==1.3.0
 propcache==0.4.1
-proto-plus==1.27.0
-protobuf==6.33.4
+proto-plus==1.27.1
+protobuf==6.33.5
 psycopg2-binary==2.9.11
-pyarrow==18.1.0
+pyarrow==23.0.1
 pyarrow-hotfix==0.7
 pyasn1==0.6.2
 pyasn1_modules==0.4.2
-pycparser==2.23
+pycares==5.0.1
+pycparser==3.0
 pydantic==2.12.5
 pydantic_core==2.41.5
 Pygments==2.19.2
 PyHamcrest==2.1.0
-pymilvus==2.6.6
+pymilvus==2.6.9
 pymongo==4.16.0
 PyMySQL==1.1.2
-pyparsing==3.3.1
+pyparsing==3.3.2
 pyproject_hooks==1.2.0
 pytest==8.4.2
 pytest-timeout==2.4.0
@@ -158,6 +163,7 @@ pytest-xdist==3.8.0
 python-dateutil==2.9.0.post0
 python-dotenv==1.2.1
 python-tds==1.17.1
+pythonmonkey==1.3.0
 pytz==2025.2
 PyYAML==6.0.3
 referencing==0.37.0
@@ -170,26 +176,26 @@ scikit-learn==1.7.2
 scipy==1.17.0
 scramp==1.4.8
 SecretStorage==3.5.0
-setuptools==80.9.0
+setuptools==82.0.0
 six==1.17.0
 sniffio==1.3.1
 sortedcontainers==2.4.0
 soupsieve==2.8.3
-SQLAlchemy==2.0.45
+SQLAlchemy==2.0.46
 sqlalchemy_pytds==1.0.2
 sqlparse==0.5.5
 tenacity==8.5.0
-testcontainers==4.14.0
+testcontainers==4.14.1
 threadpoolctl==3.6.0
-tqdm==4.67.1
+tqdm==4.67.3
 typing-inspection==0.4.2
 typing_extensions==4.15.0
 tzdata==2025.3
 urllib3==2.6.3
 virtualenv-clone==0.5.7
 websockets==15.0.1
-wheel==0.45.1
-wrapt==2.0.1
+wheel==0.46.3
+wrapt==2.1.1
 yarl==1.22.0
 zipp==3.23.0
 zstandard==0.25.0
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 8500830a17b..19aab312f27 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -168,7 +168,6 @@ ml_base = [
     'langchain',
     'sentence-transformers>=2.2.2',
     'skl2onnx',
-    'pillow',
     'pyod',
     'tensorflow',
     'tensorflow-hub',
@@ -390,6 +389,7 @@ if __name__ == '__main__':
           'numpy>=1.14.3,<2.5.0',  # Update pyproject.toml as well.
           'objsize>=0.6.1,<0.8.0',
           'packaging>=22.0',
+          'pillow',
           'pymongo>=3.8.0,<5.0.0',
           'proto-plus>=1.7.1,<2',
           # 1. Use a tighter upper bound in protobuf dependency to make sure

Reply via email to