Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 3 additions & 2 deletions sdks/python/apache_beam/ml/inference/gemini_inference.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
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
Expand Down Expand Up @@ -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(
Expand All @@ -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,
Expand Down
37 changes: 37 additions & 0 deletions sdks/python/apache_beam/ml/rag/embeddings/base_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
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


Expand Down Expand Up @@ -89,5 +90,41 @@ def test_adapter_output_conversion(self):
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()
140 changes: 131 additions & 9 deletions sdks/python/apache_beam/ml/rag/embeddings/huggingface.py
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand All @@ -31,19 +37,35 @@
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 "
Expand Down Expand Up @@ -73,3 +95,103 @@ def get_ptransform_for_processing(
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)
Loading
Loading